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/26 17:01:50 UTC

[01/13] flink git commit: [FLINK-7908][QS] Restructure the queryable state module.

Repository: flink
Updated Branches:
  refs/heads/master 8595dadb8 -> 2fd8721d0


http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/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 8f803ed..0a4b5bf 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
@@ -49,14 +49,14 @@ import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
 import org.apache.flink.core.testutils.CheckedThread;
 import org.apache.flink.core.testutils.OneShotLatch;
+import org.apache.flink.queryablestate.KvStateID;
+import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer;
 import org.apache.flink.runtime.checkpoint.CheckpointOptions;
 import org.apache.flink.runtime.checkpoint.StateAssignmentOperation;
 import org.apache.flink.runtime.execution.Environment;
 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.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;

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-tests/src/test/java/org/apache/flink/test/completeness/TypeInfoTestCoverageTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/completeness/TypeInfoTestCoverageTest.java b/flink-tests/src/test/java/org/apache/flink/test/completeness/TypeInfoTestCoverageTest.java
index 91566af..95e1598 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/completeness/TypeInfoTestCoverageTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/completeness/TypeInfoTestCoverageTest.java
@@ -50,7 +50,8 @@ public class TypeInfoTestCoverageTest extends TestLogger {
 					typeInfo.getName().contains("Test$") ||
 					typeInfo.getName().contains("TestBase$") ||
 					typeInfo.getName().contains("ITCase$") ||
-					typeInfo.getName().contains("$$anon")) {
+					typeInfo.getName().contains("$$anon") ||
+					typeInfo.getName().contains("queryablestate")) {
 				continue;
 			}
 			boolean found = false;

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 9384a48..e4a49d4 100644
--- a/pom.xml
+++ b/pom.xml
@@ -68,6 +68,7 @@ under the License.
 		<module>flink-connectors</module>
 		<module>flink-examples</module>
 		<module>flink-clients</module>
+		<module>flink-queryable-state</module>
 		<module>flink-tests</module>
 		<module>flink-end-to-end-tests</module>
 		<module>flink-test-utils-parent</module>
@@ -81,7 +82,6 @@ under the License.
 		<module>flink-yarn</module>
 		<module>flink-yarn-tests</module>
 		<module>flink-fs-tests</module>
-		<module>flink-queryable-state</module>
 	</modules>
 
 	<properties>

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/tools/travis_mvn_watchdog.sh
----------------------------------------------------------------------
diff --git a/tools/travis_mvn_watchdog.sh b/tools/travis_mvn_watchdog.sh
index f608e51..b7e3709 100755
--- a/tools/travis_mvn_watchdog.sh
+++ b/tools/travis_mvn_watchdog.sh
@@ -68,7 +68,8 @@ flink-libraries/flink-gelly-examples,\
 flink-libraries/flink-ml,\
 flink-libraries/flink-python,\
 flink-libraries/flink-table,\
-flink-queryable-state/flink-queryable-state-java"
+flink-queryable-state/flink-queryable-state-runtime,\
+flink-queryable-state/flink-queryable-state-client-java"
 
 MODULES_CONNECTORS="\
 flink-contrib/flink-connector-wikiedits,\


[07/13] flink git commit: [FLINK-7908][QS] Restructure the queryable state module.

Posted by kk...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java
deleted file mode 100644
index 4d27da2..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java
+++ /dev/null
@@ -1,1496 +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.itcases;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.functions.AggregateFunction;
-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.AggregatingState;
-import org.apache.flink.api.common.state.AggregatingStateDescriptor;
-import org.apache.flink.api.common.state.FoldingState;
-import org.apache.flink.api.common.state.FoldingStateDescriptor;
-import org.apache.flink.api.common.state.ListState;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.state.MapState;
-import org.apache.flink.api.common.state.MapStateDescriptor;
-import org.apache.flink.api.common.state.ReducingState;
-import org.apache.flink.api.common.state.ReducingStateDescriptor;
-import org.apache.flink.api.common.state.State;
-import org.apache.flink.api.common.state.StateDescriptor;
-import org.apache.flink.api.common.state.ValueState;
-import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.api.common.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.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.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;
-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.VoidNamespaceSerializer;
-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.ProcessFunction;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.Preconditions;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-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.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.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 AbstractQueryableStateTestBase extends TestLogger {
-
-	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);
-
-	private final ScheduledExecutorService executorService = Executors.newScheduledThreadPool(4);
-	private final ScheduledExecutor executor = new ScheduledExecutorServiceAdapter(executorService);
-
-	/**
-	 * 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;
-
-	/**
-	 * Client shared between all the test.
-	 */
-	protected static QueryableStateClient client;
-
-	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;
-
-		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, 1000L));
-
-			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";
-
-			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<CompletableFuture<ReducingState<Tuple2<Integer, Long>>>> futures = new ArrayList<>(numKeys);
-
-				for (int i = 0; i < numKeys; i++) {
-					final int key = i;
-
-					if (counts.get(key) > 0L) {
-						// Skip this one
-						continue;
-					} else {
-						allNonZero = false;
-					}
-
-					CompletableFuture<ReducingState<Tuple2<Integer, Long>>> result = getKvStateWithRetries(
-							client,
-							jobId,
-							queryName,
-							key,
-							BasicTypeInfo.INT_TYPE_INFO,
-							reducingState,
-							QUERY_RETRY_DELAY,
-							false,
-							executor);
-
-					result.thenAccept(response -> {
-						try {
-							Tuple2<Integer, Long> res = response.get();
-							counts.set(key, res.f1);
-							assertEquals("Key mismatch", key, res.f0.intValue());
-						} catch (Exception e) {
-							Assert.fail(e.getMessage());
-						}
-					});
-
-					futures.add(result);
-				}
-
-				// 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);
-
-			// 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) {
-				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
-						.getLeaderGateway(deadline.timeLeft())
-						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
-
-				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-			}
-		}
-	}
-
-	/**
-	 * 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, 1000L));
-
-			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();
-
-			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 =
-					failedFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-			assertEquals(JobStatus.FAILED, jobStatus.state());
-
-			// Get the job and check the cause
-			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)))
-					.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-
-			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) {
-				scala.concurrent.Future<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 long numElements = 1024L;
-
-		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, 1000L));
-
-			DataStream<Tuple2<Integer, Long>> source = env
-					.addSource(new TestAscendingValueSource(numElements));
-
-			// Value state
-			ValueStateDescriptor<Tuple2<Integer, Long>> valueState = new ValueStateDescriptor<>(
-					"any",
-					source.getType());
-
-			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);
-
-			executeValueQuery(deadline, client, jobId, "hakuna", valueState, numElements);
-		} finally {
-			// Free cluster resources
-			if (jobId != null) {
-				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
-						.getLeaderGateway(deadline.timeLeft())
-						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
-
-				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-			}
-		}
-	}
-
-	/**
-	 * 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 long numElements = 1024L;
-
-		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, 1000L));
-
-			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);
-
-			executeValueQuery(deadline, client, jobId, "hakuna", valueState, expected);
-		} finally {
-			// Free cluster resources
-			if (jobId != null) {
-				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
-						.getLeaderGateway(deadline.timeLeft())
-						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
-
-				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-			}
-		}
-	}
-
-	/**
-	 * 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 UnknownKeyOrNamespaceException} exception.
-	 *
-	 * @throws UnknownKeyOrNamespaceException thrown due querying a non-existent key
-	 */
-	@Test(expected = UnknownKeyOrNamespaceException.class)
-	public void testValueStateDefault() throws Throwable {
-
-		// Config
-		final Deadline deadline = TEST_TIMEOUT.fromNow();
-
-		final long numElements = 1024L;
-
-		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, 1000L));
-
-			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;
-			CompletableFuture<ValueState<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) {
-				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
-						.getLeaderGateway(deadline.timeLeft())
-						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
-
-				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-			}
-		}
-	}
-
-	/**
-	 * 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 long numElements = 1024L;
-
-		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, 1000L));
-
-			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);
-
-			final ValueStateDescriptor<Tuple2<Integer, Long>> stateDesc =
-					(ValueStateDescriptor<Tuple2<Integer, Long>>) queryableState.getStateDescriptor();
-			executeValueQuery(deadline, client, jobId, "matata", stateDesc, numElements);
-		} finally {
-
-			// Free cluster resources
-			if (jobId != null) {
-				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(
-						cluster.getLeaderGateway(deadline.timeLeft())
-								.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-								.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
-
-				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-			}
-		}
-	}
-
-	/**
-	 * 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;
-
-		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, 1000L));
-
-			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) {
-					CompletableFuture<FoldingState<Tuple2<Integer, Long>, String>> future = getKvStateWithRetries(
-							client,
-							jobId,
-							"pumba",
-							key,
-							BasicTypeInfo.INT_TYPE_INFO,
-							foldingState,
-							QUERY_RETRY_DELAY,
-							false,
-							executor);
-
-					String value = future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS).get();
-
-					//assertEquals("Key mismatch", key, value.f0.intValue());
-					if (expected.equals(value)) {
-						success = true;
-					} else {
-						// Retry
-						Thread.sleep(50L);
-					}
-				}
-
-				assertTrue("Did not succeed query", success);
-			}
-		} finally {
-			// Free cluster resources
-			if (jobId != null) {
-				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
-						.getLeaderGateway(deadline.timeLeft())
-						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
-
-				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-			}
-		}
-	}
-
-	/**
-	 * 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 long numElements = 1024L;
-
-		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, 1000L));
-
-			DataStream<Tuple2<Integer, Long>> source = env
-					.addSource(new TestAscendingValueSource(numElements));
-
-			// Reducing state
-			ReducingStateDescriptor<Tuple2<Integer, Long>> reducingState =
-					new ReducingStateDescriptor<>(
-							"any",
-							new SumReduce(),
-							source.getType());
-
-			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);
-
-			// Now query
-			long expected = numElements * (numElements + 1L) / 2L;
-
-			for (int key = 0; key < maxParallelism; key++) {
-				boolean success = false;
-				while (deadline.hasTimeLeft() && !success) {
-					CompletableFuture<ReducingState<Tuple2<Integer, Long>>> future = getKvStateWithRetries(
-							client,
-							jobId,
-							"jungle",
-							key,
-							BasicTypeInfo.INT_TYPE_INFO,
-							reducingState,
-							QUERY_RETRY_DELAY,
-							false,
-							executor);
-
-					Tuple2<Integer, Long> value = future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS).get();
-
-					assertEquals("Key mismatch", key, value.f0.intValue());
-					if (expected == value.f1) {
-						success = true;
-					} else {
-						// Retry
-						Thread.sleep(50L);
-					}
-				}
-
-				assertTrue("Did not succeed query", success);
-			}
-		} finally {
-			// Free cluster resources
-			if (jobId != null) {
-				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
-						.getLeaderGateway(deadline.timeLeft())
-						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
-
-				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-			}
-		}
-	}
-
-	/**
-	 * Tests simple map state queryable state instance. Each source emits
-	 * (subtaskIndex, 0)..(subtaskIndex, numElements) tuples, which are then
-	 * queried. The map state instance sums the values up. The test succeeds
-	 * after each subtask index is queried with result n*(n+1)/2.
-	 */
-	@Test
-	public void testMapState() throws Exception {
-		// Config
-		final Deadline deadline = TEST_TIMEOUT.fromNow();
-
-		final long numElements = 1024L;
-
-		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, 1000L));
-
-			DataStream<Tuple2<Integer, Long>> source = env
-					.addSource(new TestAscendingValueSource(numElements));
-
-			final MapStateDescriptor<Integer, Tuple2<Integer, Long>> mapStateDescriptor = new MapStateDescriptor<>(
-					"timon",
-					BasicTypeInfo.INT_TYPE_INFO,
-					source.getType());
-			mapStateDescriptor.setQueryable("timon-queryable");
-
-			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;
-				}
-			}).process(new ProcessFunction<Tuple2<Integer, Long>, Object>() {
-				private static final long serialVersionUID = -805125545438296619L;
-
-				private transient MapState<Integer, Tuple2<Integer, Long>> mapState;
-
-				@Override
-				public void open(Configuration parameters) throws Exception {
-					super.open(parameters);
-					mapState = getRuntimeContext().getMapState(mapStateDescriptor);
-				}
-
-				@Override
-				public void processElement(Tuple2<Integer, Long> value, Context ctx, Collector<Object> out) throws Exception {
-					Tuple2<Integer, Long> v = mapState.get(value.f0);
-					if (v == null) {
-						v = new Tuple2<>(value.f0, 0L);
-					}
-					mapState.put(value.f0, new Tuple2<>(v.f0, v.f1 + value.f1));
-				}
-			});
-
-			// Submit the job graph
-			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
-			jobId = jobGraph.getJobID();
-
-			cluster.submitJobDetached(jobGraph);
-
-			// Now query
-			long expected = numElements * (numElements + 1L) / 2L;
-
-			for (int key = 0; key < maxParallelism; key++) {
-				boolean success = false;
-				while (deadline.hasTimeLeft() && !success) {
-					CompletableFuture<MapState<Integer, Tuple2<Integer, Long>>> future = getKvStateWithRetries(
-							client,
-							jobId,
-							"timon-queryable",
-							key,
-							BasicTypeInfo.INT_TYPE_INFO,
-							mapStateDescriptor,
-							QUERY_RETRY_DELAY,
-							false,
-							executor);
-
-					Tuple2<Integer, Long> value = future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS).get(key);
-					assertEquals("Key mismatch", key, value.f0.intValue());
-					if (expected == value.f1) {
-						success = true;
-					} else {
-						// Retry
-						Thread.sleep(50L);
-					}
-				}
-
-				assertTrue("Did not succeed query", success);
-			}
-		} finally {
-			// Free cluster resources
-			if (jobId != null) {
-				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
-						.getLeaderGateway(deadline.timeLeft())
-						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
-
-				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-			}
-		}
-	}
-
-	/**
-	 * Tests simple list state queryable state instance. Each source emits
-	 * (subtaskIndex, 0)..(subtaskIndex, numElements) tuples, which are then
-	 * queried. The list state instance add the values to the list. The test
-	 * succeeds after each subtask index is queried and the list contains
-	 * the correct number of distinct elements.
-	 */
-	@Test
-	public void testListState() throws Exception {
-		// Config
-		final Deadline deadline = TEST_TIMEOUT.fromNow();
-
-		final long numElements = 1024L;
-
-		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, 1000L));
-
-			DataStream<Tuple2<Integer, Long>> source = env
-					.addSource(new TestAscendingValueSource(numElements));
-
-			final ListStateDescriptor<Long> listStateDescriptor = new ListStateDescriptor<Long>(
-					"list",
-					BasicTypeInfo.LONG_TYPE_INFO);
-			listStateDescriptor.setQueryable("list-queryable");
-
-			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;
-				}
-			}).process(new ProcessFunction<Tuple2<Integer, Long>, Object>() {
-				private static final long serialVersionUID = -805125545438296619L;
-
-				private transient ListState<Long> listState;
-
-				@Override
-				public void open(Configuration parameters) throws Exception {
-					super.open(parameters);
-					listState = getRuntimeContext().getListState(listStateDescriptor);
-				}
-
-				@Override
-				public void processElement(Tuple2<Integer, Long> value, Context ctx, Collector<Object> out) throws Exception {
-					listState.add(value.f1);
-				}
-			});
-
-			// Submit the job graph
-			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
-			jobId = jobGraph.getJobID();
-
-			cluster.submitJobDetached(jobGraph);
-
-			// Now query
-
-			Map<Integer, Set<Long>> results = new HashMap<>();
-
-			for (int key = 0; key < maxParallelism; key++) {
-				boolean success = false;
-				while (deadline.hasTimeLeft() && !success) {
-					CompletableFuture<ListState<Long>> future = getKvStateWithRetries(
-							client,
-							jobId,
-							"list-queryable",
-							key,
-							BasicTypeInfo.INT_TYPE_INFO,
-							listStateDescriptor,
-							QUERY_RETRY_DELAY,
-							false,
-							executor);
-
-					Iterable<Long> value = future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS).get();
-
-					Set<Long> res = new HashSet<>();
-					for (Long v: value) {
-						res.add(v);
-					}
-
-					// the source starts at 0, so +1
-					if (res.size() == numElements + 1L) {
-						success = true;
-						results.put(key, res);
-					} else {
-						// Retry
-						Thread.sleep(50L);
-					}
-				}
-
-				assertTrue("Did not succeed query", success);
-			}
-
-			for (int key = 0; key < maxParallelism; key++) {
-				Set<Long> values = results.get(key);
-				for (long i = 0L; i <= numElements; i++) {
-					assertTrue(values.contains(i));
-				}
-			}
-
-		} finally {
-			// Free cluster resources
-			if (jobId != null) {
-				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
-						.getLeaderGateway(deadline.timeLeft())
-						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
-
-				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-			}
-		}
-	}
-
-	@Test
-	public void testAggregatingState() throws Exception {
-		// Config
-		final Deadline deadline = TEST_TIMEOUT.fromNow();
-
-		final long numElements = 1024L;
-
-		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, 1000L));
-
-			DataStream<Tuple2<Integer, Long>> source = env
-					.addSource(new TestAscendingValueSource(numElements));
-
-			final AggregatingStateDescriptor<Tuple2<Integer, Long>, String, String> aggrStateDescriptor =
-					new AggregatingStateDescriptor<>(
-							"aggregates",
-							new SumAggr(),
-							String.class);
-			aggrStateDescriptor.setQueryable("aggr-queryable");
-
-			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;
-				}
-			}).transform(
-					"TestAggregatingOperator",
-					BasicTypeInfo.STRING_TYPE_INFO,
-					new AggregatingTestOperator(aggrStateDescriptor)
-			);
-
-			// Submit the job graph
-			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
-			jobId = jobGraph.getJobID();
-
-			cluster.submitJobDetached(jobGraph);
-
-			// Now query
-
-			for (int key = 0; key < maxParallelism; key++) {
-				boolean success = false;
-				while (deadline.hasTimeLeft() && !success) {
-					CompletableFuture<AggregatingState<Tuple2<Integer, Long>, String>> future = getKvStateWithRetries(
-							client,
-							jobId,
-							"aggr-queryable",
-							key,
-							BasicTypeInfo.INT_TYPE_INFO,
-							aggrStateDescriptor,
-							QUERY_RETRY_DELAY,
-							false,
-							executor);
-
-					String value = future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS).get();
-
-					if (Long.parseLong(value) == numElements * (numElements + 1L) / 2L) {
-						success = true;
-					} else {
-						// Retry
-						Thread.sleep(50L);
-					}
-				}
-
-				assertTrue("Did not succeed query", success);
-			}
-		} finally {
-			// Free cluster resources
-			if (jobId != null) {
-				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
-						.getLeaderGateway(deadline.timeLeft())
-						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
-
-				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-			}
-		}
-	}
-
-	/////				Sources/UDFs Used in the Tests			//////
-
-	/**
-	 * 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) {
-					wait();
-				}
-			}
-		}
-
-		@Override
-		public void cancel() {
-			isRunning = false;
-
-			synchronized (this) {
-				notifyAll();
-			}
-		}
-
-	}
-
-	/**
-	 * Test source producing (key, 1) tuples with random key in key range (numKeys).
-	 */
-	private 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(0L);
-			}
-		}
-
-		@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(1L);
-			}
-		}
-
-		@Override
-		public void cancel() {
-			isRunning = false;
-		}
-
-		@Override
-		public void notifyCheckpointComplete(long checkpointId) throws Exception {
-			if (getRuntimeContext().getIndexOfThisSubtask() == 0) {
-				LATEST_CHECKPOINT_ID.set(checkpointId);
-			}
-		}
-	}
-
-	/**
-	 * An operator that uses {@link AggregatingState}.
-	 *
-	 * <p>The operator exists for lack of possibility to get an
-	 * {@link AggregatingState} from the {@link org.apache.flink.api.common.functions.RuntimeContext}.
-	 * If this were not the case, we could have a {@link ProcessFunction}.
-	 */
-	private static class AggregatingTestOperator
-			extends AbstractStreamOperator<String>
-			implements OneInputStreamOperator<Tuple2<Integer, Long>, String> {
-
-		private static final long serialVersionUID = 1L;
-
-		private final AggregatingStateDescriptor<Tuple2<Integer, Long>, String, String> stateDescriptor;
-		private transient AggregatingState<Tuple2<Integer, Long>, String> state;
-
-		AggregatingTestOperator(AggregatingStateDescriptor<Tuple2<Integer, Long>, String, String> stateDesc) {
-			this.stateDescriptor = stateDesc;
-		}
-
-		@Override
-		public void open() throws Exception {
-			super.open();
-			this.state = getKeyedStateBackend().getPartitionedState(
-					VoidNamespace.INSTANCE,
-					VoidNamespaceSerializer.INSTANCE,
-					stateDescriptor);
-		}
-
-		@Override
-		public void processElement(StreamRecord<Tuple2<Integer, Long>> element) throws Exception {
-			state.add(element.getValue());
-		}
-	}
-
-	/**
-	 * Test {@link AggregateFunction} concatenating the already stored string with the long passed as argument.
-	 */
-	private static class SumAggr implements AggregateFunction<Tuple2<Integer, Long>, String, String> {
-
-		private static final long serialVersionUID = -6249227626701264599L;
-
-		@Override
-		public String createAccumulator() {
-			return "0";
-		}
-
-		@Override
-		public String add(Tuple2<Integer, Long> value, String accumulator) {
-			long acc = Long.valueOf(accumulator);
-			acc += value.f1;
-			return Long.toString(acc);
-		}
-
-		@Override
-		public String getResult(String accumulator) {
-			return accumulator;
-		}
-
-		@Override
-		public String merge(String a, String b) {
-			return Long.toString(Long.valueOf(a) + Long.valueOf(b));
-		}
-	}
-
-	/**
-	 * 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;
-		}
-	}
-
-	/////				General Utility Methods				//////
-
-	private static <K, S extends State, V> CompletableFuture<S> getKvStateWithRetries(
-			final QueryableStateClient client,
-			final JobID jobId,
-			final String queryName,
-			final K key,
-			final TypeInformation<K> keyTypeInfo,
-			final StateDescriptor<S, 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));
-		}
-	}
-
-	/**
-	 * 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 executeValueQuery(
-			final Deadline deadline,
-			final QueryableStateClient client,
-			final JobID jobId,
-			final String queryableStateName,
-			final ValueStateDescriptor<Tuple2<Integer, Long>> stateDescriptor,
-			final long expected) throws Exception {
-
-		for (int key = 0; key < maxParallelism; key++) {
-			boolean success = false;
-			while (deadline.hasTimeLeft() && !success) {
-				CompletableFuture<ValueState<Tuple2<Integer, Long>>> future = getKvStateWithRetries(
-						client,
-						jobId,
-						queryableStateName,
-						key,
-						BasicTypeInfo.INT_TYPE_INFO,
-						stateDescriptor,
-						QUERY_RETRY_DELAY,
-						false,
-						executor);
-
-				Tuple2<Integer, Long> value = future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS).value();
-
-				assertEquals("Key mismatch", key, value.f0.intValue());
-				if (expected == value.f1) {
-					success = true;
-				} else {
-					// Retry
-					Thread.sleep(50L);
-				}
-			}
-
-			assertTrue("Did not succeed query", success);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java
deleted file mode 100644
index ab75cf4..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.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.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.queryablestate.client.QueryableStateClient;
-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.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 HAAbstractQueryableStateTestBase extends AbstractQueryableStateTestBase {
-
-	private static final int NUM_JMS = 2;
-	private static final int NUM_TMS = 2;
-	private static final int NUM_SLOTS_PER_TM = 4;
-
-	private static TestingServer zkServer;
-	private static TemporaryFolder temporaryFolder;
-
-	public static void setup(int proxyPortRangeStart, int serverPortRangeStart) {
-		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(QueryableStateOptions.PROXY_PORT_RANGE, proxyPortRangeStart + "-" + (proxyPortRangeStart + NUM_TMS));
-			config.setString(QueryableStateOptions.SERVER_PORT_RANGE, serverPortRangeStart + "-" + (serverPortRangeStart + NUM_TMS));
-			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();
-
-			client = new QueryableStateClient("localhost", proxyPortRangeStart);
-
-			// 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();
-		}
-
-		try {
-			zkServer.stop();
-			zkServer.close();
-		} catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-
-		client.shutdown();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateFsBackendITCase.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateFsBackendITCase.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateFsBackendITCase.java
deleted file mode 100644
index 6f31e76..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateFsBackendITCase.java
+++ /dev/null
@@ -1,45 +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.itcases;
-
-import org.apache.flink.runtime.state.AbstractStateBackend;
-import org.apache.flink.runtime.state.filesystem.FsStateBackend;
-
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.rules.TemporaryFolder;
-
-/**
- * Several integration tests for queryable state using the {@link FsStateBackend}.
- */
-public class HAQueryableStateFsBackendITCase extends HAAbstractQueryableStateTestBase {
-
-	@Rule
-	public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
-	@BeforeClass
-	public static void setup() {
-		setup(9064, 9069);
-	}
-
-	@Override
-	protected AbstractStateBackend createStateBackend() throws Exception {
-		return new FsStateBackend(temporaryFolder.newFolder().toURI().toString());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateRocksDBBackendITCase.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateRocksDBBackendITCase.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateRocksDBBackendITCase.java
deleted file mode 100644
index 18b167f..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateRocksDBBackendITCase.java
+++ /dev/null
@@ -1,47 +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.itcases;
-
-import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
-import org.apache.flink.runtime.state.AbstractStateBackend;
-
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.rules.TemporaryFolder;
-
-/**
- * Several integration tests for queryable state using the {@link RocksDBStateBackend}.
- */
-@Ignore
-public class HAQueryableStateRocksDBBackendITCase extends HAAbstractQueryableStateTestBase {
-
-	@Rule
-	public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
-	@BeforeClass
-	public static void setup() {
-		setup(9074, 9079);
-	}
-
-	@Override
-	protected AbstractStateBackend createStateBackend() throws Exception {
-		return new RocksDBStateBackend(temporaryFolder.newFolder().toURI().toString());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/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
deleted file mode 100644
index 907e8a3..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/KVStateRequestSerializerRocksDBTest.java
+++ /dev/null
@@ -1,167 +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.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/0c771505/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java
deleted file mode 100644
index 2937a51..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java
+++ /dev/null
@@ -1,78 +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.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.queryablestate.client.QueryableStateClient;
-import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
-import org.apache.flink.runtime.testingUtils.TestingCluster;
-
-import org.junit.AfterClass;
-import org.junit.Assert;
-
-import static org.junit.Assert.fail;
-
-/**
- * Base class with the cluster configuration for the tests on the HA mode.
- */
-public abstract class NonHAAbstractQueryableStateTestBase extends AbstractQueryableStateTestBase {
-
-	private static final int NUM_TMS = 2;
-	private static final int NUM_SLOTS_PER_TM = 4;
-
-	public static void setup(int proxyPortRangeStart, int serverPortRangeStart) {
-		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);
-			config.setString(QueryableStateOptions.PROXY_PORT_RANGE, proxyPortRangeStart + "-" + (proxyPortRangeStart + NUM_TMS));
-			config.setString(QueryableStateOptions.SERVER_PORT_RANGE, serverPortRangeStart + "-" + (serverPortRangeStart + NUM_TMS));
-
-			cluster = new TestingCluster(config, false);
-			cluster.start(true);
-
-			client = new QueryableStateClient("localhost", proxyPortRangeStart);
-
-			// 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());
-		}
-		client.shutdown();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateFsBackendITCase.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateFsBackendITCase.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateFsBackendITCase.java
deleted file mode 100644
index 9457e0f..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateFsBackendITCase.java
+++ /dev/null
@@ -1,45 +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.itcases;
-
-import org.apache.flink.runtime.state.AbstractStateBackend;
-import org.apache.flink.runtime.state.filesystem.FsStateBackend;
-
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.rules.TemporaryFolder;
-
-/**
- * Several integration tests for queryable state using the {@link FsStateBackend}.
- */
-public class NonHAQueryableStateFsBackendITCase extends NonHAAbstractQueryableStateTestBase {
-
-	@Rule
-	public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
-	@BeforeClass
-	public static void setup() {
-		setup(9084, 9089);
-	}
-
-	@Override
-	protected AbstractStateBackend createStateBackend() throws Exception {
-		return new FsStateBackend(temporaryFolder.newFolder().toURI().toString());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateRocksDBBackendITCase.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateRocksDBBackendITCase.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateRocksDBBackendITCase.java
deleted file mode 100644
index 39fbe9e..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateRocksDBBackendITCase.java
+++ /dev/null
@@ -1,47 +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.itcases;
-
-import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
-import org.apache.flink.runtime.state.AbstractStateBackend;
-
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.rules.TemporaryFolder;
-
-/**
- * Several integration tests for queryable state using the {@link RocksDBStateBackend}.
- */
-@Ignore
-public class NonHAQueryableStateRocksDBBackendITCase extends NonHAAbstractQueryableStateTestBase {
-
-	@Rule
-	public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
-	@BeforeClass
-	public static void setup() {
-		setup(9094, 9099);
-	}
-
-	@Override
-	protected AbstractStateBackend createStateBackend() throws Exception {
-		return new RocksDBStateBackend(temporaryFolder.newFolder().toURI().toString());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java
deleted file mode 100644
index 1fd7012..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java
+++ /dev/null
@@ -1,219 +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.configuration.ConfigConstants;
-import org.apache.flink.queryablestate.network.messages.MessageBody;
-import org.apache.flink.queryablestate.network.messages.MessageDeserializer;
-import org.apache.flink.queryablestate.network.messages.MessageSerializer;
-import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats;
-import org.apache.flink.util.FlinkRuntimeException;
-import org.apache.flink.util.Preconditions;
-
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
-
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.CompletableFuture;
-
-/**
- * Tests general behavior of the {@link AbstractServerBase}.
- */
-public class AbstractServerTest {
-
-	@Rule
-	public ExpectedException expectedEx = ExpectedException.none();
-
-	/**
-	 * Tests that in case of port collision, a FlinkRuntimeException is thrown
-	 * with a specific message.
-	 */
-	@Test
-	public void testServerInitializationFailure() throws Throwable {
-
-		// the expected exception along with the adequate message
-		expectedEx.expect(FlinkRuntimeException.class);
-		expectedEx.expectMessage("Unable to start server Test Server 2. All ports in provided range are occupied.");
-
-		TestServer server1 = null;
-		TestServer server2 = null;
-		try {
-
-			server1 = startServer("Test Server 1", 7777);
-			Assert.assertEquals(7777L, server1.getServerAddress().getPort());
-
-			server2 = startServer("Test Server 2", 7777);
-		} finally {
-
-			if (server1 != null) {
-				server1.shutdown();
-			}
-
-			if (server2 != null) {
-				server2.shutdown();
-			}
-		}
-	}
-
-	/**
-	 * Tests that in case of port collision and big enough port range,
-	 * the server will try to bind to the next port in the range.
-	 */
-	@Test
-	public void testPortRangeSuccess() throws Throwable {
-		TestServer server1 = null;
-		TestServer server2 = null;
-		Client<TestMessage, TestMessage> client = null;
-
-		try {
-			server1 = startServer("Test Server 1", 7777, 7778, 7779);
-			Assert.assertEquals(7777L, server1.getServerAddress().getPort());
-
-			server2 = startServer("Test Server 2", 7777, 7778, 7779);
-			Assert.assertEquals(7778L, server2.getServerAddress().getPort());
-
-			client = new Client<>(
-					"Test Client",
-					1,
-					new MessageSerializer<>(new TestMessage.TestMessageDeserializer(), new TestMessage.TestMessageDeserializer()),
-					new DisabledKvStateRequestStats());
-
-			TestMessage response1 = client.sendRequest(server1.getServerAddress(), new TestMessage("ping")).join();
-			Assert.assertEquals(server1.getServerName() + "-ping", response1.getMessage());
-
-			TestMessage response2 = client.sendRequest(server2.getServerAddress(), new TestMessage("pong")).join();
-			Assert.assertEquals(server2.getServerName() + "-pong", response2.getMessage());
-		} finally {
-
-			if (server1 != null) {
-				server1.shutdown();
-			}
-
-			if (server2 != null) {
-				server2.shutdown();
-			}
-
-			if (client != null) {
-				client.shutdown();
-			}
-		}
-	}
-
-	/**
-	 * Initializes a {@link TestServer} with the given port range.
-	 * @param serverName the name of the server.
-	 * @param ports a range of ports.
-	 * @return A test server with the given name.
-	 */
-	private TestServer startServer(String serverName, int... ports) throws Throwable {
-		List<Integer> portList = new ArrayList<>(ports.length);
-		for (int p : ports) {
-			portList.add(p);
-		}
-
-		final TestServer server = new TestServer(serverName, portList.iterator());
-		server.start();
-		return server;
-	}
-
-	/**
-	 * A server that receives a {@link TestMessage test message} and returns another test
-	 * message containing the same string as the request with the name of the server prepended.
-	 */
-	private class TestServer extends AbstractServerBase<TestMessage, TestMessage> {
-
-		protected TestServer(String name, Iterator<Integer> bindPort) throws UnknownHostException {
-			super(name, InetAddress.getLocalHost(), bindPort, 1, 1);
-		}
-
-		@Override
-		public AbstractServerHandler<TestMessage, TestMessage> initializeHandler() {
-			return new AbstractServerHandler<TestMessage, TestMessage>(
-					this,
-					new MessageSerializer<>(new TestMessage.TestMessageDeserializer(), new TestMessage.TestMessageDeserializer()),
-					new DisabledKvStateRequestStats()) {
-
-				@Override
-				public CompletableFuture<TestMessage> handleRequest(long requestId, TestMessage request) {
-					TestMessage response = new TestMessage(getServerName() + '-' + request.getMessage());
-					return CompletableFuture.completedFuture(response);
-				}
-
-				@Override
-				public void shutdown() {
-					// do nothing
-				}
-			};
-		}
-	}
-
-	/**
-	 * Message with a string as payload.
-	 */
-	private static class TestMessage extends MessageBody {
-
-		private final String message;
-
-		TestMessage(String message) {
-			this.message = Preconditions.checkNotNull(message);
-		}
-
-		public String getMessage() {
-			return message;
-		}
-
-		@Override
-		public byte[] serialize() {
-			byte[] content = message.getBytes(ConfigConstants.DEFAULT_CHARSET);
-
-			// message size + 4 for the length itself
-			return ByteBuffer.allocate(content.length + Integer.BYTES)
-					.putInt(content.length)
-					.put(content)
-					.array();
-		}
-
-		/**
-		 * The deserializer for our {@link TestMessage test messages}.
-		 */
-		public static class TestMessageDeserializer implements MessageDeserializer<TestMessage> {
-
-			@Override
-			public TestMessage deserializeMessage(ByteBuf buf) {
-				int length = buf.readInt();
-				String message = "";
-				if (length > 0) {
-					byte[] name = new byte[length];
-					buf.readBytes(name);
-					message = new String(name, ConfigConstants.DEFAULT_CHARSET);
-				}
-				return new TestMessage(message);
-			}
-		}
-	}
-}


[03/13] flink git commit: [FLINK-7908][QS] Restructure the queryable state module.

Posted by kk...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java
new file mode 100644
index 0000000..1fa4deb
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java
@@ -0,0 +1,782 @@
+/*
+ * 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.KvStateID;
+import org.apache.flink.queryablestate.client.VoidNamespace;
+import org.apache.flink.queryablestate.client.VoidNamespaceSerializer;
+import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer;
+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.stats.AtomicKvStateRequestStats;
+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.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.apache.flink.runtime.state.KeyGroupRange;
+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);
+				}
+			});
+
+			InetSocketAddress 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();
+
+			InetSocketAddress serverAddress = new InetSocketAddress(
+					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 InetSocketAddress 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);
+				}
+			});
+
+			InetSocketAddress 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);
+				}
+			});
+
+			InetSocketAddress 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 Throwable {
+		// 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(),
+						Collections.singletonList(0).iterator(),
+						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 InetSocketAddress getKvStateServerAddress(Channel serverChannel) {
+		return (InetSocketAddress) serverChannel.localAddress();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateClientHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateClientHandlerTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateClientHandlerTest.java
new file mode 100644
index 0000000..cb490aa
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateClientHandlerTest.java
@@ -0,0 +1,119 @@
+/*
+ * 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.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 ClientHandler}.
+ */
+public class KvStateClientHandlerTest {
+
+	/**
+	 * Tests that on reads the expected callback methods are called and read
+	 * buffers are recycled.
+	 */
+	@Test
+	public void testReadCallbacksAndBufferRecycling() throws Exception {
+		final ClientHandlerCallback<KvStateResponse> callback = mock(ClientHandlerCallback.class);
+
+		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.serializeResponse(channel.alloc(), 1222112277L, response);
+		buf.skipBytes(4); // skip frame length
+
+		// Verify callback
+		channel.writeInbound(buf);
+		verify(callback, times(1)).onRequestResult(eq(1222112277L), any(KvStateResponse.class));
+		assertEquals("Buffer not recycled", 0, buf.refCnt());
+
+		//
+		// Request failure
+		//
+		buf = MessageSerializer.serializeRequestFailure(
+				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/0c771505/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateRequestSerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateRequestSerializerTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateRequestSerializerTest.java
new file mode 100644
index 0000000..d3314ab
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateRequestSerializerTest.java
@@ -0,0 +1,416 @@
+/*
+ * 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.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.api.java.tuple.Tuple2;
+import org.apache.flink.queryablestate.client.VoidNamespace;
+import org.apache.flink.queryablestate.client.VoidNamespaceSerializer;
+import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer;
+import org.apache.flink.runtime.query.TaskKvStateRegistry;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend;
+import org.apache.flink.runtime.state.internal.InternalKvState;
+import org.apache.flink.runtime.state.internal.InternalListState;
+import org.apache.flink.runtime.state.internal.InternalMapState;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for {@link KvStateSerializer}.
+ */
+@RunWith(Parameterized.class)
+public class KvStateRequestSerializerTest {
+
+	@Parameterized.Parameters
+	public static Collection<Boolean> parameters() {
+		return Arrays.asList(false, true);
+	}
+
+	@Parameterized.Parameter
+	public boolean async;
+
+	/**
+	 * Tests key and namespace serialization utils.
+	 */
+	@Test
+	public void testKeyAndNamespaceSerialization() throws Exception {
+		TypeSerializer<Long> keySerializer = LongSerializer.INSTANCE;
+		TypeSerializer<String> namespaceSerializer = StringSerializer.INSTANCE;
+
+		long expectedKey = Integer.MAX_VALUE + 12323L;
+		String expectedNamespace = "knilf";
+
+		byte[] serializedKeyAndNamespace = KvStateSerializer.serializeKeyAndNamespace(
+				expectedKey, keySerializer, expectedNamespace, namespaceSerializer);
+
+		Tuple2<Long, String> actual = KvStateSerializer.deserializeKeyAndNamespace(
+				serializedKeyAndNamespace, keySerializer, namespaceSerializer);
+
+		assertEquals(expectedKey, actual.f0.longValue());
+		assertEquals(expectedNamespace, actual.f1);
+	}
+
+	/**
+	 * Tests key and namespace deserialization utils with too few bytes.
+	 */
+	@Test(expected = IOException.class)
+	public void testKeyAndNamespaceDeserializationEmpty() throws Exception {
+		KvStateSerializer.deserializeKeyAndNamespace(
+			new byte[] {}, LongSerializer.INSTANCE, StringSerializer.INSTANCE);
+	}
+
+	/**
+	 * Tests key and namespace deserialization utils with too few bytes.
+	 */
+	@Test(expected = IOException.class)
+	public void testKeyAndNamespaceDeserializationTooShort() throws Exception {
+		KvStateSerializer.deserializeKeyAndNamespace(
+			new byte[] {1}, LongSerializer.INSTANCE, StringSerializer.INSTANCE);
+	}
+
+	/**
+	 * Tests key and namespace deserialization utils with too many bytes.
+	 */
+	@Test(expected = IOException.class)
+	public void testKeyAndNamespaceDeserializationTooMany1() throws Exception {
+		// Long + null String + 1 byte
+		KvStateSerializer.deserializeKeyAndNamespace(
+			new byte[] {1, 1, 1, 1, 1, 1, 1, 1, 42, 0, 2}, LongSerializer.INSTANCE,
+			StringSerializer.INSTANCE);
+	}
+
+	/**
+	 * Tests key and namespace deserialization utils with too many bytes.
+	 */
+	@Test(expected = IOException.class)
+	public void testKeyAndNamespaceDeserializationTooMany2() throws Exception {
+		// Long + null String + 2 bytes
+		KvStateSerializer.deserializeKeyAndNamespace(
+			new byte[] {1, 1, 1, 1, 1, 1, 1, 1, 42, 0, 2, 2}, LongSerializer.INSTANCE,
+			StringSerializer.INSTANCE);
+	}
+
+	/**
+	 * Tests value serialization utils.
+	 */
+	@Test
+	public void testValueSerialization() throws Exception {
+		TypeSerializer<Long> valueSerializer = LongSerializer.INSTANCE;
+		long expectedValue = Long.MAX_VALUE - 1292929292L;
+
+		byte[] serializedValue = KvStateSerializer.serializeValue(expectedValue, valueSerializer);
+		long actualValue = KvStateSerializer.deserializeValue(serializedValue, valueSerializer);
+
+		assertEquals(expectedValue, actualValue);
+	}
+
+	/**
+	 * Tests value deserialization with too few bytes.
+	 */
+	@Test(expected = IOException.class)
+	public void testDeserializeValueEmpty() throws Exception {
+		KvStateSerializer.deserializeValue(new byte[] {}, LongSerializer.INSTANCE);
+	}
+
+	/**
+	 * Tests value deserialization with too few bytes.
+	 */
+	@Test(expected = IOException.class)
+	public void testDeserializeValueTooShort() throws Exception {
+		// 1 byte (incomplete Long)
+		KvStateSerializer.deserializeValue(new byte[] {1}, LongSerializer.INSTANCE);
+	}
+
+	/**
+	 * Tests value deserialization with too many bytes.
+	 */
+	@Test(expected = IOException.class)
+	public void testDeserializeValueTooMany1() throws Exception {
+		// Long + 1 byte
+		KvStateSerializer.deserializeValue(new byte[] {1, 1, 1, 1, 1, 1, 1, 1, 2},
+			LongSerializer.INSTANCE);
+	}
+
+	/**
+	 * Tests value deserialization with too many bytes.
+	 */
+	@Test(expected = IOException.class)
+	public void testDeserializeValueTooMany2() throws Exception {
+		// Long + 2 bytes
+		KvStateSerializer.deserializeValue(new byte[] {1, 1, 1, 1, 1, 1, 1, 1, 2, 2},
+			LongSerializer.INSTANCE);
+	}
+
+	/**
+	 * Tests list serialization utils.
+	 */
+	@Test
+	public void testListSerialization() throws Exception {
+		final long key = 0L;
+
+		// objects for heap state list serialisation
+		final HeapKeyedStateBackend<Long> longHeapKeyedStateBackend =
+			new HeapKeyedStateBackend<>(
+				mock(TaskKvStateRegistry.class),
+				LongSerializer.INSTANCE,
+				ClassLoader.getSystemClassLoader(),
+				1,
+				new KeyGroupRange(0, 0),
+				async,
+				new ExecutionConfig()
+			);
+		longHeapKeyedStateBackend.setCurrentKey(key);
+
+		final InternalListState<VoidNamespace, Long> listState = longHeapKeyedStateBackend.createListState(
+				VoidNamespaceSerializer.INSTANCE,
+				new ListStateDescriptor<>("test", LongSerializer.INSTANCE));
+
+		testListSerialization(key, listState);
+	}
+
+	/**
+	 * Verifies that the serialization of a list using the given list state
+	 * matches the deserialization with {@link KvStateSerializer#deserializeList}.
+	 *
+	 * @param key
+	 * 		key of the list state
+	 * @param listState
+	 * 		list state using the {@link VoidNamespace}, must also be a {@link InternalKvState} instance
+	 *
+	 * @throws Exception
+	 */
+	public static void testListSerialization(
+			final long key,
+			final InternalListState<VoidNamespace, Long> listState) throws Exception {
+
+		TypeSerializer<Long> valueSerializer = LongSerializer.INSTANCE;
+		listState.setCurrentNamespace(VoidNamespace.INSTANCE);
+
+		// List
+		final int numElements = 10;
+
+		final List<Long> expectedValues = new ArrayList<>();
+		for (int i = 0; i < numElements; i++) {
+			final long value = ThreadLocalRandom.current().nextLong();
+			expectedValues.add(value);
+			listState.add(value);
+		}
+
+		final byte[] serializedKey =
+			KvStateSerializer.serializeKeyAndNamespace(
+				key, LongSerializer.INSTANCE,
+				VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE);
+
+		final byte[] serializedValues = listState.getSerializedValue(serializedKey);
+
+		List<Long> actualValues = KvStateSerializer.deserializeList(serializedValues, valueSerializer);
+		assertEquals(expectedValues, actualValues);
+
+		// Single value
+		long expectedValue = ThreadLocalRandom.current().nextLong();
+		byte[] serializedValue = KvStateSerializer.serializeValue(expectedValue, valueSerializer);
+		List<Long> actualValue = KvStateSerializer.deserializeList(serializedValue, valueSerializer);
+		assertEquals(1, actualValue.size());
+		assertEquals(expectedValue, actualValue.get(0).longValue());
+	}
+
+	/**
+	 * Tests list deserialization with too few bytes.
+	 */
+	@Test
+	public void testDeserializeListEmpty() throws Exception {
+		List<Long> actualValue = KvStateSerializer
+			.deserializeList(new byte[] {}, LongSerializer.INSTANCE);
+		assertEquals(0, actualValue.size());
+	}
+
+	/**
+	 * Tests list deserialization with too few bytes.
+	 */
+	@Test(expected = IOException.class)
+	public void testDeserializeListTooShort1() throws Exception {
+		// 1 byte (incomplete Long)
+		KvStateSerializer.deserializeList(new byte[] {1}, LongSerializer.INSTANCE);
+	}
+
+	/**
+	 * Tests list deserialization with too few bytes.
+	 */
+	@Test(expected = IOException.class)
+	public void testDeserializeListTooShort2() throws Exception {
+		// Long + 1 byte (separator) + 1 byte (incomplete Long)
+		KvStateSerializer.deserializeList(new byte[] {1, 1, 1, 1, 1, 1, 1, 1, 2, 3},
+			LongSerializer.INSTANCE);
+	}
+
+	/**
+	 * Tests map serialization utils.
+	 */
+	@Test
+	public void testMapSerialization() throws Exception {
+		final long key = 0L;
+
+		// objects for heap state list serialisation
+		final HeapKeyedStateBackend<Long> longHeapKeyedStateBackend =
+			new HeapKeyedStateBackend<>(
+					mock(TaskKvStateRegistry.class),
+					LongSerializer.INSTANCE,
+					ClassLoader.getSystemClassLoader(),
+					1,
+					new KeyGroupRange(0, 0),
+					async,
+					new ExecutionConfig()
+			);
+		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));
+
+		testMapSerialization(key, mapState);
+	}
+
+	/**
+	 * Verifies that the serialization of a map using the given map state
+	 * matches the deserialization with {@link KvStateSerializer#deserializeList}.
+	 *
+	 * @param key
+	 * 		key of the map state
+	 * @param mapState
+	 * 		map state using the {@link VoidNamespace}, must also be a {@link InternalKvState} instance
+	 *
+	 * @throws Exception
+	 */
+	public static void testMapSerialization(
+			final long key,
+			final InternalMapState<VoidNamespace, Long, String> mapState) throws Exception {
+
+		TypeSerializer<Long> userKeySerializer = LongSerializer.INSTANCE;
+		TypeSerializer<String> userValueSerializer = StringSerializer.INSTANCE;
+		mapState.setCurrentNamespace(VoidNamespace.INSTANCE);
+
+		// Map
+		final int numElements = 10;
+
+		final Map<Long, String> expectedValues = new HashMap<>();
+		for (int i = 1; i <= numElements; i++) {
+			final long value = ThreadLocalRandom.current().nextLong();
+			expectedValues.put(value, Long.toString(value));
+			mapState.put(value, Long.toString(value));
+		}
+
+		expectedValues.put(0L, null);
+		mapState.put(0L, null);
+
+		final byte[] serializedKey =
+			KvStateSerializer.serializeKeyAndNamespace(
+				key, LongSerializer.INSTANCE,
+				VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE);
+
+		final byte[] serializedValues = mapState.getSerializedValue(serializedKey);
+
+		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());
+		}
+
+		// Single value
+		ByteArrayOutputStream baos = new ByteArrayOutputStream();
+		long expectedKey = ThreadLocalRandom.current().nextLong();
+		String expectedValue = Long.toString(expectedKey);
+		byte[] isNull = {0};
+
+		baos.write(KvStateSerializer.serializeValue(expectedKey, userKeySerializer));
+		baos.write(isNull);
+		baos.write(KvStateSerializer.serializeValue(expectedValue, userValueSerializer));
+		byte[] serializedValue = baos.toByteArray();
+
+		Map<Long, String> actualValue = KvStateSerializer.deserializeMap(serializedValue, userKeySerializer, userValueSerializer);
+		assertEquals(1, actualValue.size());
+		assertEquals(expectedValue, actualValue.get(expectedKey));
+	}
+
+	/**
+	 * Tests map deserialization with too few bytes.
+	 */
+	@Test
+	public void testDeserializeMapEmpty() throws Exception {
+		Map<Long, String> actualValue = KvStateSerializer
+			.deserializeMap(new byte[] {}, LongSerializer.INSTANCE, StringSerializer.INSTANCE);
+		assertEquals(0, actualValue.size());
+	}
+
+	/**
+	 * Tests map deserialization with too few bytes.
+	 */
+	@Test(expected = IOException.class)
+	public void testDeserializeMapTooShort1() throws Exception {
+		// 1 byte (incomplete Key)
+		KvStateSerializer.deserializeMap(new byte[] {1}, LongSerializer.INSTANCE, StringSerializer.INSTANCE);
+	}
+
+	/**
+	 * Tests map deserialization with too few bytes.
+	 */
+	@Test(expected = IOException.class)
+	public void testDeserializeMapTooShort2() throws Exception {
+		// Long (Key) + 1 byte (incomplete Value)
+		KvStateSerializer.deserializeMap(new byte[]{1, 1, 1, 1, 1, 1, 1, 1, 0},
+				LongSerializer.INSTANCE, LongSerializer.INSTANCE);
+	}
+
+	/**
+	 * Tests map deserialization with too few bytes.
+	 */
+	@Test(expected = IOException.class)
+	public void testDeserializeMapTooShort3() throws Exception {
+		// Long (Key1) + Boolean (false) + Long (Value1) + 1 byte (incomplete Key2)
+		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);
+	}
+
+	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/0c771505/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java
new file mode 100644
index 0000000..041544d
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java
@@ -0,0 +1,758 @@
+/*
+ * 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.KvStateID;
+import org.apache.flink.queryablestate.client.VoidNamespace;
+import org.apache.flink.queryablestate.client.VoidNamespaceSerializer;
+import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer;
+import org.apache.flink.queryablestate.exceptions.UnknownKeyOrNamespaceException;
+import org.apache.flink.queryablestate.exceptions.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.network.messages.RequestFailure;
+import org.apache.flink.queryablestate.network.stats.AtomicKvStateRequestStats;
+import org.apache.flink.queryablestate.network.stats.DisabledKvStateRequestStats;
+import org.apache.flink.queryablestate.network.stats.KvStateRequestStats;
+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.KvStateRegistry;
+import org.apache.flink.runtime.query.KvStateRegistryListener;
+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.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.BeforeClass;
+import org.junit.Test;
+
+import java.net.InetAddress;
+import java.util.Collections;
+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 {
+
+	private static KvStateServerImpl testServer;
+
+	private static final long READ_TIMEOUT_MILLIS = 10000L;
+
+	@BeforeClass
+	public static void setup() {
+		try {
+			testServer = new KvStateServerImpl(
+					InetAddress.getLocalHost(),
+					Collections.singletonList(0).iterator(),
+					1,
+					1,
+					new KvStateRegistry(),
+					new DisabledKvStateRequestStats());
+			testServer.start();
+		} catch (Throwable e) {
+			e.printStackTrace();
+		}
+	}
+
+	@AfterClass
+	public static void tearDown() throws Exception {
+		testServer.shutdown();
+	}
+
+	/**
+	 * Tests a simple successful query via an EmbeddedChannel.
+	 */
+	@Test
+	public void testSimpleQuery() throws Exception {
+		KvStateRegistry registry = new KvStateRegistry();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		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
+		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"));
+
+		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(serRequest);
+
+		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
+		buf.skipBytes(4); // skip frame length
+
+		// Verify the response
+		assertEquals(MessageType.REQUEST_RESULT, MessageSerializer.deserializeHeader(buf));
+		long deserRequestId = MessageSerializer.getRequestId(buf);
+		KvStateResponse response = serializer.deserializeResponse(buf);
+
+		assertEquals(requestId, deserRequestId);
+
+		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() != 1L && System.nanoTime() <= deadline) {
+			Thread.sleep(10L);
+		}
+
+		assertEquals(stats.toString(), 1L, stats.getNumSuccessful());
+	}
+
+	/**
+	 * Tests the failure response with {@link UnknownKvStateIdException} as cause on
+	 * queries for unregistered KvStateIDs.
+	 */
+	@Test
+	public void testQueryUnknownKvStateID() throws Exception {
+		KvStateRegistry registry = new KvStateRegistry();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		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;
+
+		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(serRequest);
+
+		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
+		buf.skipBytes(4); // skip frame length
+
+		// Verify the response
+		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
+		RequestFailure response = MessageSerializer.deserializeRequestFailure(buf);
+
+		assertEquals(requestId, response.getRequestId());
+
+		assertTrue("Did not respond with expected failure cause", response.getCause() instanceof UnknownKvStateIdException);
+
+		assertEquals(1L, stats.getNumRequests());
+		assertEquals(1L, stats.getNumFailed());
+	}
+
+	/**
+	 * Tests the failure response with {@link UnknownKeyOrNamespaceException} as cause
+	 * on queries for non-existing keys.
+	 */
+	@Test
+	public void testQueryUnknownKey() throws Exception {
+		KvStateRegistry registry = new KvStateRegistry();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		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;
+		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"));
+
+		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(serRequest);
+
+		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
+		buf.skipBytes(4); // skip frame length
+
+		// Verify the response
+		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
+		RequestFailure response = MessageSerializer.deserializeRequestFailure(buf);
+
+		assertEquals(requestId, response.getRequestId());
+
+		assertTrue("Did not respond with expected failure cause", response.getCause() instanceof UnknownKeyOrNamespaceException);
+
+		assertEquals(1L, stats.getNumRequests());
+		assertEquals(1L, 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();
+
+		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
+		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);
+
+		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(serRequest);
+
+		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
+		buf.skipBytes(4); // skip frame length
+
+		// Verify the response
+		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
+		RequestFailure response = MessageSerializer.deserializeRequestFailure(buf);
+
+		assertTrue(response.getCause().getMessage().contains("Expected test Exception"));
+
+		assertEquals(1L, stats.getNumRequests());
+		assertEquals(1L, 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();
+
+		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"));
+
+		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 Throwable {
+		KvStateRegistry registry = new KvStateRegistry();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		KvStateServerImpl localTestServer = new KvStateServerImpl(
+				InetAddress.getLocalHost(),
+				Collections.singletonList(0).iterator(),
+				1,
+				1,
+				new KvStateRegistry(),
+				new DisabledKvStateRequestStats());
+
+		localTestServer.start();
+		localTestServer.shutdown();
+		assertTrue(localTestServer.isExecutorShutdown());
+
+		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;
+		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"));
+
+		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(serRequest);
+
+		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
+		buf.skipBytes(4); // skip frame length
+
+		// Verify the response
+		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
+		RequestFailure response = MessageSerializer.deserializeRequestFailure(buf);
+
+		assertTrue(response.getCause().getMessage().contains("RejectedExecutionException"));
+
+		assertEquals(1L, stats.getNumRequests());
+		assertEquals(1L, stats.getNumFailed());
+
+		localTestServer.shutdown();
+	}
+
+	/**
+	 * Tests response on unexpected messages.
+	 */
+	@Test
+	public void testUnexpectedMessage() throws Exception {
+		KvStateRegistry registry = new KvStateRegistry();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		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
+		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(0L, stats.getNumRequests());
+		assertEquals(0L, stats.getNumFailed());
+
+		KvStateResponse stateResponse = new KvStateResponse(new byte[0]);
+		unexpectedMessage = MessageSerializer.serializeResponse(channel.alloc(), 192L, stateResponse);
+
+		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(0L, stats.getNumRequests());
+		assertEquals(0L, stats.getNumFailed());
+	}
+
+	/**
+	 * Tests that incoming buffer instances are recycled.
+	 */
+	@Test
+	public void testIncomingBufferIsRecycled() throws Exception {
+		KvStateRegistry registry = new KvStateRegistry();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		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);
+
+		KvStateInternalRequest request = new KvStateInternalRequest(new KvStateID(), new byte[0]);
+		ByteBuf serRequest = MessageSerializer.serializeRequest(channel.alloc(), 282872L, request);
+
+		assertEquals(1L, serRequest.refCnt());
+
+		// Write regular request
+		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(1L, unexpected.refCnt());
+
+		channel.writeInbound(unexpected);
+		assertEquals("Buffer not recycled", 0L, 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();
+
+		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;
+		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"));
+
+		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(serRequest);
+
+		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
+		buf.skipBytes(4); // skip frame length
+
+		// Verify the response
+		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
+		RequestFailure response = MessageSerializer.deserializeRequestFailure(buf);
+		assertEquals(182828L, response.getRequestId());
+		assertTrue(response.getCause().getMessage().contains("IOException"));
+
+		// Repeat with wrong namespace only
+		request = new KvStateInternalRequest(registryListener.kvStateId, wrongNamespace);
+		serRequest = MessageSerializer.serializeRequest(channel.alloc(), 182829L, request);
+
+		// Write the request and wait for the response
+		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.deserializeRequestFailure(buf);
+		assertEquals(182829L, response.getRequestId());
+		assertTrue(response.getCause().getMessage().contains("IOException"));
+
+		assertEquals(2L, stats.getNumRequests());
+		assertEquals(2L, stats.getNumFailed());
+	}
+
+	/**
+	 * Tests that large responses are chunked.
+	 */
+	@Test
+	public void testChunkedResponse() throws Exception {
+		KvStateRegistry registry = new KvStateRegistry();
+		KvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		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;
+		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"));
+
+		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(serRequest);
+
+		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 long sleepMillis = 50L;
+
+		long sleptMillis = 0L;
+
+		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/0c771505/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java
new file mode 100644
index 0000000..debd190
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java
@@ -0,0 +1,212 @@
+/*
+ * 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.VoidNamespace;
+import org.apache.flink.queryablestate.client.VoidNamespaceSerializer;
+import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer;
+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.stats.AtomicKvStateRequestStats;
+import org.apache.flink.queryablestate.network.stats.KvStateRequestStats;
+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.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.apache.flink.runtime.state.KeyGroupRange;
+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.net.InetSocketAddress;
+import java.util.Collections;
+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 KvStateServerImpl}.
+ */
+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 Throwable {
+		KvStateServerImpl server = null;
+		Bootstrap bootstrap = null;
+		try {
+			KvStateRegistry registry = new KvStateRegistry();
+			KvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+			server = new KvStateServerImpl(
+					InetAddress.getLocalHost(),
+					Collections.singletonList(0).iterator(),
+					1,
+					1,
+					registry,
+					stats);
+			server.start();
+
+			InetSocketAddress serverAddress = server.getServerAddress();
+			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.getAddress(), serverAddress.getPort())
+					.sync().channel();
+
+			long requestId = Integer.MAX_VALUE + 182828L;
+
+			assertTrue(registryListener.registrationName.equals("vanilla"));
+
+			final KvStateInternalRequest request = new KvStateInternalRequest(
+					registryListener.kvStateId,
+					serializedKeyAndNamespace);
+
+			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));
+			assertEquals(requestId, MessageSerializer.getRequestId(buf));
+			KvStateResponse response = server.getSerializer().deserializeResponse(buf);
+
+			int actualValue = KvStateSerializer.deserializeValue(response.getContent(), 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);
+					}
+				});
+	}
+
+}


[12/13] flink git commit: [FLINK-7908][QS] Restructure the queryable state module.

Posted by kk...@apache.org.
[FLINK-7908][QS] Restructure the queryable state module.

The QS module is split into core and client. The core should
be put in the lib folder to enable queryable state, while the
client is the one that the user will program against. The
reason for the restructuring in mainly to remove the dependency
on the flink-runtime from the user's program.


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

Branch: refs/heads/master
Commit: 0c771505b84cdacf7a359c3be0efe38a30f9e660
Parents: 8595dad
Author: kkloudas <kk...@gmail.com>
Authored: Tue Oct 24 12:16:08 2017 +0200
Committer: kkloudas <kk...@gmail.com>
Committed: Thu Oct 26 18:57:34 2017 +0200

----------------------------------------------------------------------
 .../streaming/state/AbstractRocksDBState.java   |    2 +-
 .../streaming/state/RocksDBMapState.java        |    2 +-
 .../flink-queryable-state-client-java/pom.xml   |   90 ++
 .../flink/queryablestate/FutureUtils.java       |   43 +
 .../apache/flink/queryablestate/KvStateID.java  |   40 +
 .../client/QueryableStateClient.java            |  257 +++
 .../queryablestate/client/VoidNamespace.java    |   75 +
 .../client/VoidNamespaceSerializer.java         |   96 ++
 .../client/VoidNamespaceTypeInfo.java           |   92 ++
 .../client/state/ImmutableAggregatingState.java |   71 +
 .../client/state/ImmutableFoldingState.java     |   70 +
 .../client/state/ImmutableListState.java        |   70 +
 .../client/state/ImmutableMapState.java         |  139 ++
 .../client/state/ImmutableReducingState.java    |   69 +
 .../client/state/ImmutableState.java            |   29 +
 .../client/state/ImmutableStateBinder.java      |   80 +
 .../client/state/ImmutableValueState.java       |   69 +
 .../serialization/DataInputDeserializer.java    |  392 +++++
 .../serialization/DataOutputSerializer.java     |  344 ++++
 .../state/serialization/KvStateSerializer.java  |  265 ++++
 .../exceptions/UnknownJobManagerException.java  |   36 +
 .../UnknownKeyOrNamespaceException.java         |   39 +
 .../exceptions/UnknownKvStateIdException.java   |   42 +
 ...UnknownKvStateKeyGroupLocationException.java |   39 +
 .../queryablestate/messages/KvStateRequest.java |  140 ++
 .../messages/KvStateResponse.java               |   74 +
 .../network/AbstractServerBase.java             |  308 ++++
 .../network/AbstractServerHandler.java          |  305 ++++
 .../network/BadRequestException.java            |   35 +
 .../queryablestate/network/ChunkedByteBuf.java  |  100 ++
 .../flink/queryablestate/network/Client.java    |  536 +++++++
 .../queryablestate/network/ClientHandler.java   |  122 ++
 .../network/ClientHandlerCallback.java          |   56 +
 .../queryablestate/network/NettyBufferPool.java |  171 ++
 .../network/messages/MessageBody.java           |   38 +
 .../network/messages/MessageDeserializer.java   |   39 +
 .../network/messages/MessageSerializer.java     |  320 ++++
 .../network/messages/MessageType.java           |   42 +
 .../network/messages/RequestFailure.java        |   71 +
 .../stats/AtomicKvStateRequestStats.java        |  104 ++
 .../stats/DisabledKvStateRequestStats.java      |   45 +
 .../network/stats/KvStateRequestStats.java      |   54 +
 .../client/VoidNamespaceTypeInfoTest.java       |   32 +
 .../state/ImmutableAggregatingStateTest.java    |  107 ++
 .../client/state/ImmutableFoldingStateTest.java |   93 ++
 .../client/state/ImmutableListStateTest.java    |  110 ++
 .../client/state/ImmutableMapStateTest.java     |  188 +++
 .../state/ImmutableReducingStateTest.java       |   83 +
 .../client/state/ImmutableValueStateTest.java   |   69 +
 .../src/test/resources/log4j-test.properties    |   31 +
 .../flink-queryable-state-java/pom.xml          |  137 --
 .../UnknownJobManagerException.java             |   36 -
 .../UnknownKeyOrNamespaceException.java         |   39 -
 .../UnknownKvStateIdException.java              |   42 -
 ...UnknownKvStateKeyGroupLocationException.java |   41 -
 .../client/QueryableStateClient.java            |  260 ---
 .../client/proxy/KvStateClientProxyHandler.java |  225 ---
 .../client/proxy/KvStateClientProxyImpl.java    |  128 --
 .../client/state/ImmutableAggregatingState.java |   71 -
 .../client/state/ImmutableFoldingState.java     |   70 -
 .../client/state/ImmutableListState.java        |   70 -
 .../client/state/ImmutableMapState.java         |  139 --
 .../client/state/ImmutableReducingState.java    |   69 -
 .../client/state/ImmutableState.java            |   29 -
 .../client/state/ImmutableStateBinder.java      |   80 -
 .../client/state/ImmutableValueState.java       |   69 -
 .../messages/KvStateInternalRequest.java        |   93 --
 .../queryablestate/messages/KvStateRequest.java |  141 --
 .../messages/KvStateResponse.java               |   75 -
 .../network/AbstractServerBase.java             |  310 ----
 .../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     |  320 ----
 .../network/messages/MessageType.java           |   42 -
 .../network/messages/RequestFailure.java        |   71 -
 .../server/KvStateServerHandler.java            |  107 --
 .../server/KvStateServerImpl.java               |  111 --
 .../itcases/AbstractQueryableStateTestBase.java | 1496 ------------------
 .../HAAbstractQueryableStateTestBase.java       |   98 --
 .../HAQueryableStateFsBackendITCase.java        |   45 -
 .../HAQueryableStateRocksDBBackendITCase.java   |   47 -
 .../KVStateRequestSerializerRocksDBTest.java    |  167 --
 .../NonHAAbstractQueryableStateTestBase.java    |   78 -
 .../NonHAQueryableStateFsBackendITCase.java     |   45 -
 ...NonHAQueryableStateRocksDBBackendITCase.java |   47 -
 .../network/AbstractServerTest.java             |  219 ---
 .../queryablestate/network/ClientTest.java      |  784 ---------
 .../network/KvStateClientHandlerTest.java       |  119 --
 .../network/KvStateServerHandlerTest.java       |  758 ---------
 .../network/KvStateServerTest.java              |  207 ---
 .../network/MessageSerializerTest.java          |  220 ---
 .../state/ImmutableAggregatingStateTest.java    |  108 --
 .../state/ImmutableFoldingStateTest.java        |   94 --
 .../state/ImmutableListStateTest.java           |  112 --
 .../state/ImmutableMapStateTest.java            |  189 ---
 .../state/ImmutableReducingStateTest.java       |   84 -
 .../state/ImmutableValueStateTest.java          |   70 -
 .../src/test/resources/log4j-test.properties    |   31 -
 .../flink-queryable-state-runtime/pom.xml       |  119 ++
 .../client/proxy/KvStateClientProxyHandler.java |  225 +++
 .../client/proxy/KvStateClientProxyImpl.java    |  128 ++
 .../messages/KvStateInternalRequest.java        |   93 ++
 .../server/KvStateServerHandler.java            |  107 ++
 .../server/KvStateServerImpl.java               |  111 ++
 .../itcases/AbstractQueryableStateTestBase.java | 1496 ++++++++++++++++++
 .../HAAbstractQueryableStateTestBase.java       |   98 ++
 .../HAQueryableStateFsBackendITCase.java        |   45 +
 .../HAQueryableStateRocksDBBackendITCase.java   |   47 +
 .../KVStateRequestSerializerRocksDBTest.java    |  167 ++
 .../NonHAAbstractQueryableStateTestBase.java    |   78 +
 .../NonHAQueryableStateFsBackendITCase.java     |   45 +
 ...NonHAQueryableStateRocksDBBackendITCase.java |   47 +
 .../network/AbstractServerTest.java             |  219 +++
 .../queryablestate/network/ClientTest.java      |  782 +++++++++
 .../network/KvStateClientHandlerTest.java       |  119 ++
 .../network/KvStateRequestSerializerTest.java   |  416 +++++
 .../network/KvStateServerHandlerTest.java       |  758 +++++++++
 .../network/KvStateServerTest.java              |  212 +++
 .../network/MessageSerializerTest.java          |  220 +++
 .../src/test/resources/log4j-test.properties    |   31 +
 flink-queryable-state/pom.xml                   |    5 +-
 flink-runtime/pom.xml                           |    6 +
 .../flink/runtime/jobmaster/JobMaster.java      |    6 +-
 .../runtime/jobmaster/JobMasterGateway.java     |    6 +-
 .../apache/flink/runtime/query/KvStateID.java   |   41 -
 .../flink/runtime/query/KvStateLocation.java    |   14 +-
 .../runtime/query/KvStateLocationRegistry.java  |    4 +-
 .../flink/runtime/query/KvStateMessage.java     |   10 +-
 .../flink/runtime/query/KvStateRegistry.java    |    1 +
 .../runtime/query/KvStateRegistryGateway.java   |    5 +-
 .../runtime/query/KvStateRegistryListener.java  |    1 +
 .../flink/runtime/query/KvStateServer.java      |    6 +-
 .../runtime/query/KvStateServerAddress.java     |   95 --
 .../runtime/query/QueryableStateUtils.java      |    8 +-
 .../runtime/query/TaskKvStateRegistry.java      |    1 +
 .../query/netty/AtomicKvStateRequestStats.java  |  104 --
 .../netty/DisabledKvStateRequestStats.java      |   45 -
 .../query/netty/KvStateRequestStats.java        |   55 -
 .../query/netty/message/KvStateSerializer.java  |  267 ----
 .../runtime/state/heap/AbstractHeapState.java   |    6 +-
 .../flink/runtime/state/heap/HeapMapState.java  |    2 +-
 .../taskexecutor/TaskManagerServices.java       |    2 +-
 .../rpc/RpcKvStateRegistryListener.java         |    9 +-
 .../ActorGatewayKvStateRegistryListener.java    |   11 +-
 .../runtime/jobmanager/JobManagerTest.java      |   12 +-
 .../query/KvStateLocationRegistryTest.java      |   10 +-
 .../runtime/query/KvStateLocationTest.java      |    8 +-
 .../message/KvStateRequestSerializerTest.java   |  415 -----
 .../runtime/state/StateBackendTestBase.java     |    4 +-
 .../completeness/TypeInfoTestCoverageTest.java  |    3 +-
 pom.xml                                         |    2 +-
 tools/travis_mvn_watchdog.sh                    |    3 +-
 158 files changed, 11373 insertions(+), 10107 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/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 cf365b4..969a1fc 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.KvStateSerializer;
+import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer;
 import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
 import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.util.Preconditions;

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/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 421bb2e..e8c34cc 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.KvStateSerializer;
+import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer;
 import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
 import org.apache.flink.runtime.state.internal.InternalMapState;
 import org.apache.flink.util.Preconditions;

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/pom.xml
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/pom.xml b/flink-queryable-state/flink-queryable-state-client-java/pom.xml
new file mode 100644
index 0000000..8a4ff69
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/pom.xml
@@ -0,0 +1,90 @@
+<?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-client-java_${scala.binary.version}</artifactId>
+	<name>flink-queryable-state-client-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-shaded-netty</artifactId>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-shaded-guava</artifactId>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-core</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils-junit</artifactId>
+			<version>${project.version}</version>
+			<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/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/FutureUtils.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/FutureUtils.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/FutureUtils.java
new file mode 100644
index 0000000..e2af7b1
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/FutureUtils.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.queryablestate;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Utility class for {@link java.util.concurrent.Future Java Futures}.
+ */
+public class FutureUtils {
+
+	// ------------------------------------------------------------------------
+	//  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/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/KvStateID.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/KvStateID.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/KvStateID.java
new file mode 100644
index 0000000..992b283
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/KvStateID.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.queryablestate;
+
+import org.apache.flink.util.AbstractID;
+
+/**
+ *  Identifier for state instances.
+ *
+ * <p>Assigned when registering the state at the state registry.
+ */
+public class KvStateID extends AbstractID {
+
+	private static final long serialVersionUID = 1L;
+
+	public KvStateID() {
+		super();
+	}
+
+	public KvStateID(long lowerPart, long upperPart) {
+		super(lowerPart, upperPart);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java
new file mode 100644
index 0000000..304505a
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java
@@ -0,0 +1,257 @@
+/*
+ * 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.State;
+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.queryablestate.FutureUtils;
+import org.apache.flink.queryablestate.client.state.ImmutableStateBinder;
+import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer;
+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.queryablestate.network.stats.DisabledKvStateRequestStats;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * 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 Task Managers and the location will be reported to the Job Manager.
+ *
+ * <p>The client connects to a {@code 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);
+
+	/** The client that forwards the requests to the proxy. */
+	private final Client<KvStateRequest, KvStateResponse> client;
+
+	/** The address of the proxy this client is connected to. */
+	private final InetSocketAddress remoteAddress;
+
+	/** The execution configuration used to instantiate the different (de-)serializers. */
+	private ExecutionConfig executionConfig;
+
+	/**
+	 * Create the Queryable State Client.
+	 * @param remoteHostname the hostname of the {@code Client Proxy} to connect to.
+	 * @param remotePort the port of the proxy to connect to.
+	 */
+	public QueryableStateClient(final String remoteHostname, final int remotePort) throws UnknownHostException {
+		this(InetAddress.getByName(Preconditions.checkNotNull(remoteHostname)), remotePort);
+	}
+
+	/**
+	 * Create the Queryable State Client.
+	 * @param remoteAddress the {@link InetAddress address} of the {@code Client Proxy} to connect to.
+	 * @param remotePort the port of the proxy to connect to.
+	 */
+	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).");
+
+		this.remoteAddress = new InetSocketAddress(remoteAddress, remotePort);
+
+		final MessageSerializer<KvStateRequest, KvStateResponse> messageSerializer =
+				new MessageSerializer<>(
+						new KvStateRequest.KvStateRequestDeserializer(),
+						new KvStateResponse.KvStateResponseDeserializer());
+
+		this.client = new Client<>(
+				"Queryable State Client",
+				1,
+				messageSerializer,
+				new DisabledKvStateRequestStats());
+	}
+
+	/** Shuts down the client. */
+	public void shutdown() {
+		client.shutdown();
+	}
+
+	/**
+	 * Gets the {@link ExecutionConfig}.
+	 */
+	public ExecutionConfig getExecutionConfig() {
+		return executionConfig;
+	}
+
+	/**
+	 * 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.	 *
+	 * @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 immutable {@link State} object containing the result.
+	 */
+	@PublicEvolving
+	public <K, S extends State, V> CompletableFuture<S> getKvState(
+			final JobID jobId,
+			final String queryableStateName,
+			final K key,
+			final TypeHint<K> keyTypeHint,
+			final StateDescriptor<S, V> stateDescriptor) {
+
+		Preconditions.checkNotNull(keyTypeHint);
+
+		TypeInformation<K> keyTypeInfo = keyTypeHint.getTypeInfo();
+		return getKvState(jobId, queryableStateName, key, keyTypeInfo, stateDescriptor);
+	}
+
+	/**
+	 * 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.
+	 * @param keyTypeInfo				The {@link TypeInformation} of the key.
+	 * @param stateDescriptor			The {@link StateDescriptor} of the state we want to query.
+	 * @return Future holding the immutable {@link State} object containing the result.
+	 */
+	@PublicEvolving
+	public <K, S extends State, V> CompletableFuture<S> getKvState(
+			final JobID jobId,
+			final String queryableStateName,
+			final K key,
+			final TypeInformation<K> keyTypeInfo,
+			final StateDescriptor<S, V> stateDescriptor) {
+
+		return getKvState(jobId, queryableStateName, key, VoidNamespace.INSTANCE,
+				keyTypeInfo, VoidNamespaceTypeInfo.INSTANCE, stateDescriptor);
+	}
+
+	/**
+	 * 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 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 immutable {@link State} object containing the result.
+	 */
+	@PublicEvolving
+	public <K, N, S extends State, V> CompletableFuture<S> getKvState(
+			final JobID jobId,
+			final String queryableStateName,
+			final K key,
+			final N namespace,
+			final TypeInformation<K> keyTypeInfo,
+			final TypeInformation<N> namespaceTypeInfo,
+			final StateDescriptor<S, V> stateDescriptor) {
+
+		Preconditions.checkNotNull(jobId);
+		Preconditions.checkNotNull(queryableStateName);
+		Preconditions.checkNotNull(key);
+		Preconditions.checkNotNull(namespace);
+
+		Preconditions.checkNotNull(keyTypeInfo);
+		Preconditions.checkNotNull(namespaceTypeInfo);
+		Preconditions.checkNotNull(stateDescriptor);
+
+		TypeSerializer<K> keySerializer = keyTypeInfo.createSerializer(executionConfig);
+		TypeSerializer<N> namespaceSerializer = namespaceTypeInfo.createSerializer(executionConfig);
+
+		stateDescriptor.initializeSerializerUnlessSet(executionConfig);
+
+		final byte[] serializedKeyAndNamespace;
+		try {
+			serializedKeyAndNamespace = KvStateSerializer
+					.serializeKeyAndNamespace(key, keySerializer, namespace, namespaceSerializer);
+		} catch (IOException e) {
+			return FutureUtils.getFailedFuture(e);
+		}
+
+		return getKvState(jobId, queryableStateName, key.hashCode(), serializedKeyAndNamespace).thenApply(
+				stateResponse -> {
+					try {
+						return stateDescriptor.bind(new ImmutableStateBinder(stateResponse.getContent()));
+					} catch (Exception e) {
+						throw new FlinkRuntimeException(e);
+					}
+				});
+	}
+
+	/**
+	 * 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
+	 * @return Future holding the serialized result
+	 */
+	private CompletableFuture<KvStateResponse> getKvState(
+			final JobID jobId,
+			final String queryableStateName,
+			final int keyHashCode,
+			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/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/VoidNamespace.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/VoidNamespace.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/VoidNamespace.java
new file mode 100644
index 0000000..0560ec2
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/VoidNamespace.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.client;
+
+import org.apache.flink.annotation.Internal;
+
+import java.io.ObjectStreamException;
+
+/**
+ * Singleton placeholder class for state without a namespace.
+ *
+ * <p><b>THIS WAS COPIED FROM RUNTIME SO THAT WE AVOID THE DEPENDENCY.</b>
+ */
+@Internal
+public final class VoidNamespace {
+
+	// ------------------------------------------------------------------------
+	//  Singleton instance.
+	// ------------------------------------------------------------------------
+
+	/** The singleton instance. */
+	public static final VoidNamespace INSTANCE = new VoidNamespace();
+
+	/** Getter for the singleton instance. */
+	public static VoidNamespace get() {
+		return INSTANCE;
+	}
+
+	/** This class should not be instantiated. */
+	private VoidNamespace() {}
+
+	// ------------------------------------------------------------------------
+	//  Standard Utilities
+	// ------------------------------------------------------------------------
+
+	@Override
+	public int hashCode() {
+		return 99;
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		return obj == this;
+	}
+
+	@Override
+	public String toString() {
+		return getClass().getSimpleName();
+	}
+
+	// ------------------------------------------------------------------------
+	//  Singleton serialization
+	// ------------------------------------------------------------------------
+
+	// make sure that we preserve the singleton properly on serialization
+	private Object readResolve() throws ObjectStreamException {
+		return INSTANCE;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/VoidNamespaceSerializer.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/VoidNamespaceSerializer.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/VoidNamespaceSerializer.java
new file mode 100644
index 0000000..38db705
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/VoidNamespaceSerializer.java
@@ -0,0 +1,96 @@
+/*
+ * 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.Internal;
+import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+
+import java.io.IOException;
+
+/**
+ * Serializer for {@link VoidNamespace}.
+ *
+ * <p><b>THIS WAS COPIED FROM RUNTIME SO THAT WE AVOID THE DEPENDENCY.</b>
+ */
+@Internal
+public final class VoidNamespaceSerializer extends TypeSerializerSingleton<VoidNamespace> {
+
+	private static final long serialVersionUID = 1L;
+
+	public static final VoidNamespaceSerializer INSTANCE = new VoidNamespaceSerializer();
+
+	@Override
+	public boolean isImmutableType() {
+		return true;
+	}
+
+	@Override
+	public VoidNamespace createInstance() {
+		return VoidNamespace.get();
+	}
+
+	@Override
+	public VoidNamespace copy(VoidNamespace from) {
+		return VoidNamespace.get();
+	}
+
+	@Override
+	public VoidNamespace copy(VoidNamespace from, VoidNamespace reuse) {
+		return VoidNamespace.get();
+	}
+
+	@Override
+	public int getLength() {
+		return 0;
+	}
+
+	@Override
+	public void serialize(VoidNamespace record, DataOutputView target) throws IOException {
+		// Make progress in the stream, write one byte.
+		//
+		// We could just skip writing anything here, because of the way this is
+		// used with the state backends, but if it is ever used somewhere else
+		// (even though it is unlikely to happen), it would be a problem.
+		target.write(0);
+	}
+
+	@Override
+	public VoidNamespace deserialize(DataInputView source) throws IOException {
+		source.readByte();
+		return VoidNamespace.get();
+	}
+
+	@Override
+	public VoidNamespace deserialize(VoidNamespace reuse, DataInputView source) throws IOException {
+		source.readByte();
+		return VoidNamespace.get();
+	}
+
+	@Override
+	public void copy(DataInputView source, DataOutputView target) throws IOException {
+		target.write(source.readByte());
+	}
+
+	@Override
+	public boolean canEqual(Object obj) {
+		return obj instanceof VoidNamespaceSerializer;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/VoidNamespaceTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/VoidNamespaceTypeInfo.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/VoidNamespaceTypeInfo.java
new file mode 100644
index 0000000..2efb87b
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/VoidNamespaceTypeInfo.java
@@ -0,0 +1,92 @@
+/*
+ * 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.Internal;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+
+/**
+ * {@link TypeInformation} for {@link VoidNamespace}.
+ *
+ * <p><b>THIS WAS COPIED FROM RUNTIME SO THAT WE AVOID THE DEPENDENCY.</b>
+ */
+@Internal
+public class VoidNamespaceTypeInfo extends TypeInformation<VoidNamespace> {
+
+	private static final long serialVersionUID = 5453679706408610586L;
+
+	public static final VoidNamespaceTypeInfo INSTANCE = new VoidNamespaceTypeInfo();
+
+	@Override
+	public boolean isBasicType() {
+		return false;
+	}
+
+	@Override
+	public boolean isTupleType() {
+		return false;
+	}
+
+	@Override
+	public int getArity() {
+		return 0;
+	}
+
+	@Override
+	public int getTotalFields() {
+		return 0;
+	}
+
+	@Override
+	public Class<VoidNamespace> getTypeClass() {
+		return VoidNamespace.class;
+	}
+
+	@Override
+	public boolean isKeyType() {
+		return false;
+	}
+
+	@Override
+	public TypeSerializer<VoidNamespace> createSerializer(ExecutionConfig config) {
+		return VoidNamespaceSerializer.INSTANCE;
+	}
+
+	@Override
+	public String toString() {
+		return "VoidNamespaceTypeInfo";
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		return this == obj || obj instanceof VoidNamespaceTypeInfo;
+	}
+
+	@Override
+	public int hashCode() {
+		return 0;
+	}
+
+	@Override
+	public boolean canEqual(Object obj) {
+		return obj instanceof VoidNamespaceTypeInfo;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableAggregatingState.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableAggregatingState.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableAggregatingState.java
new file mode 100644
index 0000000..8964fbf
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableAggregatingState.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.client.state;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.state.AggregatingState;
+import org.apache.flink.api.common.state.AggregatingStateDescriptor;
+import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+
+/**
+ * A read-only {@link AggregatingState} that <b>does not</b> allow for modifications.
+ *
+ * <p>This is the type of the result returned when querying Flink's keyed state using the
+ * {@link org.apache.flink.queryablestate.client.QueryableStateClient Queryable State Client} and
+ * providing an {@link AggregatingStateDescriptor}.
+ */
+@PublicEvolving
+public final class ImmutableAggregatingState<IN, OUT> extends ImmutableState implements AggregatingState<IN, OUT> {
+
+	private final OUT value;
+
+	private ImmutableAggregatingState(OUT value) {
+		this.value = Preconditions.checkNotNull(value);
+	}
+
+	@Override
+	public OUT get() {
+		return value;
+	}
+
+	@Override
+	public void add(Object newValue) {
+		throw MODIFICATION_ATTEMPT_ERROR;
+	}
+
+	@Override
+	public void clear() {
+		throw MODIFICATION_ATTEMPT_ERROR;
+	}
+
+	public static <IN, ACC, OUT> ImmutableAggregatingState<IN, OUT> createState(
+			final AggregatingStateDescriptor<IN, ACC, OUT> stateDescriptor,
+			final byte[] serializedValue) throws IOException {
+
+		final ACC accumulator = KvStateSerializer.deserializeValue(
+				serializedValue,
+				stateDescriptor.getSerializer());
+
+		final OUT state = stateDescriptor.getAggregateFunction().getResult(accumulator);
+		return new ImmutableAggregatingState<>(state);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableFoldingState.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableFoldingState.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableFoldingState.java
new file mode 100644
index 0000000..25f3118
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableFoldingState.java
@@ -0,0 +1,70 @@
+/*
+ * 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.state;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.state.FoldingState;
+import org.apache.flink.api.common.state.FoldingStateDescriptor;
+import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+
+/**
+ * A read-only {@link FoldingState} that does not allow for modifications.
+ *
+ * <p>This is the result returned when querying Flink's keyed state using the
+ * {@link org.apache.flink.queryablestate.client.QueryableStateClient Queryable State Client} and
+ * providing an {@link FoldingStateDescriptor}.
+ */
+@PublicEvolving
+@Deprecated
+public final class ImmutableFoldingState<IN, ACC> extends ImmutableState implements FoldingState<IN, ACC> {
+
+	private final ACC value;
+
+	private ImmutableFoldingState(ACC value) {
+		this.value = Preconditions.checkNotNull(value);
+	}
+
+	@Override
+	public ACC get() {
+		return value;
+	}
+
+	@Override
+	public void add(Object newValue) {
+		throw MODIFICATION_ATTEMPT_ERROR;
+	}
+
+	@Override
+	public void clear() {
+		throw MODIFICATION_ATTEMPT_ERROR;
+	}
+
+	public static <IN, ACC> ImmutableFoldingState<IN, ACC> createState(
+			final FoldingStateDescriptor<IN, ACC> stateDescriptor,
+			final byte[] serializedState) throws IOException {
+
+		final ACC state = KvStateSerializer.deserializeValue(
+				serializedState,
+				stateDescriptor.getSerializer());
+		return new ImmutableFoldingState<>(state);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableListState.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableListState.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableListState.java
new file mode 100644
index 0000000..3dcd75d
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableListState.java
@@ -0,0 +1,70 @@
+/*
+ * 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.state;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * A read-only {@link ListState} that does not allow for modifications.
+ *
+ * <p>This is the result returned when querying Flink's keyed state using the
+ * {@link org.apache.flink.queryablestate.client.QueryableStateClient Queryable State Client} and
+ * providing an {@link ListStateDescriptor}.
+ */
+@PublicEvolving
+public final class ImmutableListState<V> extends ImmutableState implements ListState<V> {
+
+	private final List<V> listState;
+
+	private ImmutableListState(final List<V> state) {
+		this.listState = Preconditions.checkNotNull(state);
+	}
+
+	@Override
+	public Iterable<V> get() {
+		return listState;
+	}
+
+	@Override
+	public void add(V value) {
+		throw MODIFICATION_ATTEMPT_ERROR;
+	}
+
+	@Override
+	public void clear() {
+		throw MODIFICATION_ATTEMPT_ERROR;
+	}
+
+	public static <V> ImmutableListState<V> createState(
+			final ListStateDescriptor<V> stateDescriptor,
+			final byte[] serializedState) throws IOException {
+
+		final List<V> state = KvStateSerializer.deserializeList(
+				serializedState,
+				stateDescriptor.getElementSerializer());
+		return new ImmutableListState<>(state);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableMapState.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableMapState.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableMapState.java
new file mode 100644
index 0000000..bb08cf0
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableMapState.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.queryablestate.client.state;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.state.MapState;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * A read-only {@link MapState} that does not allow for modifications.
+ *
+ * <p>This is the result returned when querying Flink's keyed state using the
+ * {@link org.apache.flink.queryablestate.client.QueryableStateClient Queryable State Client} and
+ * providing an {@link MapStateDescriptor}.
+ */
+@PublicEvolving
+public final class ImmutableMapState<K, V> extends ImmutableState implements MapState<K, V> {
+
+	private final Map<K, V> state;
+
+	private ImmutableMapState(final Map<K, V> mapState) {
+		this.state = Preconditions.checkNotNull(mapState);
+	}
+
+	@Override
+	public V get(K key) {
+		return state.get(key);
+	}
+
+	@Override
+	public void put(K key, V value) {
+		throw MODIFICATION_ATTEMPT_ERROR;
+	}
+
+	@Override
+	public void putAll(Map<K, V> map) {
+		throw MODIFICATION_ATTEMPT_ERROR;
+	}
+
+	@Override
+	public void remove(K key) {
+		throw MODIFICATION_ATTEMPT_ERROR;
+	}
+
+	@Override
+	public boolean contains(K key) {
+		return state.containsKey(key);
+	}
+
+	/**
+	 * Returns all the mappings in the state in a {@link Collections#unmodifiableSet(Set)}.
+	 *
+	 * @return A read-only iterable view of all the key-value pairs in the state.
+	 *
+	 * @throws Exception Thrown if the system cannot access the state.
+	 */
+	@Override
+	public Iterable<Map.Entry<K, V>> entries() {
+		return Collections.unmodifiableSet(state.entrySet());
+	}
+
+	/**
+	 * Returns all the keys in the state in a {@link Collections#unmodifiableSet(Set)}.
+	 *
+	 * @return A read-only iterable view of all the keys in the state.
+	 *
+	 * @throws Exception Thrown if the system cannot access the state.
+	 */
+	@Override
+	public Iterable<K> keys() {
+		return Collections.unmodifiableSet(state.keySet());
+	}
+
+	/**
+	 * Returns all the values in the state in a {@link Collections#unmodifiableCollection(Collection)}.
+	 *
+	 * @return A read-only iterable view of all the values in the state.
+	 *
+	 * @throws Exception Thrown if the system cannot access the state.
+	 */
+	@Override
+	public Iterable<V> values() {
+		return Collections.unmodifiableCollection(state.values());
+	}
+
+	/**
+	 * Iterates over all the mappings in the state. The iterator cannot
+	 * remove elements.
+	 *
+	 * @return A read-only iterator over all the mappings in the state
+	 *
+	 * @throws Exception Thrown if the system cannot access the state.
+	 */
+	@Override
+	public Iterator<Map.Entry<K, V>> iterator() {
+		return Collections.unmodifiableSet(state.entrySet()).iterator();
+	}
+
+	@Override
+	public void clear() {
+		throw MODIFICATION_ATTEMPT_ERROR;
+	}
+
+	public static <K, V> ImmutableMapState<K, V> createState(
+			final MapStateDescriptor<K, V> stateDescriptor,
+			final byte[] serializedState) throws IOException {
+
+		final Map<K, V> state = KvStateSerializer.deserializeMap(
+				serializedState,
+				stateDescriptor.getKeySerializer(),
+				stateDescriptor.getValueSerializer());
+		return new ImmutableMapState<>(state);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableReducingState.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableReducingState.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableReducingState.java
new file mode 100644
index 0000000..46b477f
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableReducingState.java
@@ -0,0 +1,69 @@
+/*
+ * 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.state;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.state.ReducingState;
+import org.apache.flink.api.common.state.ReducingStateDescriptor;
+import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+
+/**
+ * A read-only {@link ReducingState} that does not allow for modifications.
+ *
+ * <p>This is the result returned when querying Flink's keyed state using the
+ * {@link org.apache.flink.queryablestate.client.QueryableStateClient Queryable State Client} and
+ * providing an {@link ReducingStateDescriptor}.
+ */
+@PublicEvolving
+public final class ImmutableReducingState<V> extends ImmutableState implements ReducingState<V> {
+
+	private final V value;
+
+	private ImmutableReducingState(V value) {
+		this.value = Preconditions.checkNotNull(value);
+	}
+
+	@Override
+	public V get() {
+		return value;
+	}
+
+	@Override
+	public void add(V newValue) {
+		throw MODIFICATION_ATTEMPT_ERROR;
+	}
+
+	@Override
+	public void clear() {
+		throw MODIFICATION_ATTEMPT_ERROR;
+	}
+
+	public static <V> ImmutableReducingState<V> createState(
+			final ReducingStateDescriptor<V> stateDescriptor,
+			final byte[] serializedState) throws IOException {
+
+		final V state = KvStateSerializer.deserializeValue(
+				serializedState,
+				stateDescriptor.getSerializer());
+		return new ImmutableReducingState<>(state);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableState.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableState.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableState.java
new file mode 100644
index 0000000..863f07b
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableState.java
@@ -0,0 +1,29 @@
+/*
+ * 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.state;
+
+/**
+ * A base class for the <b>read-only</b> types of state returned
+ * as results from the Queryable State Client.
+ */
+abstract class ImmutableState {
+
+	protected static final UnsupportedOperationException MODIFICATION_ATTEMPT_ERROR =
+			new UnsupportedOperationException("State is read-only. No modifications allowed.");
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableStateBinder.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableStateBinder.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableStateBinder.java
new file mode 100644
index 0000000..6ce2787
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableStateBinder.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.queryablestate.client.state;
+
+import org.apache.flink.api.common.state.AggregatingState;
+import org.apache.flink.api.common.state.AggregatingStateDescriptor;
+import org.apache.flink.api.common.state.FoldingState;
+import org.apache.flink.api.common.state.FoldingStateDescriptor;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.MapState;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.state.ReducingState;
+import org.apache.flink.api.common.state.ReducingStateDescriptor;
+import org.apache.flink.api.common.state.StateBinder;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.util.Preconditions;
+
+/**
+ * A {@link StateBinder} used to deserialize the results returned by the
+ * {@link org.apache.flink.queryablestate.client.QueryableStateClient}.
+ *
+ * <p>The result is an immutable {@link org.apache.flink.api.common.state.State State}
+ * object containing the requested result.
+ */
+public class ImmutableStateBinder implements StateBinder {
+
+	private final byte[] serializedState;
+
+	public ImmutableStateBinder(final byte[] content) {
+		serializedState = Preconditions.checkNotNull(content);
+	}
+
+	@Override
+	public <T> ValueState<T> createValueState(ValueStateDescriptor<T> stateDesc) throws Exception {
+		return ImmutableValueState.createState(stateDesc, serializedState);
+	}
+
+	@Override
+	public <T> ListState<T> createListState(ListStateDescriptor<T> stateDesc) throws Exception {
+		return ImmutableListState.createState(stateDesc, serializedState);
+	}
+
+	@Override
+	public <T> ReducingState<T> createReducingState(ReducingStateDescriptor<T> stateDesc) throws Exception {
+		return ImmutableReducingState.createState(stateDesc, serializedState);
+	}
+
+	@Override
+	public <IN, ACC, OUT> AggregatingState<IN, OUT> createAggregatingState(AggregatingStateDescriptor<IN, ACC, OUT> stateDesc) throws Exception {
+		return ImmutableAggregatingState.createState(stateDesc, serializedState);
+	}
+
+	@Override
+	public <T, ACC> FoldingState<T, ACC> createFoldingState(FoldingStateDescriptor<T, ACC> stateDesc) throws Exception {
+		return ImmutableFoldingState.createState(stateDesc, serializedState);
+	}
+
+	@Override
+	public <MK, MV> MapState<MK, MV> createMapState(MapStateDescriptor<MK, MV> stateDesc) throws Exception {
+		return ImmutableMapState.createState(stateDesc, serializedState);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableValueState.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableValueState.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableValueState.java
new file mode 100644
index 0000000..f3ddd2b
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableValueState.java
@@ -0,0 +1,69 @@
+/*
+ * 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.state;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+
+/**
+ * A read-only {@link ValueState} that does not allow for modifications.
+ *
+ * <p>This is the result returned when querying Flink's keyed state using the
+ * {@link org.apache.flink.queryablestate.client.QueryableStateClient Queryable State Client} and
+ * providing an {@link ValueStateDescriptor}.
+ */
+@PublicEvolving
+public final class ImmutableValueState<V> extends ImmutableState implements ValueState<V> {
+
+	private final V value;
+
+	private ImmutableValueState(V value) {
+		this.value = Preconditions.checkNotNull(value);
+	}
+
+	@Override
+	public V value() {
+		return value;
+	}
+
+	@Override
+	public void update(V newValue) {
+		throw MODIFICATION_ATTEMPT_ERROR;
+	}
+
+	@Override
+	public void clear() {
+		throw MODIFICATION_ATTEMPT_ERROR;
+	}
+
+	public static <V> ImmutableValueState<V> createState(
+			final ValueStateDescriptor<V> stateDescriptor,
+			final byte[] serializedState) throws IOException {
+
+		final V state = KvStateSerializer.deserializeValue(
+				serializedState,
+				stateDescriptor.getSerializer());
+		return new ImmutableValueState<>(state);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/serialization/DataInputDeserializer.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/serialization/DataInputDeserializer.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/serialization/DataInputDeserializer.java
new file mode 100644
index 0000000..878df85
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/serialization/DataInputDeserializer.java
@@ -0,0 +1,392 @@
+/*
+ * 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.state.serialization;
+
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.MemoryUtils;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UTFDataFormatException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+/**
+ * A simple and efficient deserializer for the {@link java.io.DataInput} interface.
+ *
+ * <p><b>THIS WAS COPIED FROM RUNTIME SO THAT WE AVOID THE DEPENDENCY.</b>
+ */
+public class DataInputDeserializer implements DataInputView, java.io.Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	// ------------------------------------------------------------------------
+
+	private byte[] buffer;
+
+	private int end;
+
+	private int position;
+
+	// ------------------------------------------------------------------------
+
+	public DataInputDeserializer() {}
+
+	public DataInputDeserializer(byte[] buffer) {
+		setBuffer(buffer, 0, buffer.length);
+	}
+
+	public DataInputDeserializer(byte[] buffer, int start, int len) {
+		setBuffer(buffer, start, len);
+	}
+
+	public DataInputDeserializer(ByteBuffer buffer) {
+		setBuffer(buffer);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Changing buffers
+	// ------------------------------------------------------------------------
+
+	public void setBuffer(ByteBuffer buffer) {
+		if (buffer.hasArray()) {
+			this.buffer = buffer.array();
+			this.position = buffer.arrayOffset() + buffer.position();
+			this.end = this.position + buffer.remaining();
+		} else if (buffer.isDirect()) {
+			this.buffer = new byte[buffer.remaining()];
+			this.position = 0;
+			this.end = this.buffer.length;
+
+			buffer.get(this.buffer);
+		} else {
+			throw new IllegalArgumentException("The given buffer is neither an array-backed heap ByteBuffer, nor a direct ByteBuffer.");
+		}
+	}
+
+	public void setBuffer(byte[] buffer, int start, int len) {
+		if (buffer == null) {
+			throw new NullPointerException();
+		}
+
+		if (start < 0 || len < 0 || start + len > buffer.length) {
+			throw new IllegalArgumentException();
+		}
+
+		this.buffer = buffer;
+		this.position = start;
+		this.end = start + len;
+	}
+
+	public void releaseArrays() {
+		this.buffer = null;
+	}
+
+	// ----------------------------------------------------------------------------------------
+	//                               Data Input
+	// ----------------------------------------------------------------------------------------
+
+	public int available() {
+		if (position < end) {
+			return end - position;
+		} else {
+			return 0;
+		}
+	}
+
+	@Override
+	public boolean readBoolean() throws IOException {
+		if (this.position < this.end) {
+			return this.buffer[this.position++] != 0;
+		} else {
+			throw new EOFException();
+		}
+	}
+
+	@Override
+	public byte readByte() throws IOException {
+		if (this.position < this.end) {
+			return this.buffer[this.position++];
+		} else {
+			throw new EOFException();
+		}
+	}
+
+	@Override
+	public char readChar() throws IOException {
+		if (this.position < this.end - 1) {
+			return (char) (((this.buffer[this.position++] & 0xff) << 8) | (this.buffer[this.position++] & 0xff));
+		} else {
+			throw new EOFException();
+		}
+	}
+
+	@Override
+	public double readDouble() throws IOException {
+		return Double.longBitsToDouble(readLong());
+	}
+
+	@Override
+	public float readFloat() throws IOException {
+		return Float.intBitsToFloat(readInt());
+	}
+
+	@Override
+	public void readFully(byte[] b) throws IOException {
+		readFully(b, 0, b.length);
+	}
+
+	@Override
+	public void readFully(byte[] b, int off, int len) throws IOException {
+		if (len >= 0) {
+			if (off <= b.length - len) {
+				if (this.position <= this.end - len) {
+					System.arraycopy(this.buffer, position, b, off, len);
+					position += len;
+				} else {
+					throw new EOFException();
+				}
+			} else {
+				throw new ArrayIndexOutOfBoundsException();
+			}
+		} else if (len < 0) {
+			throw new IllegalArgumentException("Length may not be negative.");
+		}
+	}
+
+	@Override
+	public int readInt() throws IOException {
+		if (this.position >= 0 && this.position < this.end - 3) {
+			@SuppressWarnings("restriction")
+			int value = UNSAFE.getInt(this.buffer, BASE_OFFSET + this.position);
+			if (LITTLE_ENDIAN) {
+				value = Integer.reverseBytes(value);
+			}
+
+			this.position += 4;
+			return value;
+		} else {
+			throw new EOFException();
+		}
+	}
+
+	@Override
+	public String readLine() throws IOException {
+		if (this.position < this.end) {
+			// read until a newline is found
+			StringBuilder bld = new StringBuilder();
+			char curr = (char) readUnsignedByte();
+			while (position < this.end && curr != '\n') {
+				bld.append(curr);
+				curr = (char) readUnsignedByte();
+			}
+			// trim a trailing carriage return
+			int len = bld.length();
+			if (len > 0 && bld.charAt(len - 1) == '\r') {
+				bld.setLength(len - 1);
+			}
+			String s = bld.toString();
+			bld.setLength(0);
+			return s;
+		} else {
+			return null;
+		}
+	}
+
+	@Override
+	public long readLong() throws IOException {
+		if (position >= 0 && position < this.end - 7) {
+			@SuppressWarnings("restriction")
+			long value = UNSAFE.getLong(this.buffer, BASE_OFFSET + this.position);
+			if (LITTLE_ENDIAN) {
+				value = Long.reverseBytes(value);
+			}
+			this.position += 8;
+			return value;
+		} else {
+			throw new EOFException();
+		}
+	}
+
+	@Override
+	public short readShort() throws IOException {
+		if (position >= 0 && position < this.end - 1) {
+			return (short) ((((this.buffer[position++]) & 0xff) << 8) | ((this.buffer[position++]) & 0xff));
+		} else {
+			throw new EOFException();
+		}
+	}
+
+	@Override
+	public String readUTF() throws IOException {
+		int utflen = readUnsignedShort();
+		byte[] bytearr = new byte[utflen];
+		char[] chararr = new char[utflen];
+
+		int c, char2, char3;
+		int count = 0;
+		int chararrCount = 0;
+
+		readFully(bytearr, 0, utflen);
+
+		while (count < utflen) {
+			c = (int) bytearr[count] & 0xff;
+			if (c > 127) {
+				break;
+			}
+			count++;
+			chararr[chararrCount++] = (char) c;
+		}
+
+		while (count < utflen) {
+			c = (int) bytearr[count] & 0xff;
+			switch (c >> 4) {
+			case 0:
+			case 1:
+			case 2:
+			case 3:
+			case 4:
+			case 5:
+			case 6:
+			case 7:
+				/* 0xxxxxxx */
+				count++;
+				chararr[chararrCount++] = (char) c;
+				break;
+			case 12:
+			case 13:
+				/* 110x xxxx 10xx xxxx */
+				count += 2;
+				if (count > utflen) {
+					throw new UTFDataFormatException("malformed input: partial character at end");
+				}
+				char2 = (int) bytearr[count - 1];
+				if ((char2 & 0xC0) != 0x80) {
+					throw new UTFDataFormatException("malformed input around byte " + count);
+				}
+				chararr[chararrCount++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
+				break;
+			case 14:
+				/* 1110 xxxx 10xx xxxx 10xx xxxx */
+				count += 3;
+				if (count > utflen) {
+					throw new UTFDataFormatException("malformed input: partial character at end");
+				}
+				char2 = (int) bytearr[count - 2];
+				char3 = (int) bytearr[count - 1];
+				if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) {
+					throw new UTFDataFormatException("malformed input around byte " + (count - 1));
+				}
+				chararr[chararrCount++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | (char3 & 0x3F));
+				break;
+			default:
+				/* 10xx xxxx, 1111 xxxx */
+				throw new UTFDataFormatException("malformed input around byte " + count);
+			}
+		}
+		// The number of chars produced may be less than utflen
+		return new String(chararr, 0, chararrCount);
+	}
+
+	@Override
+	public int readUnsignedByte() throws IOException {
+		if (this.position < this.end) {
+			return (this.buffer[this.position++] & 0xff);
+		} else {
+			throw new EOFException();
+		}
+	}
+
+	@Override
+	public int readUnsignedShort() throws IOException {
+		if (this.position < this.end - 1) {
+			return ((this.buffer[this.position++] & 0xff) << 8) | (this.buffer[this.position++] & 0xff);
+		} else {
+			throw new EOFException();
+		}
+	}
+
+	@Override
+	public int skipBytes(int n) throws IOException {
+		if (this.position <= this.end - n) {
+			this.position += n;
+			return n;
+		} else {
+			n = this.end - this.position;
+			this.position = this.end;
+			return n;
+		}
+	}
+
+	@Override
+	public void skipBytesToRead(int numBytes) throws IOException {
+		int skippedBytes = skipBytes(numBytes);
+
+		if (skippedBytes < numBytes){
+			throw new EOFException("Could not skip " + numBytes + " bytes.");
+		}
+	}
+
+	@Override
+	public int read(byte[] b, int off, int len) throws IOException {
+		if (b == null){
+			throw new NullPointerException("Byte array b cannot be null.");
+		}
+
+		if (off < 0){
+			throw new IndexOutOfBoundsException("Offset cannot be negative.");
+		}
+
+		if (len < 0){
+			throw new IndexOutOfBoundsException("Length cannot be negative.");
+		}
+
+		if (b.length - off < len){
+			throw new IndexOutOfBoundsException("Byte array does not provide enough space to store requested data" +
+					".");
+		}
+
+		if (this.position >= this.end) {
+			return -1;
+		} else {
+			int toRead = Math.min(this.end - this.position, len);
+			System.arraycopy(this.buffer, this.position, b, off, toRead);
+			this.position += toRead;
+
+			return toRead;
+		}
+	}
+
+	@Override
+	public int read(byte[] b) throws IOException {
+		return read(b, 0, b.length);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	@SuppressWarnings("restriction")
+	private static final sun.misc.Unsafe UNSAFE = MemoryUtils.UNSAFE;
+
+	@SuppressWarnings("restriction")
+	private static final long BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
+
+	private static final boolean LITTLE_ENDIAN = (MemoryUtils.NATIVE_BYTE_ORDER == ByteOrder.LITTLE_ENDIAN);
+}


[05/13] flink git commit: [FLINK-7908][QS] Restructure the queryable state module.

Posted by kk...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableListStateTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableListStateTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableListStateTest.java
deleted file mode 100644
index 3283295..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableListStateTest.java
+++ /dev/null
@@ -1,112 +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.state;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
-import org.apache.flink.queryablestate.client.state.ImmutableListState;
-import org.apache.flink.runtime.state.heap.HeapListState;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Tests the {@link ImmutableListState}.
- */
-public class ImmutableListStateTest {
-
-	private final ListStateDescriptor<Long> listStateDesc =
-			new ListStateDescriptor<>("test", BasicTypeInfo.LONG_TYPE_INFO);
-
-	private ImmutableListState<Long> listState;
-
-	@Before
-	public void setUp() throws Exception {
-		if (!listStateDesc.isSerializerInitialized()) {
-			listStateDesc.initializeSerializerUnlessSet(new ExecutionConfig());
-		}
-
-		List<Long> init = new ArrayList<>();
-		init.add(42L);
-
-		byte[] serInit = serializeInitValue(init);
-		listState = ImmutableListState.createState(listStateDesc, serInit);
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testUpdate() {
-		List<Long> list = getStateContents();
-		assertEquals(1L, list.size());
-
-		long element = list.get(0);
-		assertEquals(42L, element);
-
-		listState.add(54L);
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testClear() {
-		List<Long> list = getStateContents();
-		assertEquals(1L, list.size());
-
-		long element = list.get(0);
-		assertEquals(42L, element);
-
-		listState.clear();
-	}
-
-	/**
-	 * Copied from {@link HeapListState#getSerializedValue(Object, Object)}.
-	 */
-	private byte[] serializeInitValue(List<Long> toSerialize) throws IOException {
-		TypeSerializer<Long> serializer = listStateDesc.getElementSerializer();
-
-		ByteArrayOutputStream baos = new ByteArrayOutputStream();
-		DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(baos);
-
-		// write the same as RocksDB writes lists, with one ',' separator
-		for (int i = 0; i < toSerialize.size(); i++) {
-			serializer.serialize(toSerialize.get(i), view);
-			if (i < toSerialize.size() - 1) {
-				view.writeByte(',');
-			}
-		}
-		view.flush();
-
-		return baos.toByteArray();
-	}
-
-	private List<Long> getStateContents() {
-		List<Long> list = new ArrayList<>();
-		for (Long elem: listState.get()) {
-			list.add(elem);
-		}
-		return list;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableMapStateTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableMapStateTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableMapStateTest.java
deleted file mode 100644
index 30a8a50..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableMapStateTest.java
+++ /dev/null
@@ -1,189 +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.state;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.state.MapStateDescriptor;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.queryablestate.client.state.ImmutableMapState;
-import org.apache.flink.runtime.query.netty.message.KvStateSerializer;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Tests the {@link ImmutableMapState}.
- */
-public class ImmutableMapStateTest {
-
-	private final MapStateDescriptor<Long, Long> mapStateDesc =
-			new MapStateDescriptor<>(
-					"test",
-					BasicTypeInfo.LONG_TYPE_INFO,
-					BasicTypeInfo.LONG_TYPE_INFO);
-
-	private ImmutableMapState<Long, Long> mapState;
-
-	@Before
-	public void setUp() throws Exception {
-		if (!mapStateDesc.isSerializerInitialized()) {
-			mapStateDesc.initializeSerializerUnlessSet(new ExecutionConfig());
-		}
-
-		Map<Long, Long> initMap = new HashMap<>();
-		initMap.put(1L, 5L);
-		initMap.put(2L, 5L);
-
-		byte[] initSer = KvStateSerializer.serializeMap(
-				initMap.entrySet(),
-				BasicTypeInfo.LONG_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				BasicTypeInfo.LONG_TYPE_INFO.createSerializer(new ExecutionConfig()));
-
-		mapState = ImmutableMapState.createState(mapStateDesc, initSer);
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testPut() {
-		assertTrue(mapState.contains(1L));
-		long value = mapState.get(1L);
-		assertEquals(5L, value);
-
-		assertTrue(mapState.contains(2L));
-		value = mapState.get(2L);
-		assertEquals(5L, value);
-
-		mapState.put(2L, 54L);
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testPutAll() {
-		assertTrue(mapState.contains(1L));
-		long value = mapState.get(1L);
-		assertEquals(5L, value);
-
-		assertTrue(mapState.contains(2L));
-		value = mapState.get(2L);
-		assertEquals(5L, value);
-
-		Map<Long, Long> nMap = new HashMap<>();
-		nMap.put(1L, 7L);
-		nMap.put(2L, 7L);
-
-		mapState.putAll(nMap);
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testUpdate() {
-		assertTrue(mapState.contains(1L));
-		long value = mapState.get(1L);
-		assertEquals(5L, value);
-
-		assertTrue(mapState.contains(2L));
-		value = mapState.get(2L);
-		assertEquals(5L, value);
-
-		mapState.put(2L, 54L);
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testIterator() {
-		assertTrue(mapState.contains(1L));
-		long value = mapState.get(1L);
-		assertEquals(5L, value);
-
-		assertTrue(mapState.contains(2L));
-		value = mapState.get(2L);
-		assertEquals(5L, value);
-
-		Iterator<Map.Entry<Long, Long>> iterator = mapState.iterator();
-		while (iterator.hasNext()) {
-			iterator.remove();
-		}
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testIterable() {
-		assertTrue(mapState.contains(1L));
-		long value = mapState.get(1L);
-		assertEquals(5L, value);
-
-		assertTrue(mapState.contains(2L));
-		value = mapState.get(2L);
-		assertEquals(5L, value);
-
-		Iterable<Map.Entry<Long, Long>> iterable = mapState.entries();
-		Iterator<Map.Entry<Long, Long>> iterator = iterable.iterator();
-		while (iterator.hasNext()) {
-			assertEquals(5L, (long) iterator.next().getValue());
-			iterator.remove();
-		}
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testKeys() {
-		assertTrue(mapState.contains(1L));
-		long value = mapState.get(1L);
-		assertEquals(5L, value);
-
-		assertTrue(mapState.contains(2L));
-		value = mapState.get(2L);
-		assertEquals(5L, value);
-
-		Iterator<Long> iterator = mapState.keys().iterator();
-		while (iterator.hasNext()) {
-			iterator.remove();
-		}
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testValues() {
-		assertTrue(mapState.contains(1L));
-		long value = mapState.get(1L);
-		assertEquals(5L, value);
-
-		assertTrue(mapState.contains(2L));
-		value = mapState.get(2L);
-		assertEquals(5L, value);
-
-		Iterator<Long> iterator = mapState.values().iterator();
-		while (iterator.hasNext()) {
-			iterator.remove();
-		}
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testClear() {
-		assertTrue(mapState.contains(1L));
-		long value = mapState.get(1L);
-		assertEquals(5L, value);
-
-		assertTrue(mapState.contains(2L));
-		value = mapState.get(2L);
-		assertEquals(5L, value);
-
-		mapState.clear();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableReducingStateTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableReducingStateTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableReducingStateTest.java
deleted file mode 100644
index 9b1ecf8..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableReducingStateTest.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.queryablestate.state;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.state.ReducingStateDescriptor;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.queryablestate.client.state.ImmutableReducingState;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import java.nio.ByteBuffer;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Tests the {@link ImmutableReducingState}.
- */
-public class ImmutableReducingStateTest {
-
-	private final ReducingStateDescriptor<Long> reducingStateDesc =
-			new ReducingStateDescriptor<>("test", new SumReduce(), BasicTypeInfo.LONG_TYPE_INFO);
-
-	private ImmutableReducingState<Long> reduceState;
-
-	@Before
-	public void setUp() throws Exception {
-		if (!reducingStateDesc.isSerializerInitialized()) {
-			reducingStateDesc.initializeSerializerUnlessSet(new ExecutionConfig());
-		}
-
-		reduceState = ImmutableReducingState.createState(
-				reducingStateDesc,
-				ByteBuffer.allocate(Long.BYTES).putLong(42L).array()
-		);
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testUpdate() {
-		long value = reduceState.get();
-		assertEquals(42L, value);
-
-		reduceState.add(54L);
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testClear() {
-		long value = reduceState.get();
-		assertEquals(42L, value);
-
-		reduceState.clear();
-	}
-
-	/**
-	 * Test {@link ReduceFunction} summing up its two arguments.
-	 */
-	private static class SumReduce implements ReduceFunction<Long> {
-
-		private static final long serialVersionUID = 6041237513913189144L;
-
-		@Override
-		public Long reduce(Long value1, Long value2) throws Exception {
-			return value1 + value2;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableValueStateTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableValueStateTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableValueStateTest.java
deleted file mode 100644
index 5f7032d..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableValueStateTest.java
+++ /dev/null
@@ -1,70 +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.state;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.queryablestate.client.state.ImmutableValueState;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import java.nio.ByteBuffer;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Tests the {@link ImmutableValueState}.
- */
-public class ImmutableValueStateTest {
-
-	private final ValueStateDescriptor<Long> valueStateDesc =
-			new ValueStateDescriptor<>("test", BasicTypeInfo.LONG_TYPE_INFO);
-
-	private ImmutableValueState<Long> valueState;
-
-	@Before
-	public void setUp() throws Exception {
-		if (!valueStateDesc.isSerializerInitialized()) {
-			valueStateDesc.initializeSerializerUnlessSet(new ExecutionConfig());
-		}
-
-		valueState = ImmutableValueState.createState(
-				valueStateDesc,
-				ByteBuffer.allocate(Long.BYTES).putLong(42L).array()
-		);
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testUpdate() {
-		long value = valueState.value();
-		assertEquals(42L, value);
-
-		valueState.update(54L);
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testClear() {
-		long value = valueState.value();
-		assertEquals(42L, value);
-
-		valueState.clear();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/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
deleted file mode 100644
index 10792cd..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/resources/log4j-test.properties
+++ /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.
-#
-
-# 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/0c771505/flink-queryable-state/flink-queryable-state-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-runtime/pom.xml b/flink-queryable-state/flink-queryable-state-runtime/pom.xml
new file mode 100644
index 0000000..f39498e
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-runtime/pom.xml
@@ -0,0 +1,119 @@
+<?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-runtime_${scala.binary.version}</artifactId>
+	<name>flink-queryable-state-runtime</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-runtime_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-queryable-state-client-java_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- ===================================================
+								Testing
+			=================================================== -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-statebackend-rocksdb_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<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>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/0c771505/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyHandler.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyHandler.java b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyHandler.java
new file mode 100644
index 0000000..d434336
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-runtime/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.KvStateID;
+import org.apache.flink.queryablestate.exceptions.UnknownKvStateIdException;
+import org.apache.flink.queryablestate.exceptions.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.network.stats.DisabledKvStateRequestStats;
+import org.apache.flink.queryablestate.network.stats.KvStateRequestStats;
+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.KvStateLocation;
+import org.apache.flink.runtime.query.KvStateMessage;
+import org.apache.flink.runtime.query.UnknownKvStateLocation;
+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.net.InetSocketAddress;
+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 InetSocketAddress 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/0c771505/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java
new file mode 100644
index 0000000..f473443
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java
@@ -0,0 +1,128 @@
+/*
+ * 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.exceptions.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.queryablestate.network.stats.KvStateRequestStats;
+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.util.Preconditions;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.Iterator;
+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, Iterator, int, int, KvStateRequestStats)
+	 * QueryableStateUtils.createKvStateClientProxy(InetAddress, Iterator, 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 bindPortIterator the port range to try to bind 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 Iterator<Integer> bindPortIterator,
+			final Integer numEventLoopThreads,
+			final Integer numQueryThreads,
+			final KvStateRequestStats stats) {
+
+		super("Queryable State Proxy Server", bindAddress, bindPortIterator, numEventLoopThreads, numQueryThreads);
+		Preconditions.checkArgument(numQueryThreads >= 1, "Non-positive number of query threads.");
+		this.queryExecutorThreads = numQueryThreads;
+		this.stats = Preconditions.checkNotNull(stats);
+	}
+
+	@Override
+	public InetSocketAddress getServerAddress() {
+		return super.getServerAddress();
+	}
+
+	@Override
+	public void start() throws Throwable {
+		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/0c771505/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/messages/KvStateInternalRequest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/messages/KvStateInternalRequest.java b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/messages/KvStateInternalRequest.java
new file mode 100644
index 0000000..8c8de59
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-runtime/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.KvStateID;
+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 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/0c771505/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java
new file mode 100644
index 0000000..476f153
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.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.server;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.queryablestate.exceptions.UnknownKeyOrNamespaceException;
+import org.apache.flink.queryablestate.exceptions.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.stats.KvStateRequestStats;
+import org.apache.flink.runtime.query.KvStateRegistry;
+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.channel.ChannelHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * 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.).
+ */
+@Internal
+@ChannelHandler.Sharable
+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;
+
+	/**
+	 * Create the handler used by the {@link KvStateServerImpl}.
+	 *
+	 * @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(
+			final KvStateServerImpl server,
+			final KvStateRegistry kvStateRegistry,
+			final MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer,
+			final KvStateRequestStats stats) {
+
+		super(server, serializer, stats);
+		this.registry = Preconditions.checkNotNull(kvStateRegistry);
+	}
+
+	@Override
+	public CompletableFuture<KvStateResponse> handleRequest(final long requestId, final KvStateInternalRequest request) {
+		final CompletableFuture<KvStateResponse> responseFuture = new CompletableFuture<>();
+
+		try {
+			final InternalKvState<?> kvState = registry.getKvState(request.getKvStateId());
+			if (kvState == null) {
+				responseFuture.completeExceptionally(new UnknownKvStateIdException(getServerName(), request.getKvStateId()));
+			} else {
+				byte[] serializedKeyAndNamespace = request.getSerializedKeyAndNamespace();
+
+				byte[] serializedResult = kvState.getSerializedValue(serializedKeyAndNamespace);
+				if (serializedResult != null) {
+					responseFuture.complete(new KvStateResponse(serializedResult));
+				} else {
+					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 void shutdown() {
+		// do nothing
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java
new file mode 100644
index 0000000..fe07687
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java
@@ -0,0 +1,111 @@
+/*
+ * 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.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.queryablestate.network.stats.KvStateRequestStats;
+import org.apache.flink.runtime.query.KvStateRegistry;
+import org.apache.flink.runtime.query.KvStateServer;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.Iterator;
+
+/**
+ * The default implementation of the {@link KvStateServer}.
+ */
+@Internal
+public class KvStateServerImpl extends AbstractServerBase<KvStateInternalRequest, KvStateResponse> implements KvStateServer {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KvStateServerImpl.class);
+
+	/** The {@link KvStateRegistry} to query for state instances. */
+	private final KvStateRegistry kvStateRegistry;
+
+	private final KvStateRequestStats stats;
+
+	private MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer;
+
+	/**
+	 * Creates the state server.
+	 *
+	 * <p>The server is instantiated using reflection by the
+	 * {@link org.apache.flink.runtime.query.QueryableStateUtils#createKvStateServer(InetAddress, Iterator, int, int, KvStateRegistry, KvStateRequestStats)
+	 * QueryableStateUtils.createKvStateServer(InetAddress, Iterator, 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 the address to listen to.
+	 * @param bindPortIterator the port range to try to bind 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(
+			final InetAddress bindAddress,
+			final Iterator<Integer> bindPortIterator,
+			final Integer numEventLoopThreads,
+			final Integer numQueryThreads,
+			final KvStateRegistry kvStateRegistry,
+			final KvStateRequestStats stats) {
+
+		super("Queryable State Server", bindAddress, bindPortIterator, numEventLoopThreads, numQueryThreads);
+		this.stats = Preconditions.checkNotNull(stats);
+		this.kvStateRegistry = Preconditions.checkNotNull(kvStateRegistry);
+	}
+
+	@Override
+	public AbstractServerHandler<KvStateInternalRequest, KvStateResponse> initializeHandler() {
+		this.serializer = new MessageSerializer<>(
+				new KvStateInternalRequest.KvStateInternalRequestDeserializer(),
+				new KvStateResponse.KvStateResponseDeserializer());
+		return new KvStateServerHandler(this, kvStateRegistry, serializer, stats);
+	}
+
+	public MessageSerializer<KvStateInternalRequest, KvStateResponse> getSerializer() {
+		Preconditions.checkState(serializer != null, "Server " + getServerName() + " has not been started.");
+		return serializer;
+	}
+
+	@Override
+	public void start() throws Throwable {
+		super.start();
+	}
+
+	@Override
+	public InetSocketAddress getServerAddress() {
+		return super.getServerAddress();
+	}
+
+	@Override
+	public void shutdown() {
+		super.shutdown();
+	}
+}


[04/13] flink git commit: [FLINK-7908][QS] Restructure the queryable state module.

Posted by kk...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java
new file mode 100644
index 0000000..b4bae9c
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java
@@ -0,0 +1,1496 @@
+/*
+ * 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.AggregateFunction;
+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.AggregatingState;
+import org.apache.flink.api.common.state.AggregatingStateDescriptor;
+import org.apache.flink.api.common.state.FoldingState;
+import org.apache.flink.api.common.state.FoldingStateDescriptor;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.MapState;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.state.ReducingState;
+import org.apache.flink.api.common.state.ReducingStateDescriptor;
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.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.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.client.QueryableStateClient;
+import org.apache.flink.queryablestate.client.VoidNamespace;
+import org.apache.flink.queryablestate.client.VoidNamespaceSerializer;
+import org.apache.flink.queryablestate.client.VoidNamespaceTypeInfo;
+import org.apache.flink.queryablestate.exceptions.UnknownKeyOrNamespaceException;
+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;
+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.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.ProcessFunction;
+import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.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.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 AbstractQueryableStateTestBase extends TestLogger {
+
+	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);
+
+	private final ScheduledExecutorService executorService = Executors.newScheduledThreadPool(4);
+	private final ScheduledExecutor executor = new ScheduledExecutorServiceAdapter(executorService);
+
+	/**
+	 * 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;
+
+	/**
+	 * Client shared between all the test.
+	 */
+	protected static QueryableStateClient client;
+
+	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;
+
+		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, 1000L));
+
+			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";
+
+			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<CompletableFuture<ReducingState<Tuple2<Integer, Long>>>> futures = new ArrayList<>(numKeys);
+
+				for (int i = 0; i < numKeys; i++) {
+					final int key = i;
+
+					if (counts.get(key) > 0L) {
+						// Skip this one
+						continue;
+					} else {
+						allNonZero = false;
+					}
+
+					CompletableFuture<ReducingState<Tuple2<Integer, Long>>> result = getKvStateWithRetries(
+							client,
+							jobId,
+							queryName,
+							key,
+							BasicTypeInfo.INT_TYPE_INFO,
+							reducingState,
+							QUERY_RETRY_DELAY,
+							false,
+							executor);
+
+					result.thenAccept(response -> {
+						try {
+							Tuple2<Integer, Long> res = response.get();
+							counts.set(key, res.f1);
+							assertEquals("Key mismatch", key, res.f0.intValue());
+						} catch (Exception e) {
+							Assert.fail(e.getMessage());
+						}
+					});
+
+					futures.add(result);
+				}
+
+				// 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);
+
+			// 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) {
+				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
+						.getLeaderGateway(deadline.timeLeft())
+						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
+						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
+
+				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+			}
+		}
+	}
+
+	/**
+	 * 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, 1000L));
+
+			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();
+
+			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 =
+					failedFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+			assertEquals(JobStatus.FAILED, jobStatus.state());
+
+			// Get the job and check the cause
+			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)))
+					.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+
+			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) {
+				scala.concurrent.Future<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 long numElements = 1024L;
+
+		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, 1000L));
+
+			DataStream<Tuple2<Integer, Long>> source = env
+					.addSource(new TestAscendingValueSource(numElements));
+
+			// Value state
+			ValueStateDescriptor<Tuple2<Integer, Long>> valueState = new ValueStateDescriptor<>(
+					"any",
+					source.getType());
+
+			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);
+
+			executeValueQuery(deadline, client, jobId, "hakuna", valueState, numElements);
+		} finally {
+			// Free cluster resources
+			if (jobId != null) {
+				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
+						.getLeaderGateway(deadline.timeLeft())
+						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
+						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
+
+				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+			}
+		}
+	}
+
+	/**
+	 * 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 long numElements = 1024L;
+
+		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, 1000L));
+
+			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);
+
+			executeValueQuery(deadline, client, jobId, "hakuna", valueState, expected);
+		} finally {
+			// Free cluster resources
+			if (jobId != null) {
+				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
+						.getLeaderGateway(deadline.timeLeft())
+						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
+						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
+
+				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+			}
+		}
+	}
+
+	/**
+	 * 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 UnknownKeyOrNamespaceException} exception.
+	 *
+	 * @throws UnknownKeyOrNamespaceException thrown due querying a non-existent key
+	 */
+	@Test(expected = UnknownKeyOrNamespaceException.class)
+	public void testValueStateDefault() throws Throwable {
+
+		// Config
+		final Deadline deadline = TEST_TIMEOUT.fromNow();
+
+		final long numElements = 1024L;
+
+		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, 1000L));
+
+			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;
+			CompletableFuture<ValueState<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) {
+				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
+						.getLeaderGateway(deadline.timeLeft())
+						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
+						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
+
+				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+			}
+		}
+	}
+
+	/**
+	 * 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 long numElements = 1024L;
+
+		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, 1000L));
+
+			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);
+
+			final ValueStateDescriptor<Tuple2<Integer, Long>> stateDesc =
+					(ValueStateDescriptor<Tuple2<Integer, Long>>) queryableState.getStateDescriptor();
+			executeValueQuery(deadline, client, jobId, "matata", stateDesc, numElements);
+		} finally {
+
+			// Free cluster resources
+			if (jobId != null) {
+				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(
+						cluster.getLeaderGateway(deadline.timeLeft())
+								.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
+								.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
+
+				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+			}
+		}
+	}
+
+	/**
+	 * 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;
+
+		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, 1000L));
+
+			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) {
+					CompletableFuture<FoldingState<Tuple2<Integer, Long>, String>> future = getKvStateWithRetries(
+							client,
+							jobId,
+							"pumba",
+							key,
+							BasicTypeInfo.INT_TYPE_INFO,
+							foldingState,
+							QUERY_RETRY_DELAY,
+							false,
+							executor);
+
+					String value = future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS).get();
+
+					//assertEquals("Key mismatch", key, value.f0.intValue());
+					if (expected.equals(value)) {
+						success = true;
+					} else {
+						// Retry
+						Thread.sleep(50L);
+					}
+				}
+
+				assertTrue("Did not succeed query", success);
+			}
+		} finally {
+			// Free cluster resources
+			if (jobId != null) {
+				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
+						.getLeaderGateway(deadline.timeLeft())
+						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
+						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
+
+				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+			}
+		}
+	}
+
+	/**
+	 * 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 long numElements = 1024L;
+
+		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, 1000L));
+
+			DataStream<Tuple2<Integer, Long>> source = env
+					.addSource(new TestAscendingValueSource(numElements));
+
+			// Reducing state
+			ReducingStateDescriptor<Tuple2<Integer, Long>> reducingState =
+					new ReducingStateDescriptor<>(
+							"any",
+							new SumReduce(),
+							source.getType());
+
+			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);
+
+			// Now query
+			long expected = numElements * (numElements + 1L) / 2L;
+
+			for (int key = 0; key < maxParallelism; key++) {
+				boolean success = false;
+				while (deadline.hasTimeLeft() && !success) {
+					CompletableFuture<ReducingState<Tuple2<Integer, Long>>> future = getKvStateWithRetries(
+							client,
+							jobId,
+							"jungle",
+							key,
+							BasicTypeInfo.INT_TYPE_INFO,
+							reducingState,
+							QUERY_RETRY_DELAY,
+							false,
+							executor);
+
+					Tuple2<Integer, Long> value = future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS).get();
+
+					assertEquals("Key mismatch", key, value.f0.intValue());
+					if (expected == value.f1) {
+						success = true;
+					} else {
+						// Retry
+						Thread.sleep(50L);
+					}
+				}
+
+				assertTrue("Did not succeed query", success);
+			}
+		} finally {
+			// Free cluster resources
+			if (jobId != null) {
+				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
+						.getLeaderGateway(deadline.timeLeft())
+						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
+						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
+
+				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+			}
+		}
+	}
+
+	/**
+	 * Tests simple map state queryable state instance. Each source emits
+	 * (subtaskIndex, 0)..(subtaskIndex, numElements) tuples, which are then
+	 * queried. The map state instance sums the values up. The test succeeds
+	 * after each subtask index is queried with result n*(n+1)/2.
+	 */
+	@Test
+	public void testMapState() throws Exception {
+		// Config
+		final Deadline deadline = TEST_TIMEOUT.fromNow();
+
+		final long numElements = 1024L;
+
+		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, 1000L));
+
+			DataStream<Tuple2<Integer, Long>> source = env
+					.addSource(new TestAscendingValueSource(numElements));
+
+			final MapStateDescriptor<Integer, Tuple2<Integer, Long>> mapStateDescriptor = new MapStateDescriptor<>(
+					"timon",
+					BasicTypeInfo.INT_TYPE_INFO,
+					source.getType());
+			mapStateDescriptor.setQueryable("timon-queryable");
+
+			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;
+				}
+			}).process(new ProcessFunction<Tuple2<Integer, Long>, Object>() {
+				private static final long serialVersionUID = -805125545438296619L;
+
+				private transient MapState<Integer, Tuple2<Integer, Long>> mapState;
+
+				@Override
+				public void open(Configuration parameters) throws Exception {
+					super.open(parameters);
+					mapState = getRuntimeContext().getMapState(mapStateDescriptor);
+				}
+
+				@Override
+				public void processElement(Tuple2<Integer, Long> value, Context ctx, Collector<Object> out) throws Exception {
+					Tuple2<Integer, Long> v = mapState.get(value.f0);
+					if (v == null) {
+						v = new Tuple2<>(value.f0, 0L);
+					}
+					mapState.put(value.f0, new Tuple2<>(v.f0, v.f1 + value.f1));
+				}
+			});
+
+			// Submit the job graph
+			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
+			jobId = jobGraph.getJobID();
+
+			cluster.submitJobDetached(jobGraph);
+
+			// Now query
+			long expected = numElements * (numElements + 1L) / 2L;
+
+			for (int key = 0; key < maxParallelism; key++) {
+				boolean success = false;
+				while (deadline.hasTimeLeft() && !success) {
+					CompletableFuture<MapState<Integer, Tuple2<Integer, Long>>> future = getKvStateWithRetries(
+							client,
+							jobId,
+							"timon-queryable",
+							key,
+							BasicTypeInfo.INT_TYPE_INFO,
+							mapStateDescriptor,
+							QUERY_RETRY_DELAY,
+							false,
+							executor);
+
+					Tuple2<Integer, Long> value = future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS).get(key);
+					assertEquals("Key mismatch", key, value.f0.intValue());
+					if (expected == value.f1) {
+						success = true;
+					} else {
+						// Retry
+						Thread.sleep(50L);
+					}
+				}
+
+				assertTrue("Did not succeed query", success);
+			}
+		} finally {
+			// Free cluster resources
+			if (jobId != null) {
+				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
+						.getLeaderGateway(deadline.timeLeft())
+						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
+						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
+
+				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+			}
+		}
+	}
+
+	/**
+	 * Tests simple list state queryable state instance. Each source emits
+	 * (subtaskIndex, 0)..(subtaskIndex, numElements) tuples, which are then
+	 * queried. The list state instance add the values to the list. The test
+	 * succeeds after each subtask index is queried and the list contains
+	 * the correct number of distinct elements.
+	 */
+	@Test
+	public void testListState() throws Exception {
+		// Config
+		final Deadline deadline = TEST_TIMEOUT.fromNow();
+
+		final long numElements = 1024L;
+
+		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, 1000L));
+
+			DataStream<Tuple2<Integer, Long>> source = env
+					.addSource(new TestAscendingValueSource(numElements));
+
+			final ListStateDescriptor<Long> listStateDescriptor = new ListStateDescriptor<Long>(
+					"list",
+					BasicTypeInfo.LONG_TYPE_INFO);
+			listStateDescriptor.setQueryable("list-queryable");
+
+			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;
+				}
+			}).process(new ProcessFunction<Tuple2<Integer, Long>, Object>() {
+				private static final long serialVersionUID = -805125545438296619L;
+
+				private transient ListState<Long> listState;
+
+				@Override
+				public void open(Configuration parameters) throws Exception {
+					super.open(parameters);
+					listState = getRuntimeContext().getListState(listStateDescriptor);
+				}
+
+				@Override
+				public void processElement(Tuple2<Integer, Long> value, Context ctx, Collector<Object> out) throws Exception {
+					listState.add(value.f1);
+				}
+			});
+
+			// Submit the job graph
+			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
+			jobId = jobGraph.getJobID();
+
+			cluster.submitJobDetached(jobGraph);
+
+			// Now query
+
+			Map<Integer, Set<Long>> results = new HashMap<>();
+
+			for (int key = 0; key < maxParallelism; key++) {
+				boolean success = false;
+				while (deadline.hasTimeLeft() && !success) {
+					CompletableFuture<ListState<Long>> future = getKvStateWithRetries(
+							client,
+							jobId,
+							"list-queryable",
+							key,
+							BasicTypeInfo.INT_TYPE_INFO,
+							listStateDescriptor,
+							QUERY_RETRY_DELAY,
+							false,
+							executor);
+
+					Iterable<Long> value = future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS).get();
+
+					Set<Long> res = new HashSet<>();
+					for (Long v: value) {
+						res.add(v);
+					}
+
+					// the source starts at 0, so +1
+					if (res.size() == numElements + 1L) {
+						success = true;
+						results.put(key, res);
+					} else {
+						// Retry
+						Thread.sleep(50L);
+					}
+				}
+
+				assertTrue("Did not succeed query", success);
+			}
+
+			for (int key = 0; key < maxParallelism; key++) {
+				Set<Long> values = results.get(key);
+				for (long i = 0L; i <= numElements; i++) {
+					assertTrue(values.contains(i));
+				}
+			}
+
+		} finally {
+			// Free cluster resources
+			if (jobId != null) {
+				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
+						.getLeaderGateway(deadline.timeLeft())
+						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
+						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
+
+				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+			}
+		}
+	}
+
+	@Test
+	public void testAggregatingState() throws Exception {
+		// Config
+		final Deadline deadline = TEST_TIMEOUT.fromNow();
+
+		final long numElements = 1024L;
+
+		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, 1000L));
+
+			DataStream<Tuple2<Integer, Long>> source = env
+					.addSource(new TestAscendingValueSource(numElements));
+
+			final AggregatingStateDescriptor<Tuple2<Integer, Long>, String, String> aggrStateDescriptor =
+					new AggregatingStateDescriptor<>(
+							"aggregates",
+							new SumAggr(),
+							String.class);
+			aggrStateDescriptor.setQueryable("aggr-queryable");
+
+			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;
+				}
+			}).transform(
+					"TestAggregatingOperator",
+					BasicTypeInfo.STRING_TYPE_INFO,
+					new AggregatingTestOperator(aggrStateDescriptor)
+			);
+
+			// Submit the job graph
+			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
+			jobId = jobGraph.getJobID();
+
+			cluster.submitJobDetached(jobGraph);
+
+			// Now query
+
+			for (int key = 0; key < maxParallelism; key++) {
+				boolean success = false;
+				while (deadline.hasTimeLeft() && !success) {
+					CompletableFuture<AggregatingState<Tuple2<Integer, Long>, String>> future = getKvStateWithRetries(
+							client,
+							jobId,
+							"aggr-queryable",
+							key,
+							BasicTypeInfo.INT_TYPE_INFO,
+							aggrStateDescriptor,
+							QUERY_RETRY_DELAY,
+							false,
+							executor);
+
+					String value = future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS).get();
+
+					if (Long.parseLong(value) == numElements * (numElements + 1L) / 2L) {
+						success = true;
+					} else {
+						// Retry
+						Thread.sleep(50L);
+					}
+				}
+
+				assertTrue("Did not succeed query", success);
+			}
+		} finally {
+			// Free cluster resources
+			if (jobId != null) {
+				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
+						.getLeaderGateway(deadline.timeLeft())
+						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
+						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
+
+				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+			}
+		}
+	}
+
+	/////				Sources/UDFs Used in the Tests			//////
+
+	/**
+	 * 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) {
+					wait();
+				}
+			}
+		}
+
+		@Override
+		public void cancel() {
+			isRunning = false;
+
+			synchronized (this) {
+				notifyAll();
+			}
+		}
+
+	}
+
+	/**
+	 * Test source producing (key, 1) tuples with random key in key range (numKeys).
+	 */
+	private 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(0L);
+			}
+		}
+
+		@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(1L);
+			}
+		}
+
+		@Override
+		public void cancel() {
+			isRunning = false;
+		}
+
+		@Override
+		public void notifyCheckpointComplete(long checkpointId) throws Exception {
+			if (getRuntimeContext().getIndexOfThisSubtask() == 0) {
+				LATEST_CHECKPOINT_ID.set(checkpointId);
+			}
+		}
+	}
+
+	/**
+	 * An operator that uses {@link AggregatingState}.
+	 *
+	 * <p>The operator exists for lack of possibility to get an
+	 * {@link AggregatingState} from the {@link org.apache.flink.api.common.functions.RuntimeContext}.
+	 * If this were not the case, we could have a {@link ProcessFunction}.
+	 */
+	private static class AggregatingTestOperator
+			extends AbstractStreamOperator<String>
+			implements OneInputStreamOperator<Tuple2<Integer, Long>, String> {
+
+		private static final long serialVersionUID = 1L;
+
+		private final AggregatingStateDescriptor<Tuple2<Integer, Long>, String, String> stateDescriptor;
+		private transient AggregatingState<Tuple2<Integer, Long>, String> state;
+
+		AggregatingTestOperator(AggregatingStateDescriptor<Tuple2<Integer, Long>, String, String> stateDesc) {
+			this.stateDescriptor = stateDesc;
+		}
+
+		@Override
+		public void open() throws Exception {
+			super.open();
+			this.state = getKeyedStateBackend().getPartitionedState(
+					VoidNamespace.INSTANCE,
+					VoidNamespaceSerializer.INSTANCE,
+					stateDescriptor);
+		}
+
+		@Override
+		public void processElement(StreamRecord<Tuple2<Integer, Long>> element) throws Exception {
+			state.add(element.getValue());
+		}
+	}
+
+	/**
+	 * Test {@link AggregateFunction} concatenating the already stored string with the long passed as argument.
+	 */
+	private static class SumAggr implements AggregateFunction<Tuple2<Integer, Long>, String, String> {
+
+		private static final long serialVersionUID = -6249227626701264599L;
+
+		@Override
+		public String createAccumulator() {
+			return "0";
+		}
+
+		@Override
+		public String add(Tuple2<Integer, Long> value, String accumulator) {
+			long acc = Long.valueOf(accumulator);
+			acc += value.f1;
+			return Long.toString(acc);
+		}
+
+		@Override
+		public String getResult(String accumulator) {
+			return accumulator;
+		}
+
+		@Override
+		public String merge(String a, String b) {
+			return Long.toString(Long.valueOf(a) + Long.valueOf(b));
+		}
+	}
+
+	/**
+	 * 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;
+		}
+	}
+
+	/////				General Utility Methods				//////
+
+	private static <K, S extends State, V> CompletableFuture<S> getKvStateWithRetries(
+			final QueryableStateClient client,
+			final JobID jobId,
+			final String queryName,
+			final K key,
+			final TypeInformation<K> keyTypeInfo,
+			final StateDescriptor<S, 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));
+		}
+	}
+
+	/**
+	 * 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 executeValueQuery(
+			final Deadline deadline,
+			final QueryableStateClient client,
+			final JobID jobId,
+			final String queryableStateName,
+			final ValueStateDescriptor<Tuple2<Integer, Long>> stateDescriptor,
+			final long expected) throws Exception {
+
+		for (int key = 0; key < maxParallelism; key++) {
+			boolean success = false;
+			while (deadline.hasTimeLeft() && !success) {
+				CompletableFuture<ValueState<Tuple2<Integer, Long>>> future = getKvStateWithRetries(
+						client,
+						jobId,
+						queryableStateName,
+						key,
+						BasicTypeInfo.INT_TYPE_INFO,
+						stateDescriptor,
+						QUERY_RETRY_DELAY,
+						false,
+						executor);
+
+				Tuple2<Integer, Long> value = future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS).value();
+
+				assertEquals("Key mismatch", key, value.f0.intValue());
+				if (expected == value.f1) {
+					success = true;
+				} else {
+					// Retry
+					Thread.sleep(50L);
+				}
+			}
+
+			assertTrue("Did not succeed query", success);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java
new file mode 100644
index 0000000..ab75cf4
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.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.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.queryablestate.client.QueryableStateClient;
+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.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 HAAbstractQueryableStateTestBase extends AbstractQueryableStateTestBase {
+
+	private static final int NUM_JMS = 2;
+	private static final int NUM_TMS = 2;
+	private static final int NUM_SLOTS_PER_TM = 4;
+
+	private static TestingServer zkServer;
+	private static TemporaryFolder temporaryFolder;
+
+	public static void setup(int proxyPortRangeStart, int serverPortRangeStart) {
+		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(QueryableStateOptions.PROXY_PORT_RANGE, proxyPortRangeStart + "-" + (proxyPortRangeStart + NUM_TMS));
+			config.setString(QueryableStateOptions.SERVER_PORT_RANGE, serverPortRangeStart + "-" + (serverPortRangeStart + NUM_TMS));
+			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();
+
+			client = new QueryableStateClient("localhost", proxyPortRangeStart);
+
+			// 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();
+		}
+
+		try {
+			zkServer.stop();
+			zkServer.close();
+		} catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+
+		client.shutdown();
+	}
+}

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/KVStateRequestSerializerRocksDBTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/KVStateRequestSerializerRocksDBTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/KVStateRequestSerializerRocksDBTest.java
new file mode 100644
index 0000000..cb6fb3d
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-runtime/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.queryablestate.client.VoidNamespace;
+import org.apache.flink.queryablestate.client.VoidNamespaceSerializer;
+import org.apache.flink.queryablestate.network.KvStateRequestSerializerTest;
+import org.apache.flink.runtime.query.TaskKvStateRegistry;
+import org.apache.flink.runtime.state.KeyGroupRange;
+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/0c771505/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java
new file mode 100644
index 0000000..2937a51
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.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.queryablestate.client.QueryableStateClient;
+import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
+import org.apache.flink.runtime.testingUtils.TestingCluster;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+
+import static org.junit.Assert.fail;
+
+/**
+ * Base class with the cluster configuration for the tests on the HA mode.
+ */
+public abstract class NonHAAbstractQueryableStateTestBase extends AbstractQueryableStateTestBase {
+
+	private static final int NUM_TMS = 2;
+	private static final int NUM_SLOTS_PER_TM = 4;
+
+	public static void setup(int proxyPortRangeStart, int serverPortRangeStart) {
+		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);
+			config.setString(QueryableStateOptions.PROXY_PORT_RANGE, proxyPortRangeStart + "-" + (proxyPortRangeStart + NUM_TMS));
+			config.setString(QueryableStateOptions.SERVER_PORT_RANGE, serverPortRangeStart + "-" + (serverPortRangeStart + NUM_TMS));
+
+			cluster = new TestingCluster(config, false);
+			cluster.start(true);
+
+			client = new QueryableStateClient("localhost", proxyPortRangeStart);
+
+			// 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());
+		}
+		client.shutdown();
+	}
+}

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java
new file mode 100644
index 0000000..0b2727c
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java
@@ -0,0 +1,219 @@
+/*
+ * 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.configuration.ConfigConstants;
+import org.apache.flink.queryablestate.network.messages.MessageBody;
+import org.apache.flink.queryablestate.network.messages.MessageDeserializer;
+import org.apache.flink.queryablestate.network.messages.MessageSerializer;
+import org.apache.flink.queryablestate.network.stats.DisabledKvStateRequestStats;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Tests general behavior of the {@link AbstractServerBase}.
+ */
+public class AbstractServerTest {
+
+	@Rule
+	public ExpectedException expectedEx = ExpectedException.none();
+
+	/**
+	 * Tests that in case of port collision, a FlinkRuntimeException is thrown
+	 * with a specific message.
+	 */
+	@Test
+	public void testServerInitializationFailure() throws Throwable {
+
+		// the expected exception along with the adequate message
+		expectedEx.expect(FlinkRuntimeException.class);
+		expectedEx.expectMessage("Unable to start server Test Server 2. All ports in provided range are occupied.");
+
+		TestServer server1 = null;
+		TestServer server2 = null;
+		try {
+
+			server1 = startServer("Test Server 1", 7777);
+			Assert.assertEquals(7777L, server1.getServerAddress().getPort());
+
+			server2 = startServer("Test Server 2", 7777);
+		} finally {
+
+			if (server1 != null) {
+				server1.shutdown();
+			}
+
+			if (server2 != null) {
+				server2.shutdown();
+			}
+		}
+	}
+
+	/**
+	 * Tests that in case of port collision and big enough port range,
+	 * the server will try to bind to the next port in the range.
+	 */
+	@Test
+	public void testPortRangeSuccess() throws Throwable {
+		TestServer server1 = null;
+		TestServer server2 = null;
+		Client<TestMessage, TestMessage> client = null;
+
+		try {
+			server1 = startServer("Test Server 1", 7777, 7778, 7779);
+			Assert.assertEquals(7777L, server1.getServerAddress().getPort());
+
+			server2 = startServer("Test Server 2", 7777, 7778, 7779);
+			Assert.assertEquals(7778L, server2.getServerAddress().getPort());
+
+			client = new Client<>(
+					"Test Client",
+					1,
+					new MessageSerializer<>(new TestMessage.TestMessageDeserializer(), new TestMessage.TestMessageDeserializer()),
+					new DisabledKvStateRequestStats());
+
+			TestMessage response1 = client.sendRequest(server1.getServerAddress(), new TestMessage("ping")).join();
+			Assert.assertEquals(server1.getServerName() + "-ping", response1.getMessage());
+
+			TestMessage response2 = client.sendRequest(server2.getServerAddress(), new TestMessage("pong")).join();
+			Assert.assertEquals(server2.getServerName() + "-pong", response2.getMessage());
+		} finally {
+
+			if (server1 != null) {
+				server1.shutdown();
+			}
+
+			if (server2 != null) {
+				server2.shutdown();
+			}
+
+			if (client != null) {
+				client.shutdown();
+			}
+		}
+	}
+
+	/**
+	 * Initializes a {@link TestServer} with the given port range.
+	 * @param serverName the name of the server.
+	 * @param ports a range of ports.
+	 * @return A test server with the given name.
+	 */
+	private TestServer startServer(String serverName, int... ports) throws Throwable {
+		List<Integer> portList = new ArrayList<>(ports.length);
+		for (int p : ports) {
+			portList.add(p);
+		}
+
+		final TestServer server = new TestServer(serverName, portList.iterator());
+		server.start();
+		return server;
+	}
+
+	/**
+	 * A server that receives a {@link TestMessage test message} and returns another test
+	 * message containing the same string as the request with the name of the server prepended.
+	 */
+	private class TestServer extends AbstractServerBase<TestMessage, TestMessage> {
+
+		protected TestServer(String name, Iterator<Integer> bindPort) throws UnknownHostException {
+			super(name, InetAddress.getLocalHost(), bindPort, 1, 1);
+		}
+
+		@Override
+		public AbstractServerHandler<TestMessage, TestMessage> initializeHandler() {
+			return new AbstractServerHandler<TestMessage, TestMessage>(
+					this,
+					new MessageSerializer<>(new TestMessage.TestMessageDeserializer(), new TestMessage.TestMessageDeserializer()),
+					new DisabledKvStateRequestStats()) {
+
+				@Override
+				public CompletableFuture<TestMessage> handleRequest(long requestId, TestMessage request) {
+					TestMessage response = new TestMessage(getServerName() + '-' + request.getMessage());
+					return CompletableFuture.completedFuture(response);
+				}
+
+				@Override
+				public void shutdown() {
+					// do nothing
+				}
+			};
+		}
+	}
+
+	/**
+	 * Message with a string as payload.
+	 */
+	private static class TestMessage extends MessageBody {
+
+		private final String message;
+
+		TestMessage(String message) {
+			this.message = Preconditions.checkNotNull(message);
+		}
+
+		public String getMessage() {
+			return message;
+		}
+
+		@Override
+		public byte[] serialize() {
+			byte[] content = message.getBytes(ConfigConstants.DEFAULT_CHARSET);
+
+			// message size + 4 for the length itself
+			return ByteBuffer.allocate(content.length + Integer.BYTES)
+					.putInt(content.length)
+					.put(content)
+					.array();
+		}
+
+		/**
+		 * The deserializer for our {@link TestMessage test messages}.
+		 */
+		public static class TestMessageDeserializer implements MessageDeserializer<TestMessage> {
+
+			@Override
+			public TestMessage deserializeMessage(ByteBuf buf) {
+				int length = buf.readInt();
+				String message = "";
+				if (length > 0) {
+					byte[] name = new byte[length];
+					buf.readBytes(name);
+					message = new String(name, ConfigConstants.DEFAULT_CHARSET);
+				}
+				return new TestMessage(message);
+			}
+		}
+	}
+}


[02/13] flink git commit: [FLINK-7908][QS] Restructure the queryable state module.

Posted by kk...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/MessageSerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/MessageSerializerTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/MessageSerializerTest.java
new file mode 100644
index 0000000..acaa067
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-runtime/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.KvStateID;
+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.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;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-runtime/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/resources/log4j-test.properties b/flink-queryable-state/flink-queryable-state-runtime/src/test/resources/log4j-test.properties
new file mode 100644
index 0000000..10792cd
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-runtime/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/0c771505/flink-queryable-state/pom.xml
----------------------------------------------------------------------
diff --git a/flink-queryable-state/pom.xml b/flink-queryable-state/pom.xml
index e2579f6..9300fb3 100644
--- a/flink-queryable-state/pom.xml
+++ b/flink-queryable-state/pom.xml
@@ -35,8 +35,9 @@ under the License.
 	<packaging>pom</packaging>
 
 	<modules>
-		<module>flink-queryable-state-java</module>
-	   <!-- <module>flink-state-client-scala</module>-->
+		<module>flink-queryable-state-runtime</module>
+		<module>flink-queryable-state-client-java</module>
+		<!-- <module>flink-state-client-scala</module>-->
 	</modules>
 
 	<dependencies>

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml
index 83ac781..134c414 100644
--- a/flink-runtime/pom.xml
+++ b/flink-runtime/pom.xml
@@ -50,6 +50,12 @@ under the License.
 			<version>${project.version}</version>
 		</dependency>
 
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-queryable-state-client-java_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
 		<!-- The Hadoop FS support has only an optional dependency on Hadoop and
 			gracefully handles absence of Hadoop classes -->
 

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
index f60f561..4535290 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
@@ -28,6 +28,7 @@ import org.apache.flink.core.io.InputSplit;
 import org.apache.flink.core.io.InputSplitAssigner;
 import org.apache.flink.metrics.MetricGroup;
 import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
+import org.apache.flink.queryablestate.KvStateID;
 import org.apache.flink.runtime.StoppingException;
 import org.apache.flink.runtime.blob.BlobServer;
 import org.apache.flink.runtime.checkpoint.CheckpointCoordinator;
@@ -74,10 +75,8 @@ import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
 import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint;
 import org.apache.flink.runtime.messages.webmonitor.JobDetails;
 import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup;
-import org.apache.flink.runtime.query.KvStateID;
 import org.apache.flink.runtime.query.KvStateLocation;
 import org.apache.flink.runtime.query.KvStateLocationRegistry;
-import org.apache.flink.runtime.query.KvStateServerAddress;
 import org.apache.flink.runtime.query.UnknownKvStateLocation;
 import org.apache.flink.runtime.registration.RegisteredRpcConnection;
 import org.apache.flink.runtime.registration.RegistrationResponse;
@@ -104,6 +103,7 @@ import org.slf4j.Logger;
 import javax.annotation.Nullable;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -586,7 +586,7 @@ public class JobMaster extends FencedRpcEndpoint<JobMasterId> implements JobMast
 			final KeyGroupRange keyGroupRange,
 			final String registrationName,
 			final KvStateID kvStateId,
-			final KvStateServerAddress kvStateServerAddress)
+			final InetSocketAddress kvStateServerAddress)
 	{
 		if (log.isDebugEnabled()) {
 			log.debug("Key value state registered for job {} under name {}.",

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
index d59feed..2c7e438 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.jobmaster;
 
 import org.apache.flink.api.common.time.Time;
+import org.apache.flink.queryablestate.KvStateID;
 import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorGateway;
 import org.apache.flink.runtime.clusterframework.types.AllocationID;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
@@ -36,9 +37,7 @@ import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobmaster.message.ClassloadingProps;
 import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.messages.webmonitor.JobDetails;
-import org.apache.flink.runtime.query.KvStateID;
 import org.apache.flink.runtime.query.KvStateLocation;
-import org.apache.flink.runtime.query.KvStateServerAddress;
 import org.apache.flink.runtime.registration.RegistrationResponse;
 import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
 import org.apache.flink.runtime.rpc.FencedRpcGateway;
@@ -49,6 +48,7 @@ import org.apache.flink.runtime.taskexecutor.slot.SlotOffer;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 
+import java.net.InetSocketAddress;
 import java.util.Collection;
 import java.util.concurrent.CompletableFuture;
 
@@ -165,7 +165,7 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway, FencedRp
 			final KeyGroupRange keyGroupRange,
 			final String registrationName,
 			final KvStateID kvStateId,
-			final KvStateServerAddress kvStateServerAddress);
+			final InetSocketAddress kvStateServerAddress);
 
 	/**
 	 * Notifies that queryable state has been unregistered.

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateID.java
deleted file mode 100644
index c122508..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateID.java
+++ /dev/null
@@ -1,41 +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.runtime.state.internal.InternalKvState;
-import org.apache.flink.util.AbstractID;
-
-/**
- * Identifier for {@link InternalKvState} instances.
- *
- * <p>Assigned when registering state at the {@link KvStateRegistry}.
- */
-public class KvStateID extends AbstractID {
-
-	private static final long serialVersionUID = 1L;
-
-	public KvStateID() {
-		super();
-	}
-
-	public KvStateID(long lowerPart, long upperPart) {
-		super(lowerPart, upperPart);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/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 03e8238..e4fdda5 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
@@ -19,12 +19,14 @@
 package org.apache.flink.runtime.query;
 
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.queryablestate.KvStateID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.util.Preconditions;
 
 import java.io.Serializable;
+import java.net.InetSocketAddress;
 import java.util.Arrays;
 
 /**
@@ -56,7 +58,7 @@ public class KvStateLocation implements Serializable {
 	 * Server address for each KvState instance where array index corresponds to
 	 * key group index.
 	 */
-	private final KvStateServerAddress[] kvStateAddresses;
+	private final InetSocketAddress[] kvStateAddresses;
 
 	/** Current number of registered key groups. */
 	private int numRegisteredKeyGroups;
@@ -76,7 +78,7 @@ public class KvStateLocation implements Serializable {
 		this.numKeyGroups = numKeyGroups;
 		this.registrationName = Preconditions.checkNotNull(registrationName, "Registration name");
 		this.kvStateIds = new KvStateID[numKeyGroups];
-		this.kvStateAddresses = new KvStateServerAddress[numKeyGroups];
+		this.kvStateAddresses = new InetSocketAddress[numKeyGroups];
 	}
 
 	/**
@@ -142,15 +144,15 @@ public class KvStateLocation implements Serializable {
 	}
 
 	/**
-	 * Returns the registered KvStateServerAddress for the key group index or
+	 * Returns the registered server address for the key group index or
 	 * <code>null</code> if none is registered yet.
 	 *
 	 * @param keyGroupIndex Key group index to get server address for.
-	 * @return KvStateServerAddress for the key group index or <code>null</code>
+	 * @return the server address for the key group index or <code>null</code>
 	 * if none is registered yet
 	 * @throws IndexOutOfBoundsException If key group index < 0 or >= Number of key groups
 	 */
-	public KvStateServerAddress getKvStateServerAddress(int keyGroupIndex) {
+	public InetSocketAddress getKvStateServerAddress(int keyGroupIndex) {
 		if (keyGroupIndex < 0 || keyGroupIndex >= numKeyGroups) {
 			throw new IndexOutOfBoundsException("Key group index");
 		}
@@ -166,7 +168,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
 	 */
-	public void registerKvState(KeyGroupRange keyGroupRange, KvStateID kvStateId, KvStateServerAddress kvStateAddress) {
+	public void registerKvState(KeyGroupRange keyGroupRange, KvStateID kvStateId, InetSocketAddress kvStateAddress) {
 
 		if (keyGroupRange.getStartKeyGroup() < 0 || keyGroupRange.getEndKeyGroup() >= numKeyGroups) {
 			throw new IndexOutOfBoundsException("Key group index");

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateLocationRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateLocationRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateLocationRegistry.java
index cb61905..05ee017 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateLocationRegistry.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateLocationRegistry.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.query;
 
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.queryablestate.KvStateID;
 import org.apache.flink.runtime.execution.SuppressRestartsException;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
@@ -26,6 +27,7 @@ import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.util.Preconditions;
 
+import java.net.InetSocketAddress;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -89,7 +91,7 @@ public class KvStateLocationRegistry {
 			KeyGroupRange keyGroupRange,
 			String registrationName,
 			KvStateID kvStateId,
-			KvStateServerAddress kvStateServerAddress) {
+			InetSocketAddress kvStateServerAddress) {
 
 		KvStateLocation location = lookupTable.get(registrationName);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateMessage.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateMessage.java
index 04684ee..e94d2f8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateMessage.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateMessage.java
@@ -19,12 +19,14 @@
 package org.apache.flink.runtime.query;
 
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.queryablestate.KvStateID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.util.Preconditions;
 
 import java.io.Serializable;
+import java.net.InetSocketAddress;
 
 /**
  * Actor messages for {@link InternalKvState} lookup and registration.
@@ -114,7 +116,7 @@ public interface KvStateMessage extends Serializable {
 		private final KvStateID kvStateId;
 
 		/** Server address where to find the KvState instance. */
-		private final KvStateServerAddress kvStateServerAddress;
+		private final InetSocketAddress kvStateServerAddress;
 
 		/**
 		 * Notifies the JobManager about a registered {@link InternalKvState} instance.
@@ -132,7 +134,7 @@ public interface KvStateMessage extends Serializable {
 				KeyGroupRange keyGroupRange,
 				String registrationName,
 				KvStateID kvStateId,
-				KvStateServerAddress kvStateServerAddress) {
+				InetSocketAddress kvStateServerAddress) {
 
 			this.jobId = Preconditions.checkNotNull(jobId, "JobID");
 			this.jobVertexId = Preconditions.checkNotNull(jobVertexId, "JobVertexID");
@@ -140,7 +142,7 @@ public interface KvStateMessage extends Serializable {
 			this.keyGroupRange = Preconditions.checkNotNull(keyGroupRange);
 			this.registrationName = Preconditions.checkNotNull(registrationName, "Registration name");
 			this.kvStateId = Preconditions.checkNotNull(kvStateId, "KvStateID");
-			this.kvStateServerAddress = Preconditions.checkNotNull(kvStateServerAddress, "KvStateServerAddress");
+			this.kvStateServerAddress = Preconditions.checkNotNull(kvStateServerAddress, "ServerAddress");
 		}
 
 		/**
@@ -193,7 +195,7 @@ public interface KvStateMessage extends Serializable {
 		 *
 		 * @return Server address where to find the KvState instance
 		 */
-		public KvStateServerAddress getKvStateServerAddress() {
+		public InetSocketAddress getKvStateServerAddress() {
 			return kvStateServerAddress;
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/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 90fa5cc..af19d81 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
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.query;
 
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.queryablestate.KvStateID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.internal.InternalKvState;

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistryGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistryGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistryGateway.java
index 13862c9..4b9834a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistryGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistryGateway.java
@@ -19,10 +19,13 @@
 package org.apache.flink.runtime.query;
 
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.queryablestate.KvStateID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.state.KeyGroupRange;
 
+import java.net.InetSocketAddress;
+
 /**
  * A gateway to listen for {@code KvState} registrations.
  */
@@ -42,7 +45,7 @@ public interface KvStateRegistryGateway extends RpcGateway {
 		KeyGroupRange keyGroupRange,
 		String registrationName,
 		KvStateID kvStateId,
-		KvStateServerAddress kvStateServerAddress);
+		InetSocketAddress kvStateServerAddress);
 
 	/**
 	 * Notifies the listener about an unregistered KvState instance.

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistryListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistryListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistryListener.java
index 29bee9a..dc90c96 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistryListener.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistryListener.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.query;
 
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.queryablestate.KvStateID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.state.KeyGroupRange;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/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 17ffe0d..ae58714 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
@@ -18,6 +18,8 @@
 
 package org.apache.flink.runtime.query;
 
+import java.net.InetSocketAddress;
+
 /**
  * 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 {@link KvStateClientProxy
@@ -26,10 +28,10 @@ package org.apache.flink.runtime.query;
 public interface KvStateServer {
 
 	/**
-	 * Returns the {@link KvStateServerAddress address} the server is listening to.
+	 * Returns the {@link InetSocketAddress address} the server is listening to.
 	 * @return Server address.
 	 */
-	KvStateServerAddress getServerAddress();
+	InetSocketAddress getServerAddress();
 
 
 	/** Starts the server. */

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/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
deleted file mode 100644
index 2599855..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateServerAddress.java
+++ /dev/null
@@ -1,95 +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.util.Preconditions;
-
-import java.io.Serializable;
-import java.net.InetAddress;
-
-/**
- * The (host, port)-address of a {@link KvStateServer}.
- */
-public class KvStateServerAddress implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-
-	/** KvStateServer host address. */
-	private final InetAddress hostAddress;
-
-	/** KvStateServer port. */
-	private final int port;
-
-	/**
-	 * Creates a KvStateServerAddress for the given KvStateServer host address
-	 * and port.
-	 *
-	 * @param hostAddress KvStateServer host address
-	 * @param port        KvStateServer port
-	 */
-	public KvStateServerAddress(InetAddress hostAddress, int port) {
-		this.hostAddress = Preconditions.checkNotNull(hostAddress, "Host address");
-		Preconditions.checkArgument(port > 0 && port <= 65535, "Port " + port + " is out of range 1-65535");
-		this.port = port;
-	}
-
-	/**
-	 * Returns the host address of the KvStateServer.
-	 *
-	 * @return KvStateServer host address
-	 */
-	public InetAddress getHost() {
-		return hostAddress;
-	}
-
-	/**
-	 * Returns the port of the KvStateServer.
-	 *
-	 * @return KvStateServer port
-	 */
-	public int getPort() {
-		return port;
-	}
-
-	@Override
-	public boolean equals(Object o) {
-		if (this == o) {
-			return true;
-		}
-		if (o == null || getClass() != o.getClass()) {
-			return false;
-		}
-
-		KvStateServerAddress that = (KvStateServerAddress) o;
-
-		return port == that.port && hostAddress.equals(that.hostAddress);
-	}
-
-	@Override
-	public int hashCode() {
-		int result = hostAddress.hashCode();
-		result = 31 * result + port;
-		return result;
-	}
-
-	@Override
-	public String toString() {
-		return hostAddress.getHostName() + ':' + port;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/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 fa021df..adbe15d 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
@@ -18,7 +18,7 @@
 
 package org.apache.flink.runtime.query;
 
-import org.apache.flink.runtime.query.netty.KvStateRequestStats;
+import org.apache.flink.queryablestate.network.stats.KvStateRequestStats;
 import org.apache.flink.util.Preconditions;
 
 import org.slf4j.Logger;
@@ -74,7 +74,8 @@ public final class QueryableStateUtils {
 			return constructor.newInstance(address, ports, 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");
+					"Probable reason: flink-queryable-state-runtime is not in the classpath. " +
+					"Please put the corresponding jar from the opt to the lib folder.");
 			LOG.debug("Caught exception", e);
 			return null;
 		} catch (InvocationTargetException e) {
@@ -128,7 +129,8 @@ public final class QueryableStateUtils {
 			return constructor.newInstance(address, ports, eventLoopThreads, queryThreads, kvStateRegistry, stats);
 		} catch (ClassNotFoundException e) {
 			LOG.warn("Could not load Queryable State Server. " +
-					"Probable reason: flink-queryable-state is not in the classpath");
+					"Probable reason: flink-queryable-state-runtime is not in the classpath. " +
+					"Please put the corresponding jar from the opt to the lib folder.");
 			LOG.debug("Caught exception", e);
 			return null;
 		} catch (InvocationTargetException e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-runtime/src/main/java/org/apache/flink/runtime/query/TaskKvStateRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/TaskKvStateRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/TaskKvStateRegistry.java
index 8d0eede..f799b5a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/TaskKvStateRegistry.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/TaskKvStateRegistry.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.query;
 
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.queryablestate.KvStateID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.internal.InternalKvState;

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/AtomicKvStateRequestStats.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/AtomicKvStateRequestStats.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/AtomicKvStateRequestStats.java
deleted file mode 100644
index 1d80bab..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/AtomicKvStateRequestStats.java
+++ /dev/null
@@ -1,104 +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 java.util.concurrent.atomic.AtomicLong;
-
-/**
- * Atomic {@link KvStateRequestStats} implementation.
- */
-public class AtomicKvStateRequestStats implements KvStateRequestStats {
-
-	/**
-	 * Number of active connections.
-	 */
-	private final AtomicLong numConnections = new AtomicLong();
-
-	/**
-	 * Total number of reported requests.
-	 */
-	private final AtomicLong numRequests = new AtomicLong();
-
-	/**
-	 * Total number of successful requests (<= reported requests).
-	 */
-	private final AtomicLong numSuccessful = new AtomicLong();
-
-	/**
-	 * Total duration of all successful requests.
-	 */
-	private final AtomicLong successfulDuration = new AtomicLong();
-
-	/**
-	 * Total number of failed requests (<= reported requests).
-	 */
-	private final AtomicLong numFailed = new AtomicLong();
-
-	@Override
-	public void reportActiveConnection() {
-		numConnections.incrementAndGet();
-	}
-
-	@Override
-	public void reportInactiveConnection() {
-		numConnections.decrementAndGet();
-	}
-
-	@Override
-	public void reportRequest() {
-		numRequests.incrementAndGet();
-	}
-
-	@Override
-	public void reportSuccessfulRequest(long durationTotalMillis) {
-		numSuccessful.incrementAndGet();
-		successfulDuration.addAndGet(durationTotalMillis);
-	}
-
-	@Override
-	public void reportFailedRequest() {
-		numFailed.incrementAndGet();
-	}
-
-	public long getNumConnections() {
-		return numConnections.get();
-	}
-
-	public long getNumRequests() {
-		return numRequests.get();
-	}
-
-	public long getNumSuccessful() {
-		return numSuccessful.get();
-	}
-
-	public long getNumFailed() {
-		return numFailed.get();
-	}
-
-	@Override
-	public String toString() {
-		return "AtomicKvStateRequestStats{" +
-				"numConnections=" + numConnections +
-				", numRequests=" + numRequests +
-				", numSuccessful=" + numSuccessful +
-				", numFailed=" + numFailed +
-				'}';
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/DisabledKvStateRequestStats.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/DisabledKvStateRequestStats.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/DisabledKvStateRequestStats.java
deleted file mode 100644
index de8824d..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/DisabledKvStateRequestStats.java
+++ /dev/null
@@ -1,45 +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;
-
-/**
- * Disabled {@link KvStateRequestStats} implementation.
- */
-public class DisabledKvStateRequestStats implements KvStateRequestStats {
-
-	@Override
-	public void reportActiveConnection() {
-	}
-
-	@Override
-	public void reportInactiveConnection() {
-	}
-
-	@Override
-	public void reportRequest() {
-	}
-
-	@Override
-	public void reportSuccessfulRequest(long durationTotalMillis) {
-	}
-
-	@Override
-	public void reportFailedRequest() {
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/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
deleted file mode 100644
index 19caf92..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateRequestStats.java
+++ /dev/null
@@ -1,55 +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;
-
-/**
- * Simple statistics for
- * {@link org.apache.flink.runtime.query.KvStateServer} and
- * {@link org.apache.flink.runtime.query.KvStateClientProxy} monitoring.
- */
-public interface KvStateRequestStats {
-
-	/**
-	 * Reports an active connection.
-	 */
-	void reportActiveConnection();
-
-	/**
-	 * Reports an inactive connection.
-	 */
-	void reportInactiveConnection();
-
-	/**
-	 * Reports an incoming request.
-	 */
-	void reportRequest();
-
-	/**
-	 * Reports a successfully handled request.
-	 *
-	 * @param durationTotalMillis Duration of the request (in milliseconds).
-	 */
-	void reportSuccessfulRequest(long durationTotalMillis);
-
-	/**
-	 * Reports a failure during a request.
-	 */
-	void reportFailedRequest();
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/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
deleted file mode 100644
index 44ee571..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateSerializer.java
+++ /dev/null
@@ -1,267 +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.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/0c771505/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 97b6bcd..66360e4 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.KvStateSerializer;
+import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer;
 import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.util.Preconditions;
 
@@ -41,7 +41,7 @@ import org.apache.flink.util.Preconditions;
 public abstract class AbstractHeapState<K, N, SV, S extends State, SD extends StateDescriptor<S, ?>>
 		implements InternalKvState<N> {
 
-	/** Map containing the actual key/value pairs */
+	/** Map containing the actual key/value pairs. */
 	protected final StateTable<K, N, SV> stateTable;
 
 	/** This holds the name of the state and can create an initial default value for the state. */
@@ -118,4 +118,4 @@ public abstract class AbstractHeapState<K, N, SV, S extends State, SD extends St
 	public StateTable<K, N, SV> getStateTable() {
 		return stateTable;
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/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 f981b9f..206f10a 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.KvStateSerializer;
+import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer;
 import org.apache.flink.runtime.state.internal.InternalMapState;
 import org.apache.flink.util.Preconditions;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/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 1cc94d2a..2baf644 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
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.taskexecutor;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.core.memory.MemoryType;
+import org.apache.flink.queryablestate.network.stats.DisabledKvStateRequestStats;
 import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
 import org.apache.flink.runtime.clusterframework.types.AllocationID;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
@@ -42,7 +43,6 @@ 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.taskexecutor.slot.TaskSlotTable;
 import org.apache.flink.runtime.taskexecutor.slot.TimerService;
 import org.apache.flink.runtime.taskexecutor.utils.TaskExecutorMetricsInitializer;

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcKvStateRegistryListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcKvStateRegistryListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcKvStateRegistryListener.java
index 3692a71..6312d08 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcKvStateRegistryListener.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcKvStateRegistryListener.java
@@ -19,22 +19,23 @@
 package org.apache.flink.runtime.taskexecutor.rpc;
 
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.queryablestate.KvStateID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.query.KvStateID;
 import org.apache.flink.runtime.query.KvStateRegistryGateway;
 import org.apache.flink.runtime.query.KvStateRegistryListener;
-import org.apache.flink.runtime.query.KvStateServerAddress;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.util.Preconditions;
 
+import java.net.InetSocketAddress;
+
 public class RpcKvStateRegistryListener implements KvStateRegistryListener {
 
 	private final KvStateRegistryGateway kvStateRegistryGateway;
-	private final KvStateServerAddress kvStateServerAddress;
+	private final InetSocketAddress kvStateServerAddress;
 
 	public RpcKvStateRegistryListener(
 			KvStateRegistryGateway kvStateRegistryGateway,
-			KvStateServerAddress kvStateServerAddress) {
+			InetSocketAddress kvStateServerAddress) {
 		this.kvStateRegistryGateway = Preconditions.checkNotNull(kvStateRegistryGateway);
 		this.kvStateServerAddress = Preconditions.checkNotNull(kvStateServerAddress);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayKvStateRegistryListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayKvStateRegistryListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayKvStateRegistryListener.java
index 4404867..63bda99 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayKvStateRegistryListener.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayKvStateRegistryListener.java
@@ -19,15 +19,16 @@
 package org.apache.flink.runtime.taskmanager;
 
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.queryablestate.KvStateID;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.query.KvStateID;
 import org.apache.flink.runtime.query.KvStateMessage;
 import org.apache.flink.runtime.query.KvStateRegistryListener;
-import org.apache.flink.runtime.query.KvStateServerAddress;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.util.Preconditions;
 
+import java.net.InetSocketAddress;
+
 /**
  * This implementation uses {@link ActorGateway} to forward key-value state notifications to the job
  * manager. The notifications are wrapped in an actor message and send to the given actor gateway.
@@ -36,14 +37,14 @@ public class ActorGatewayKvStateRegistryListener implements KvStateRegistryListe
 
 	private ActorGateway jobManager;
 
-	private KvStateServerAddress kvStateServerAddress;
+	private InetSocketAddress kvStateServerAddress;
 
 	public ActorGatewayKvStateRegistryListener(
 		ActorGateway jobManager,
-		KvStateServerAddress kvStateServerAddress) {
+		InetSocketAddress kvStateServerAddress) {
 
 		this.jobManager = Preconditions.checkNotNull(jobManager, "JobManager");
-		this.kvStateServerAddress = Preconditions.checkNotNull(kvStateServerAddress, "KvStateServerAddress");
+		this.kvStateServerAddress = Preconditions.checkNotNull(kvStateServerAddress, "ServerAddress");
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java
index 68da362..889191f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java
@@ -25,6 +25,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.queryablestate.KvStateID;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.akka.ListeningBehaviour;
 import org.apache.flink.runtime.checkpoint.CheckpointDeclineReason;
@@ -74,12 +75,10 @@ import org.apache.flink.runtime.messages.JobManagerMessages.SubmitJob;
 import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepoint;
 import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepointSuccess;
 import org.apache.flink.runtime.messages.RegistrationMessages;
-import org.apache.flink.runtime.query.KvStateID;
 import org.apache.flink.runtime.query.KvStateLocation;
 import org.apache.flink.runtime.query.KvStateMessage.LookupKvStateLocation;
 import org.apache.flink.runtime.query.KvStateMessage.NotifyKvStateRegistered;
 import org.apache.flink.runtime.query.KvStateMessage.NotifyKvStateUnregistered;
-import org.apache.flink.runtime.query.KvStateServerAddress;
 import org.apache.flink.runtime.query.UnknownKvStateLocation;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.taskmanager.TaskManager;
@@ -119,6 +118,7 @@ import org.junit.rules.TemporaryFolder;
 
 import java.io.File;
 import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.util.Collections;
 import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
@@ -717,7 +717,7 @@ public class JobManagerTest extends TestLogger {
 				new KeyGroupRange(0, 0),
 				"any-name",
 				new KvStateID(),
-				new KvStateServerAddress(InetAddress.getLocalHost(), 1233));
+				new InetSocketAddress(InetAddress.getLocalHost(), 1233));
 
 		jobManager.tell(registerNonExistingJob);
 
@@ -742,7 +742,7 @@ public class JobManagerTest extends TestLogger {
 				new KeyGroupRange(0, 0),
 				"register-me",
 				new KvStateID(),
-				new KvStateServerAddress(InetAddress.getLocalHost(), 1293));
+				new InetSocketAddress(InetAddress.getLocalHost(), 1293));
 
 		jobManager.tell(registerForExistingJob);
 
@@ -797,7 +797,7 @@ public class JobManagerTest extends TestLogger {
 				new KeyGroupRange(0, 0),
 				"duplicate-me",
 				new KvStateID(),
-				new KvStateServerAddress(InetAddress.getLocalHost(), 1293));
+				new InetSocketAddress(InetAddress.getLocalHost(), 1293));
 
 		NotifyKvStateRegistered duplicate = new NotifyKvStateRegistered(
 				jobGraph.getJobID(),
@@ -805,7 +805,7 @@ public class JobManagerTest extends TestLogger {
 				new KeyGroupRange(0, 0),
 				"duplicate-me", // ...same name
 				new KvStateID(),
-				new KvStateServerAddress(InetAddress.getLocalHost(), 1293));
+				new InetSocketAddress(InetAddress.getLocalHost(), 1293));
 
 		Future<TestingJobManagerMessages.JobStatusIs> failedFuture = jobManager
 				.ask(new NotifyWhenJobStatus(jobGraph.getJobID(), JobStatus.FAILED), deadline.timeLeft())

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-runtime/src/test/java/org/apache/flink/runtime/query/KvStateLocationRegistryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/query/KvStateLocationRegistryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/query/KvStateLocationRegistryTest.java
index 7bf9ee7..74e16a0 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/query/KvStateLocationRegistryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/query/KvStateLocationRegistryTest.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.query;
 
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.queryablestate.KvStateID;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.state.KeyGroupRange;
@@ -26,6 +27,7 @@ import org.apache.flink.runtime.state.KeyGroupRange;
 import org.junit.Test;
 
 import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -63,7 +65,7 @@ public class KvStateLocationRegistryTest {
 			}
 		}
 
-		KvStateServerAddress server = new KvStateServerAddress(InetAddress.getLocalHost(), 12032);
+		InetSocketAddress server = new InetSocketAddress(InetAddress.getLocalHost(), 12032);
 
 		// Create registry
 		Map<JobVertexID, ExecutionJobVertex> vertexMap = createVertexMap(vertices);
@@ -129,7 +131,7 @@ public class KvStateLocationRegistryTest {
 				new KeyGroupRange(0, 0),
 				registrationName,
 				new KvStateID(),
-				new KvStateServerAddress(InetAddress.getLocalHost(), 12328));
+				new InetSocketAddress(InetAddress.getLocalHost(), 12328));
 
 		try {
 			// Second operator registers same name
@@ -138,7 +140,7 @@ public class KvStateLocationRegistryTest {
 					new KeyGroupRange(0, 0),
 					registrationName,
 					new KvStateID(),
-					new KvStateServerAddress(InetAddress.getLocalHost(), 12032));
+					new InetSocketAddress(InetAddress.getLocalHost(), 12032));
 
 			fail("Did not throw expected Exception after duplicated name");
 		} catch (IllegalStateException ignored) {
@@ -187,7 +189,7 @@ public class KvStateLocationRegistryTest {
 				new KeyGroupRange(0, 0),
 				name,
 				new KvStateID(),
-				mock(KvStateServerAddress.class));
+				mock(InetSocketAddress.class));
 
 		try {
 			// Unregister not registered keyGroupIndex

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-runtime/src/test/java/org/apache/flink/runtime/query/KvStateLocationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/query/KvStateLocationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/query/KvStateLocationTest.java
index 116deea..3c79948 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/query/KvStateLocationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/query/KvStateLocationTest.java
@@ -19,12 +19,14 @@
 package org.apache.flink.runtime.query;
 
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.queryablestate.KvStateID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.state.KeyGroupRange;
 
 import org.junit.Test;
 
 import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -65,7 +67,7 @@ public class KvStateLocationTest {
 		KvStateLocation location = new KvStateLocation(jobId, jobVertexId, numKeyGroups, registrationName);
 
 		KvStateID[] kvStateIds = new KvStateID[numRanges];
-		KvStateServerAddress[] serverAddresses = new KvStateServerAddress[numRanges];
+		InetSocketAddress[] serverAddresses = new InetSocketAddress[numRanges];
 
 		InetAddress host = InetAddress.getLocalHost();
 
@@ -73,7 +75,7 @@ public class KvStateLocationTest {
 		int registeredCount = 0;
 		for (int rangeIdx = 0; rangeIdx < numRanges; rangeIdx++) {
 			kvStateIds[rangeIdx] = new KvStateID();
-			serverAddresses[rangeIdx] = new KvStateServerAddress(host, 1024 + rangeIdx);
+			serverAddresses[rangeIdx] = new InetSocketAddress(host, 1024 + rangeIdx);
 			KeyGroupRange keyGroupRange = keyGroupRanges.get(rangeIdx);
 			location.registerKvState(keyGroupRange, kvStateIds[rangeIdx], serverAddresses[rangeIdx]);
 			registeredCount += keyGroupRange.getNumberOfKeyGroups();
@@ -92,7 +94,7 @@ public class KvStateLocationTest {
 		// Overwrite
 		for (int rangeIdx = 0; rangeIdx < numRanges; rangeIdx++) {
 			kvStateIds[rangeIdx] = new KvStateID();
-			serverAddresses[rangeIdx] = new KvStateServerAddress(host, 1024 + rangeIdx);
+			serverAddresses[rangeIdx] = new InetSocketAddress(host, 1024 + rangeIdx);
 
 			location.registerKvState(keyGroupRanges.get(rangeIdx), kvStateIds[rangeIdx], serverAddresses[rangeIdx]);
 			assertEquals(registeredCount, location.getNumRegisteredKeyGroups());

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/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
deleted file mode 100644
index aa4e6d8..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/message/KvStateRequestSerializerTest.java
+++ /dev/null
@@ -1,415 +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.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.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.query.TaskKvStateRegistry;
-import org.apache.flink.runtime.state.KeyGroupRange;
-import org.apache.flink.runtime.state.VoidNamespace;
-import org.apache.flink.runtime.state.VoidNamespaceSerializer;
-import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend;
-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.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ThreadLocalRandom;
-
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.mock;
-
-/**
- * Tests for {@link KvStateSerializer}.
- */
-@RunWith(Parameterized.class)
-public class KvStateRequestSerializerTest {
-
-	@Parameterized.Parameters
-	public static Collection<Boolean> parameters() {
-		return Arrays.asList(false, true);
-	}
-
-	@Parameterized.Parameter
-	public boolean async;
-
-	/**
-	 * Tests key and namespace serialization utils.
-	 */
-	@Test
-	public void testKeyAndNamespaceSerialization() throws Exception {
-		TypeSerializer<Long> keySerializer = LongSerializer.INSTANCE;
-		TypeSerializer<String> namespaceSerializer = StringSerializer.INSTANCE;
-
-		long expectedKey = Integer.MAX_VALUE + 12323L;
-		String expectedNamespace = "knilf";
-
-		byte[] serializedKeyAndNamespace = KvStateSerializer.serializeKeyAndNamespace(
-				expectedKey, keySerializer, expectedNamespace, namespaceSerializer);
-
-		Tuple2<Long, String> actual = KvStateSerializer.deserializeKeyAndNamespace(
-				serializedKeyAndNamespace, keySerializer, namespaceSerializer);
-
-		assertEquals(expectedKey, actual.f0.longValue());
-		assertEquals(expectedNamespace, actual.f1);
-	}
-
-	/**
-	 * Tests key and namespace deserialization utils with too few bytes.
-	 */
-	@Test(expected = IOException.class)
-	public void testKeyAndNamespaceDeserializationEmpty() throws Exception {
-		KvStateSerializer.deserializeKeyAndNamespace(
-			new byte[] {}, LongSerializer.INSTANCE, StringSerializer.INSTANCE);
-	}
-
-	/**
-	 * Tests key and namespace deserialization utils with too few bytes.
-	 */
-	@Test(expected = IOException.class)
-	public void testKeyAndNamespaceDeserializationTooShort() throws Exception {
-		KvStateSerializer.deserializeKeyAndNamespace(
-			new byte[] {1}, LongSerializer.INSTANCE, StringSerializer.INSTANCE);
-	}
-
-	/**
-	 * Tests key and namespace deserialization utils with too many bytes.
-	 */
-	@Test(expected = IOException.class)
-	public void testKeyAndNamespaceDeserializationTooMany1() throws Exception {
-		// Long + null String + 1 byte
-		KvStateSerializer.deserializeKeyAndNamespace(
-			new byte[] {1, 1, 1, 1, 1, 1, 1, 1, 42, 0, 2}, LongSerializer.INSTANCE,
-			StringSerializer.INSTANCE);
-	}
-
-	/**
-	 * Tests key and namespace deserialization utils with too many bytes.
-	 */
-	@Test(expected = IOException.class)
-	public void testKeyAndNamespaceDeserializationTooMany2() throws Exception {
-		// Long + null String + 2 bytes
-		KvStateSerializer.deserializeKeyAndNamespace(
-			new byte[] {1, 1, 1, 1, 1, 1, 1, 1, 42, 0, 2, 2}, LongSerializer.INSTANCE,
-			StringSerializer.INSTANCE);
-	}
-
-	/**
-	 * Tests value serialization utils.
-	 */
-	@Test
-	public void testValueSerialization() throws Exception {
-		TypeSerializer<Long> valueSerializer = LongSerializer.INSTANCE;
-		long expectedValue = Long.MAX_VALUE - 1292929292L;
-
-		byte[] serializedValue = KvStateSerializer.serializeValue(expectedValue, valueSerializer);
-		long actualValue = KvStateSerializer.deserializeValue(serializedValue, valueSerializer);
-
-		assertEquals(expectedValue, actualValue);
-	}
-
-	/**
-	 * Tests value deserialization with too few bytes.
-	 */
-	@Test(expected = IOException.class)
-	public void testDeserializeValueEmpty() throws Exception {
-		KvStateSerializer.deserializeValue(new byte[] {}, LongSerializer.INSTANCE);
-	}
-
-	/**
-	 * Tests value deserialization with too few bytes.
-	 */
-	@Test(expected = IOException.class)
-	public void testDeserializeValueTooShort() throws Exception {
-		// 1 byte (incomplete Long)
-		KvStateSerializer.deserializeValue(new byte[] {1}, LongSerializer.INSTANCE);
-	}
-
-	/**
-	 * Tests value deserialization with too many bytes.
-	 */
-	@Test(expected = IOException.class)
-	public void testDeserializeValueTooMany1() throws Exception {
-		// Long + 1 byte
-		KvStateSerializer.deserializeValue(new byte[] {1, 1, 1, 1, 1, 1, 1, 1, 2},
-			LongSerializer.INSTANCE);
-	}
-
-	/**
-	 * Tests value deserialization with too many bytes.
-	 */
-	@Test(expected = IOException.class)
-	public void testDeserializeValueTooMany2() throws Exception {
-		// Long + 2 bytes
-		KvStateSerializer.deserializeValue(new byte[] {1, 1, 1, 1, 1, 1, 1, 1, 2, 2},
-			LongSerializer.INSTANCE);
-	}
-
-	/**
-	 * Tests list serialization utils.
-	 */
-	@Test
-	public void testListSerialization() throws Exception {
-		final long key = 0L;
-
-		// objects for heap state list serialisation
-		final HeapKeyedStateBackend<Long> longHeapKeyedStateBackend =
-			new HeapKeyedStateBackend<>(
-				mock(TaskKvStateRegistry.class),
-				LongSerializer.INSTANCE,
-				ClassLoader.getSystemClassLoader(),
-				1,
-				new KeyGroupRange(0, 0),
-				async,
-				new ExecutionConfig()
-			);
-		longHeapKeyedStateBackend.setCurrentKey(key);
-
-		final InternalListState<VoidNamespace, Long> listState = longHeapKeyedStateBackend.createListState(
-				VoidNamespaceSerializer.INSTANCE,
-				new ListStateDescriptor<>("test", LongSerializer.INSTANCE));
-
-		testListSerialization(key, listState);
-	}
-
-	/**
-	 * Verifies that the serialization of a list using the given list state
-	 * matches the deserialization with {@link KvStateSerializer#deserializeList}.
-	 *
-	 * @param key
-	 * 		key of the list state
-	 * @param listState
-	 * 		list state using the {@link VoidNamespace}, must also be a {@link InternalKvState} instance
-	 *
-	 * @throws Exception
-	 */
-	public static void testListSerialization(
-			final long key,
-			final InternalListState<VoidNamespace, Long> listState) throws Exception {
-
-		TypeSerializer<Long> valueSerializer = LongSerializer.INSTANCE;
-		listState.setCurrentNamespace(VoidNamespace.INSTANCE);
-
-		// List
-		final int numElements = 10;
-
-		final List<Long> expectedValues = new ArrayList<>();
-		for (int i = 0; i < numElements; i++) {
-			final long value = ThreadLocalRandom.current().nextLong();
-			expectedValues.add(value);
-			listState.add(value);
-		}
-
-		final byte[] serializedKey =
-			KvStateSerializer.serializeKeyAndNamespace(
-				key, LongSerializer.INSTANCE,
-				VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE);
-
-		final byte[] serializedValues = listState.getSerializedValue(serializedKey);
-
-		List<Long> actualValues = KvStateSerializer.deserializeList(serializedValues, valueSerializer);
-		assertEquals(expectedValues, actualValues);
-
-		// Single value
-		long expectedValue = ThreadLocalRandom.current().nextLong();
-		byte[] serializedValue = KvStateSerializer.serializeValue(expectedValue, valueSerializer);
-		List<Long> actualValue = KvStateSerializer.deserializeList(serializedValue, valueSerializer);
-		assertEquals(1, actualValue.size());
-		assertEquals(expectedValue, actualValue.get(0).longValue());
-	}
-
-	/**
-	 * Tests list deserialization with too few bytes.
-	 */
-	@Test
-	public void testDeserializeListEmpty() throws Exception {
-		List<Long> actualValue = KvStateSerializer
-			.deserializeList(new byte[] {}, LongSerializer.INSTANCE);
-		assertEquals(0, actualValue.size());
-	}
-
-	/**
-	 * Tests list deserialization with too few bytes.
-	 */
-	@Test(expected = IOException.class)
-	public void testDeserializeListTooShort1() throws Exception {
-		// 1 byte (incomplete Long)
-		KvStateSerializer.deserializeList(new byte[] {1}, LongSerializer.INSTANCE);
-	}
-
-	/**
-	 * Tests list deserialization with too few bytes.
-	 */
-	@Test(expected = IOException.class)
-	public void testDeserializeListTooShort2() throws Exception {
-		// Long + 1 byte (separator) + 1 byte (incomplete Long)
-		KvStateSerializer.deserializeList(new byte[] {1, 1, 1, 1, 1, 1, 1, 1, 2, 3},
-			LongSerializer.INSTANCE);
-	}
-
-	/**
-	 * Tests map serialization utils.
-	 */
-	@Test
-	public void testMapSerialization() throws Exception {
-		final long key = 0L;
-
-		// objects for heap state list serialisation
-		final HeapKeyedStateBackend<Long> longHeapKeyedStateBackend =
-			new HeapKeyedStateBackend<>(
-					mock(TaskKvStateRegistry.class),
-					LongSerializer.INSTANCE,
-					ClassLoader.getSystemClassLoader(),
-					1,
-					new KeyGroupRange(0, 0),
-					async,
-					new ExecutionConfig()
-			);
-		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));
-
-		testMapSerialization(key, mapState);
-	}
-
-	/**
-	 * Verifies that the serialization of a map using the given map state
-	 * matches the deserialization with {@link KvStateSerializer#deserializeList}.
-	 *
-	 * @param key
-	 * 		key of the map state
-	 * @param mapState
-	 * 		map state using the {@link VoidNamespace}, must also be a {@link InternalKvState} instance
-	 *
-	 * @throws Exception
-	 */
-	public static void testMapSerialization(
-			final long key,
-			final InternalMapState<VoidNamespace, Long, String> mapState) throws Exception {
-
-		TypeSerializer<Long> userKeySerializer = LongSerializer.INSTANCE;
-		TypeSerializer<String> userValueSerializer = StringSerializer.INSTANCE;
-		mapState.setCurrentNamespace(VoidNamespace.INSTANCE);
-
-		// Map
-		final int numElements = 10;
-
-		final Map<Long, String> expectedValues = new HashMap<>();
-		for (int i = 1; i <= numElements; i++) {
-			final long value = ThreadLocalRandom.current().nextLong();
-			expectedValues.put(value, Long.toString(value));
-			mapState.put(value, Long.toString(value));
-		}
-
-		expectedValues.put(0L, null);
-		mapState.put(0L, null);
-
-		final byte[] serializedKey =
-			KvStateSerializer.serializeKeyAndNamespace(
-				key, LongSerializer.INSTANCE,
-				VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE);
-
-		final byte[] serializedValues = mapState.getSerializedValue(serializedKey);
-
-		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());
-		}
-
-		// Single value
-		ByteArrayOutputStream baos = new ByteArrayOutputStream();
-		long expectedKey = ThreadLocalRandom.current().nextLong();
-		String expectedValue = Long.toString(expectedKey);
-		byte[] isNull = {0};
-
-		baos.write(KvStateSerializer.serializeValue(expectedKey, userKeySerializer));
-		baos.write(isNull);
-		baos.write(KvStateSerializer.serializeValue(expectedValue, userValueSerializer));
-		byte[] serializedValue = baos.toByteArray();
-
-		Map<Long, String> actualValue = KvStateSerializer.deserializeMap(serializedValue, userKeySerializer, userValueSerializer);
-		assertEquals(1, actualValue.size());
-		assertEquals(expectedValue, actualValue.get(expectedKey));
-	}
-
-	/**
-	 * Tests map deserialization with too few bytes.
-	 */
-	@Test
-	public void testDeserializeMapEmpty() throws Exception {
-		Map<Long, String> actualValue = KvStateSerializer
-			.deserializeMap(new byte[] {}, LongSerializer.INSTANCE, StringSerializer.INSTANCE);
-		assertEquals(0, actualValue.size());
-	}
-
-	/**
-	 * Tests map deserialization with too few bytes.
-	 */
-	@Test(expected = IOException.class)
-	public void testDeserializeMapTooShort1() throws Exception {
-		// 1 byte (incomplete Key)
-		KvStateSerializer.deserializeMap(new byte[] {1}, LongSerializer.INSTANCE, StringSerializer.INSTANCE);
-	}
-
-	/**
-	 * Tests map deserialization with too few bytes.
-	 */
-	@Test(expected = IOException.class)
-	public void testDeserializeMapTooShort2() throws Exception {
-		// Long (Key) + 1 byte (incomplete Value)
-		KvStateSerializer.deserializeMap(new byte[]{1, 1, 1, 1, 1, 1, 1, 1, 0},
-				LongSerializer.INSTANCE, LongSerializer.INSTANCE);
-	}
-
-	/**
-	 * Tests map deserialization with too few bytes.
-	 */
-	@Test(expected = IOException.class)
-	public void testDeserializeMapTooShort3() throws Exception {
-		// Long (Key1) + Boolean (false) + Long (Value1) + 1 byte (incomplete Key2)
-		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);
-	}
-
-	private byte[] randomByteArray(int capacity) {
-		byte[] bytes = new byte[capacity];
-		ThreadLocalRandom.current().nextBytes(bytes);
-		return bytes;
-	}
-}


[06/13] flink git commit: [FLINK-7908][QS] Restructure the queryable state module.

Posted by kk...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/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
deleted file mode 100644
index 4023925..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java
+++ /dev/null
@@ -1,784 +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.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 Throwable {
-		// 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(),
-						Collections.singletonList(0).iterator(),
-						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/0c771505/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
deleted file mode 100644
index cb490aa..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateClientHandlerTest.java
+++ /dev/null
@@ -1,119 +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.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;
-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 ClientHandler}.
- */
-public class KvStateClientHandlerTest {
-
-	/**
-	 * Tests that on reads the expected callback methods are called and read
-	 * buffers are recycled.
-	 */
-	@Test
-	public void testReadCallbacksAndBufferRecycling() throws Exception {
-		final ClientHandlerCallback<KvStateResponse> callback = mock(ClientHandlerCallback.class);
-
-		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.serializeResponse(channel.alloc(), 1222112277L, response);
-		buf.skipBytes(4); // skip frame length
-
-		// Verify callback
-		channel.writeInbound(buf);
-		verify(callback, times(1)).onRequestResult(eq(1222112277L), any(KvStateResponse.class));
-		assertEquals("Buffer not recycled", 0, buf.refCnt());
-
-		//
-		// Request failure
-		//
-		buf = MessageSerializer.serializeRequestFailure(
-				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/0c771505/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
deleted file mode 100644
index 217d0b5..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java
+++ /dev/null
@@ -1,758 +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.api.common.typeutils.base.StringSerializer;
-import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer;
-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.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;
-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.BeforeClass;
-import org.junit.Test;
-
-import java.net.InetAddress;
-import java.util.Collections;
-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 {
-
-	private static KvStateServerImpl testServer;
-
-	private static final long READ_TIMEOUT_MILLIS = 10000L;
-
-	@BeforeClass
-	public static void setup() {
-		try {
-			testServer = new KvStateServerImpl(
-					InetAddress.getLocalHost(),
-					Collections.singletonList(0).iterator(),
-					1,
-					1,
-					new KvStateRegistry(),
-					new DisabledKvStateRequestStats());
-			testServer.start();
-		} catch (Throwable e) {
-			e.printStackTrace();
-		}
-	}
-
-	@AfterClass
-	public static void tearDown() throws Exception {
-		testServer.shutdown();
-	}
-
-	/**
-	 * Tests a simple successful query via an EmbeddedChannel.
-	 */
-	@Test
-	public void testSimpleQuery() throws Exception {
-		KvStateRegistry registry = new KvStateRegistry();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		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
-		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"));
-
-		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(serRequest);
-
-		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
-		buf.skipBytes(4); // skip frame length
-
-		// Verify the response
-		assertEquals(MessageType.REQUEST_RESULT, MessageSerializer.deserializeHeader(buf));
-		long deserRequestId = MessageSerializer.getRequestId(buf);
-		KvStateResponse response = serializer.deserializeResponse(buf);
-
-		assertEquals(requestId, deserRequestId);
-
-		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() != 1L && System.nanoTime() <= deadline) {
-			Thread.sleep(10L);
-		}
-
-		assertEquals(stats.toString(), 1L, stats.getNumSuccessful());
-	}
-
-	/**
-	 * Tests the failure response with {@link UnknownKvStateIdException} as cause on
-	 * queries for unregistered KvStateIDs.
-	 */
-	@Test
-	public void testQueryUnknownKvStateID() throws Exception {
-		KvStateRegistry registry = new KvStateRegistry();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		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;
-
-		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(serRequest);
-
-		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
-		buf.skipBytes(4); // skip frame length
-
-		// Verify the response
-		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
-		RequestFailure response = MessageSerializer.deserializeRequestFailure(buf);
-
-		assertEquals(requestId, response.getRequestId());
-
-		assertTrue("Did not respond with expected failure cause", response.getCause() instanceof UnknownKvStateIdException);
-
-		assertEquals(1L, stats.getNumRequests());
-		assertEquals(1L, stats.getNumFailed());
-	}
-
-	/**
-	 * Tests the failure response with {@link UnknownKeyOrNamespaceException} as cause
-	 * on queries for non-existing keys.
-	 */
-	@Test
-	public void testQueryUnknownKey() throws Exception {
-		KvStateRegistry registry = new KvStateRegistry();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		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;
-		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"));
-
-		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(serRequest);
-
-		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
-		buf.skipBytes(4); // skip frame length
-
-		// Verify the response
-		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
-		RequestFailure response = MessageSerializer.deserializeRequestFailure(buf);
-
-		assertEquals(requestId, response.getRequestId());
-
-		assertTrue("Did not respond with expected failure cause", response.getCause() instanceof UnknownKeyOrNamespaceException);
-
-		assertEquals(1L, stats.getNumRequests());
-		assertEquals(1L, 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();
-
-		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
-		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);
-
-		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(serRequest);
-
-		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
-		buf.skipBytes(4); // skip frame length
-
-		// Verify the response
-		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
-		RequestFailure response = MessageSerializer.deserializeRequestFailure(buf);
-
-		assertTrue(response.getCause().getMessage().contains("Expected test Exception"));
-
-		assertEquals(1L, stats.getNumRequests());
-		assertEquals(1L, 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();
-
-		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"));
-
-		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 Throwable {
-		KvStateRegistry registry = new KvStateRegistry();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		KvStateServerImpl localTestServer = new KvStateServerImpl(
-				InetAddress.getLocalHost(),
-				Collections.singletonList(0).iterator(),
-				1,
-				1,
-				new KvStateRegistry(),
-				new DisabledKvStateRequestStats());
-
-		localTestServer.start();
-		localTestServer.shutdown();
-		assertTrue(localTestServer.isExecutorShutdown());
-
-		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;
-		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"));
-
-		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(serRequest);
-
-		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
-		buf.skipBytes(4); // skip frame length
-
-		// Verify the response
-		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
-		RequestFailure response = MessageSerializer.deserializeRequestFailure(buf);
-
-		assertTrue(response.getCause().getMessage().contains("RejectedExecutionException"));
-
-		assertEquals(1L, stats.getNumRequests());
-		assertEquals(1L, stats.getNumFailed());
-
-		localTestServer.shutdown();
-	}
-
-	/**
-	 * Tests response on unexpected messages.
-	 */
-	@Test
-	public void testUnexpectedMessage() throws Exception {
-		KvStateRegistry registry = new KvStateRegistry();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		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
-		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(0L, stats.getNumRequests());
-		assertEquals(0L, stats.getNumFailed());
-
-		KvStateResponse stateResponse = new KvStateResponse(new byte[0]);
-		unexpectedMessage = MessageSerializer.serializeResponse(channel.alloc(), 192L, stateResponse);
-
-		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(0L, stats.getNumRequests());
-		assertEquals(0L, stats.getNumFailed());
-	}
-
-	/**
-	 * Tests that incoming buffer instances are recycled.
-	 */
-	@Test
-	public void testIncomingBufferIsRecycled() throws Exception {
-		KvStateRegistry registry = new KvStateRegistry();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		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);
-
-		KvStateInternalRequest request = new KvStateInternalRequest(new KvStateID(), new byte[0]);
-		ByteBuf serRequest = MessageSerializer.serializeRequest(channel.alloc(), 282872L, request);
-
-		assertEquals(1L, serRequest.refCnt());
-
-		// Write regular request
-		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(1L, unexpected.refCnt());
-
-		channel.writeInbound(unexpected);
-		assertEquals("Buffer not recycled", 0L, 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();
-
-		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;
-		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"));
-
-		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(serRequest);
-
-		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
-		buf.skipBytes(4); // skip frame length
-
-		// Verify the response
-		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
-		RequestFailure response = MessageSerializer.deserializeRequestFailure(buf);
-		assertEquals(182828L, response.getRequestId());
-		assertTrue(response.getCause().getMessage().contains("IOException"));
-
-		// Repeat with wrong namespace only
-		request = new KvStateInternalRequest(registryListener.kvStateId, wrongNamespace);
-		serRequest = MessageSerializer.serializeRequest(channel.alloc(), 182829L, request);
-
-		// Write the request and wait for the response
-		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.deserializeRequestFailure(buf);
-		assertEquals(182829L, response.getRequestId());
-		assertTrue(response.getCause().getMessage().contains("IOException"));
-
-		assertEquals(2L, stats.getNumRequests());
-		assertEquals(2L, stats.getNumFailed());
-	}
-
-	/**
-	 * Tests that large responses are chunked.
-	 */
-	@Test
-	public void testChunkedResponse() throws Exception {
-		KvStateRegistry registry = new KvStateRegistry();
-		KvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		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;
-		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"));
-
-		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(serRequest);
-
-		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 long sleepMillis = 50L;
-
-		long sleptMillis = 0L;
-
-		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/0c771505/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
deleted file mode 100644
index 7abc84e..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java
+++ /dev/null
@@ -1,207 +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.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.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.Collections;
-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 KvStateServerImpl}.
- */
-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 Throwable {
-		KvStateServerImpl server = null;
-		Bootstrap bootstrap = null;
-		try {
-			KvStateRegistry registry = new KvStateRegistry();
-			KvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-			server = new KvStateServerImpl(InetAddress.getLocalHost(),
-					Collections.singletonList(0).iterator(), 1, 1, registry, stats);
-			server.start();
-
-			KvStateServerAddress serverAddress = server.getServerAddress();
-			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"));
-
-			final KvStateInternalRequest request = new KvStateInternalRequest(
-					registryListener.kvStateId,
-					serializedKeyAndNamespace);
-
-			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));
-			assertEquals(requestId, MessageSerializer.getRequestId(buf));
-			KvStateResponse response = server.getSerializer().deserializeResponse(buf);
-
-			int actualValue = KvStateSerializer.deserializeValue(response.getContent(), 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/0c771505/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
deleted file mode 100644
index 32a0c9b..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/MessageSerializerTest.java
+++ /dev/null
@@ -1,220 +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.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;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableAggregatingStateTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableAggregatingStateTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableAggregatingStateTest.java
deleted file mode 100644
index 2e05f61..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableAggregatingStateTest.java
+++ /dev/null
@@ -1,108 +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.state;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.AggregateFunction;
-import org.apache.flink.api.common.state.AggregatingStateDescriptor;
-import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
-import org.apache.flink.queryablestate.client.state.ImmutableAggregatingState;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.ByteArrayOutputStream;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Tests the {@link ImmutableAggregatingStateTest}.
- */
-public class ImmutableAggregatingStateTest {
-
-	private final AggregatingStateDescriptor<Long, String, String> aggrStateDesc =
-			new AggregatingStateDescriptor<>(
-					"test",
-					new SumAggr(),
-					String.class);
-
-	private ImmutableAggregatingState<Long, String> aggrState;
-
-	@Before
-	public void setUp() throws Exception {
-		if (!aggrStateDesc.isSerializerInitialized()) {
-			aggrStateDesc.initializeSerializerUnlessSet(new ExecutionConfig());
-		}
-
-		final String initValue = "42";
-
-		ByteArrayOutputStream out = new ByteArrayOutputStream();
-		aggrStateDesc.getSerializer().serialize(initValue, new DataOutputViewStreamWrapper(out));
-
-		aggrState = ImmutableAggregatingState.createState(
-				aggrStateDesc,
-				out.toByteArray()
-		);
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testUpdate() {
-		String value = aggrState.get();
-		assertEquals("42", value);
-
-		aggrState.add(54L);
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testClear() {
-		String value = aggrState.get();
-		assertEquals("42", value);
-
-		aggrState.clear();
-	}
-
-	/**
-	 * Test {@link AggregateFunction} concatenating the already stored string with the long passed as argument.
-	 */
-	private static class SumAggr implements AggregateFunction<Long, String, String> {
-
-		private static final long serialVersionUID = -6249227626701264599L;
-
-		@Override
-		public String createAccumulator() {
-			return "";
-		}
-
-		@Override
-		public String add(Long value, String accumulator) {
-			accumulator += ", " + value;
-			return accumulator;
-		}
-
-		@Override
-		public String getResult(String accumulator) {
-			return accumulator;
-		}
-
-		@Override
-		public String merge(String a, String b) {
-			return a + ", " + b;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableFoldingStateTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableFoldingStateTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableFoldingStateTest.java
deleted file mode 100644
index d2c9535..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableFoldingStateTest.java
+++ /dev/null
@@ -1,94 +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.state;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.FoldFunction;
-import org.apache.flink.api.common.state.FoldingStateDescriptor;
-import org.apache.flink.api.common.typeutils.base.StringSerializer;
-import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
-import org.apache.flink.queryablestate.client.state.ImmutableFoldingState;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.ByteArrayOutputStream;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Tests the {@link ImmutableFoldingState}.
- */
-public class ImmutableFoldingStateTest {
-
-	private final FoldingStateDescriptor<Long, String> foldingStateDesc =
-			new FoldingStateDescriptor<>(
-					"test",
-					"0",
-					new SumFold(),
-					StringSerializer.INSTANCE);
-
-	private ImmutableFoldingState<Long, String> foldingState;
-
-	@Before
-	public void setUp() throws Exception {
-		if (!foldingStateDesc.isSerializerInitialized()) {
-			foldingStateDesc.initializeSerializerUnlessSet(new ExecutionConfig());
-		}
-
-		ByteArrayOutputStream out = new ByteArrayOutputStream();
-		StringSerializer.INSTANCE.serialize("42", new DataOutputViewStreamWrapper(out));
-
-		foldingState = ImmutableFoldingState.createState(
-				foldingStateDesc,
-				out.toByteArray()
-		);
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testUpdate() {
-		String value = foldingState.get();
-		assertEquals("42", value);
-
-		foldingState.add(54L);
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testClear() {
-		String value = foldingState.get();
-		assertEquals("42", value);
-
-		foldingState.clear();
-	}
-
-	/**
-	 * Test {@link FoldFunction} concatenating the already stored string with the long passed as argument.
-	 */
-	private static class SumFold implements FoldFunction<Long, String> {
-
-		private static final long serialVersionUID = -6249227626701264599L;
-
-		@Override
-		public String fold(String accumulator, Long value) throws Exception {
-			long acc = Long.valueOf(accumulator);
-			acc += value;
-			return Long.toString(acc);
-		}
-	}
-}


[10/13] flink git commit: [FLINK-7908][QS] Restructure the queryable state module.

Posted by kk...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/ClientHandler.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/ClientHandler.java b/flink-queryable-state/flink-queryable-state-client-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-client-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/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/ClientHandlerCallback.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/ClientHandlerCallback.java b/flink-queryable-state/flink-queryable-state-client-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-client-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/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/NettyBufferPool.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/NettyBufferPool.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/NettyBufferPool.java
new file mode 100644
index 0000000..5e014b8
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/NettyBufferPool.java
@@ -0,0 +1,171 @@
+/*
+ * 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.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.CompositeByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.buffer.PooledByteBufAllocator;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/**
+ * Wrapper around Netty's {@link PooledByteBufAllocator} with strict control
+ * over the number of created arenas.
+ */
+public class NettyBufferPool implements ByteBufAllocator {
+
+	/** The wrapped buffer allocator. */
+	private final PooledByteBufAllocator alloc;
+
+	/**
+	 * Creates Netty's buffer pool with the specified number of direct arenas.
+	 *
+	 * @param numberOfArenas Number of arenas (recommended: 2 * number of task
+	 *                       slots)
+	 */
+	public NettyBufferPool(int numberOfArenas) {
+		checkArgument(numberOfArenas >= 1, "Number of arenas");
+
+		// We strictly prefer direct buffers and disallow heap allocations.
+		boolean preferDirect = true;
+
+		// Arenas allocate chunks of pageSize << maxOrder bytes. With these
+		// defaults, this results in chunks of 16 MB.
+		int pageSize = 8192;
+		int maxOrder = 11;
+
+		// Number of direct arenas. Each arena allocates a chunk of 16 MB, i.e.
+		// we allocate numDirectArenas * 16 MB of direct memory. This can grow
+		// to multiple chunks per arena during runtime, but this should only
+		// happen with a large amount of connections per task manager. We
+		// control the memory allocations with low/high watermarks when writing
+		// to the TCP channels. Chunks are allocated lazily.
+		int numDirectArenas = numberOfArenas;
+
+		// No heap arenas, please.
+		int numHeapArenas = 0;
+
+		this.alloc = new PooledByteBufAllocator(
+				preferDirect,
+				numHeapArenas,
+				numDirectArenas,
+				pageSize,
+				maxOrder);
+	}
+
+	// ------------------------------------------------------------------------
+	// Delegate calls to the allocated and prohibit heap buffer allocations
+	// ------------------------------------------------------------------------
+
+	@Override
+	public ByteBuf buffer() {
+		return alloc.buffer();
+	}
+
+	@Override
+	public ByteBuf buffer(int initialCapacity) {
+		return alloc.buffer(initialCapacity);
+	}
+
+	@Override
+	public ByteBuf buffer(int initialCapacity, int maxCapacity) {
+		return alloc.buffer(initialCapacity, maxCapacity);
+	}
+
+	@Override
+	public ByteBuf ioBuffer() {
+		return alloc.ioBuffer();
+	}
+
+	@Override
+	public ByteBuf ioBuffer(int initialCapacity) {
+		return alloc.ioBuffer(initialCapacity);
+	}
+
+	@Override
+	public ByteBuf ioBuffer(int initialCapacity, int maxCapacity) {
+		return alloc.ioBuffer(initialCapacity, maxCapacity);
+	}
+
+	@Override
+	public ByteBuf heapBuffer() {
+		throw new UnsupportedOperationException("Heap buffer");
+	}
+
+	@Override
+	public ByteBuf heapBuffer(int initialCapacity) {
+		throw new UnsupportedOperationException("Heap buffer");
+	}
+
+	@Override
+	public ByteBuf heapBuffer(int initialCapacity, int maxCapacity) {
+		throw new UnsupportedOperationException("Heap buffer");
+	}
+
+	@Override
+	public ByteBuf directBuffer() {
+		return alloc.directBuffer();
+	}
+
+	@Override
+	public ByteBuf directBuffer(int initialCapacity) {
+		return alloc.directBuffer(initialCapacity);
+	}
+
+	@Override
+	public ByteBuf directBuffer(int initialCapacity, int maxCapacity) {
+		return alloc.directBuffer(initialCapacity, maxCapacity);
+	}
+
+	@Override
+	public CompositeByteBuf compositeBuffer() {
+		return alloc.compositeBuffer();
+	}
+
+	@Override
+	public CompositeByteBuf compositeBuffer(int maxNumComponents) {
+		return alloc.compositeBuffer(maxNumComponents);
+	}
+
+	@Override
+	public CompositeByteBuf compositeHeapBuffer() {
+		throw new UnsupportedOperationException("Heap buffer");
+	}
+
+	@Override
+	public CompositeByteBuf compositeHeapBuffer(int maxNumComponents) {
+		throw new UnsupportedOperationException("Heap buffer");
+	}
+
+	@Override
+	public CompositeByteBuf compositeDirectBuffer() {
+		return alloc.compositeDirectBuffer();
+	}
+
+	@Override
+	public CompositeByteBuf compositeDirectBuffer(int maxNumComponents) {
+		return alloc.compositeDirectBuffer(maxNumComponents);
+	}
+
+	@Override
+	public boolean isDirectBufferPooled() {
+		return alloc.isDirectBufferPooled();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageBody.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageBody.java b/flink-queryable-state/flink-queryable-state-client-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-client-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/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageDeserializer.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageDeserializer.java b/flink-queryable-state/flink-queryable-state-client-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-client-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/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageSerializer.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageSerializer.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageSerializer.java
new file mode 100644
index 0000000..c0a0d32
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageSerializer.java
@@ -0,0 +1,320 @@
+/*
+ * 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.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.network.Client client} and
+ * {@link org.apache.flink.queryablestate.network.AbstractServerBase 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}.
+ *
+ * @param <REQ>		Type of the requests of the protocol.
+ * @param <RESP>	Type of the responses of the protocol.
+ */
+@Internal
+public final class MessageSerializer<REQ extends MessageBody, RESP extends MessageBody> {
+
+	/** 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;
+
+	/** 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
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Serializes the request sent to the
+	 * {@link org.apache.flink.queryablestate.network.AbstractServerBase}.
+	 *
+	 * @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 <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());
+	}
+
+	/**
+	 * Serializes the response sent to the
+	 * {@link org.apache.flink.queryablestate.network.Client}.
+	 *
+	 * @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 <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.network.Client} 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 serializeRequestFailure(
+			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.network.Client} 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());
+	}
+
+	/**
+	 * 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
+	// ------------------------------------------------------------------------
+
+	/**
+	 * 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];
+	}
+
+	/**
+	 * 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 long getRequestId(final ByteBuf buf) {
+		return buf.readLong();
+	}
+
+	/**
+	 * 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 REQ deserializeRequest(final ByteBuf buf) {
+		Preconditions.checkNotNull(buf);
+		return requestDeserializer.deserializeMessage(buf);
+	}
+
+	/**
+	 * 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 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>
+	 * </pre>
+	 * @param buf	The {@link ByteBuf} containing the serialized failure message.
+	 * @return		The failure message.
+	 */
+	public static RequestFailure deserializeRequestFailure(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 RequestFailure(requestId, cause);
+	}
+
+	/**
+	 * De-serializes the failure message sent to the
+	 * {@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>
+	 * </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/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageType.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageType.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageType.java
new file mode 100644
index 0000000..562ce93
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageType.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.network.messages;
+
+import org.apache.flink.annotation.Internal;
+
+/**
+ * Expected message types during the communication between
+ * {@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. */
+	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/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/messages/RequestFailure.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/messages/RequestFailure.java b/flink-queryable-state/flink-queryable-state-client-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-client-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/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats/AtomicKvStateRequestStats.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats/AtomicKvStateRequestStats.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats/AtomicKvStateRequestStats.java
new file mode 100644
index 0000000..9ba5f84
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats/AtomicKvStateRequestStats.java
@@ -0,0 +1,104 @@
+/*
+ * 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.stats;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * Atomic {@link KvStateRequestStats} implementation.
+ */
+public class AtomicKvStateRequestStats implements KvStateRequestStats {
+
+	/**
+	 * Number of active connections.
+	 */
+	private final AtomicLong numConnections = new AtomicLong();
+
+	/**
+	 * Total number of reported requests.
+	 */
+	private final AtomicLong numRequests = new AtomicLong();
+
+	/**
+	 * Total number of successful requests (<= reported requests).
+	 */
+	private final AtomicLong numSuccessful = new AtomicLong();
+
+	/**
+	 * Total duration of all successful requests.
+	 */
+	private final AtomicLong successfulDuration = new AtomicLong();
+
+	/**
+	 * Total number of failed requests (<= reported requests).
+	 */
+	private final AtomicLong numFailed = new AtomicLong();
+
+	@Override
+	public void reportActiveConnection() {
+		numConnections.incrementAndGet();
+	}
+
+	@Override
+	public void reportInactiveConnection() {
+		numConnections.decrementAndGet();
+	}
+
+	@Override
+	public void reportRequest() {
+		numRequests.incrementAndGet();
+	}
+
+	@Override
+	public void reportSuccessfulRequest(long durationTotalMillis) {
+		numSuccessful.incrementAndGet();
+		successfulDuration.addAndGet(durationTotalMillis);
+	}
+
+	@Override
+	public void reportFailedRequest() {
+		numFailed.incrementAndGet();
+	}
+
+	public long getNumConnections() {
+		return numConnections.get();
+	}
+
+	public long getNumRequests() {
+		return numRequests.get();
+	}
+
+	public long getNumSuccessful() {
+		return numSuccessful.get();
+	}
+
+	public long getNumFailed() {
+		return numFailed.get();
+	}
+
+	@Override
+	public String toString() {
+		return "AtomicKvStateRequestStats{" +
+				"numConnections=" + numConnections +
+				", numRequests=" + numRequests +
+				", numSuccessful=" + numSuccessful +
+				", numFailed=" + numFailed +
+				'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats/DisabledKvStateRequestStats.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats/DisabledKvStateRequestStats.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats/DisabledKvStateRequestStats.java
new file mode 100644
index 0000000..b34ac3e
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats/DisabledKvStateRequestStats.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.queryablestate.network.stats;
+
+/**
+ * Disabled {@link KvStateRequestStats} implementation.
+ */
+public class DisabledKvStateRequestStats implements KvStateRequestStats {
+
+	@Override
+	public void reportActiveConnection() {
+	}
+
+	@Override
+	public void reportInactiveConnection() {
+	}
+
+	@Override
+	public void reportRequest() {
+	}
+
+	@Override
+	public void reportSuccessfulRequest(long durationTotalMillis) {
+	}
+
+	@Override
+	public void reportFailedRequest() {
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats/KvStateRequestStats.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats/KvStateRequestStats.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats/KvStateRequestStats.java
new file mode 100644
index 0000000..8e9edd8
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats/KvStateRequestStats.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.network.stats;
+
+/**
+ * Simple statistics for monitoring the state server
+ * and the client proxy.
+ */
+public interface KvStateRequestStats {
+
+	/**
+	 * Reports an active connection.
+	 */
+	void reportActiveConnection();
+
+	/**
+	 * Reports an inactive connection.
+	 */
+	void reportInactiveConnection();
+
+	/**
+	 * Reports an incoming request.
+	 */
+	void reportRequest();
+
+	/**
+	 * Reports a successfully handled request.
+	 *
+	 * @param durationTotalMillis Duration of the request (in milliseconds).
+	 */
+	void reportSuccessfulRequest(long durationTotalMillis);
+
+	/**
+	 * Reports a failure during a request.
+	 */
+	void reportFailedRequest();
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/VoidNamespaceTypeInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/VoidNamespaceTypeInfoTest.java b/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/VoidNamespaceTypeInfoTest.java
new file mode 100644
index 0000000..ca11a32
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/VoidNamespaceTypeInfoTest.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.queryablestate.client;
+
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
+
+/**
+ * Test for {@link VoidNamespaceTypeInfo}.
+ */
+public class VoidNamespaceTypeInfoTest extends TypeInformationTestBase<VoidNamespaceTypeInfo> {
+
+	@Override
+	protected VoidNamespaceTypeInfo[] getTestData() {
+		return new VoidNamespaceTypeInfo[] { VoidNamespaceTypeInfo.INSTANCE };
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableAggregatingStateTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableAggregatingStateTest.java b/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableAggregatingStateTest.java
new file mode 100644
index 0000000..ebbc896
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableAggregatingStateTest.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.state;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.AggregateFunction;
+import org.apache.flink.api.common.state.AggregatingStateDescriptor;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests the {@link ImmutableAggregatingStateTest}.
+ */
+public class ImmutableAggregatingStateTest {
+
+	private final AggregatingStateDescriptor<Long, String, String> aggrStateDesc =
+			new AggregatingStateDescriptor<>(
+					"test",
+					new SumAggr(),
+					String.class);
+
+	private ImmutableAggregatingState<Long, String> aggrState;
+
+	@Before
+	public void setUp() throws Exception {
+		if (!aggrStateDesc.isSerializerInitialized()) {
+			aggrStateDesc.initializeSerializerUnlessSet(new ExecutionConfig());
+		}
+
+		final String initValue = "42";
+
+		ByteArrayOutputStream out = new ByteArrayOutputStream();
+		aggrStateDesc.getSerializer().serialize(initValue, new DataOutputViewStreamWrapper(out));
+
+		aggrState = ImmutableAggregatingState.createState(
+				aggrStateDesc,
+				out.toByteArray()
+		);
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testUpdate() {
+		String value = aggrState.get();
+		assertEquals("42", value);
+
+		aggrState.add(54L);
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testClear() {
+		String value = aggrState.get();
+		assertEquals("42", value);
+
+		aggrState.clear();
+	}
+
+	/**
+	 * Test {@link AggregateFunction} concatenating the already stored string with the long passed as argument.
+	 */
+	private static class SumAggr implements AggregateFunction<Long, String, String> {
+
+		private static final long serialVersionUID = -6249227626701264599L;
+
+		@Override
+		public String createAccumulator() {
+			return "";
+		}
+
+		@Override
+		public String add(Long value, String accumulator) {
+			accumulator += ", " + value;
+			return accumulator;
+		}
+
+		@Override
+		public String getResult(String accumulator) {
+			return accumulator;
+		}
+
+		@Override
+		public String merge(String a, String b) {
+			return a + ", " + b;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableFoldingStateTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableFoldingStateTest.java b/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableFoldingStateTest.java
new file mode 100644
index 0000000..9e8dfc9
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableFoldingStateTest.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.client.state;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.FoldFunction;
+import org.apache.flink.api.common.state.FoldingStateDescriptor;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests the {@link ImmutableFoldingState}.
+ */
+public class ImmutableFoldingStateTest {
+
+	private final FoldingStateDescriptor<Long, String> foldingStateDesc =
+			new FoldingStateDescriptor<>(
+					"test",
+					"0",
+					new SumFold(),
+					StringSerializer.INSTANCE);
+
+	private ImmutableFoldingState<Long, String> foldingState;
+
+	@Before
+	public void setUp() throws Exception {
+		if (!foldingStateDesc.isSerializerInitialized()) {
+			foldingStateDesc.initializeSerializerUnlessSet(new ExecutionConfig());
+		}
+
+		ByteArrayOutputStream out = new ByteArrayOutputStream();
+		StringSerializer.INSTANCE.serialize("42", new DataOutputViewStreamWrapper(out));
+
+		foldingState = ImmutableFoldingState.createState(
+				foldingStateDesc,
+				out.toByteArray()
+		);
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testUpdate() {
+		String value = foldingState.get();
+		assertEquals("42", value);
+
+		foldingState.add(54L);
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testClear() {
+		String value = foldingState.get();
+		assertEquals("42", value);
+
+		foldingState.clear();
+	}
+
+	/**
+	 * Test {@link FoldFunction} concatenating the already stored string with the long passed as argument.
+	 */
+	private static class SumFold implements FoldFunction<Long, String> {
+
+		private static final long serialVersionUID = -6249227626701264599L;
+
+		@Override
+		public String fold(String accumulator, Long value) throws Exception {
+			long acc = Long.valueOf(accumulator);
+			acc += value;
+			return Long.toString(acc);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableListStateTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableListStateTest.java b/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableListStateTest.java
new file mode 100644
index 0000000..a78ed1f
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableListStateTest.java
@@ -0,0 +1,110 @@
+/*
+ * 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.state;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests the {@link ImmutableListState}.
+ */
+public class ImmutableListStateTest {
+
+	private final ListStateDescriptor<Long> listStateDesc =
+			new ListStateDescriptor<>("test", BasicTypeInfo.LONG_TYPE_INFO);
+
+	private ImmutableListState<Long> listState;
+
+	@Before
+	public void setUp() throws Exception {
+		if (!listStateDesc.isSerializerInitialized()) {
+			listStateDesc.initializeSerializerUnlessSet(new ExecutionConfig());
+		}
+
+		List<Long> init = new ArrayList<>();
+		init.add(42L);
+
+		byte[] serInit = serializeInitValue(init);
+		listState = ImmutableListState.createState(listStateDesc, serInit);
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testUpdate() {
+		List<Long> list = getStateContents();
+		assertEquals(1L, list.size());
+
+		long element = list.get(0);
+		assertEquals(42L, element);
+
+		listState.add(54L);
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testClear() {
+		List<Long> list = getStateContents();
+		assertEquals(1L, list.size());
+
+		long element = list.get(0);
+		assertEquals(42L, element);
+
+		listState.clear();
+	}
+
+	/**
+	 * Copied from HeapListState.getSerializedValue(Object, Object).
+	 */
+	private byte[] serializeInitValue(List<Long> toSerialize) throws IOException {
+		TypeSerializer<Long> serializer = listStateDesc.getElementSerializer();
+
+		ByteArrayOutputStream baos = new ByteArrayOutputStream();
+		DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(baos);
+
+		// write the same as RocksDB writes lists, with one ',' separator
+		for (int i = 0; i < toSerialize.size(); i++) {
+			serializer.serialize(toSerialize.get(i), view);
+			if (i < toSerialize.size() - 1) {
+				view.writeByte(',');
+			}
+		}
+		view.flush();
+
+		return baos.toByteArray();
+	}
+
+	private List<Long> getStateContents() {
+		List<Long> list = new ArrayList<>();
+		for (Long elem: listState.get()) {
+			list.add(elem);
+		}
+		return list;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableMapStateTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableMapStateTest.java b/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableMapStateTest.java
new file mode 100644
index 0000000..ffeabae
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableMapStateTest.java
@@ -0,0 +1,188 @@
+/*
+ * 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.state;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests the {@link ImmutableMapState}.
+ */
+public class ImmutableMapStateTest {
+
+	private final MapStateDescriptor<Long, Long> mapStateDesc =
+			new MapStateDescriptor<>(
+					"test",
+					BasicTypeInfo.LONG_TYPE_INFO,
+					BasicTypeInfo.LONG_TYPE_INFO);
+
+	private ImmutableMapState<Long, Long> mapState;
+
+	@Before
+	public void setUp() throws Exception {
+		if (!mapStateDesc.isSerializerInitialized()) {
+			mapStateDesc.initializeSerializerUnlessSet(new ExecutionConfig());
+		}
+
+		Map<Long, Long> initMap = new HashMap<>();
+		initMap.put(1L, 5L);
+		initMap.put(2L, 5L);
+
+		byte[] initSer = KvStateSerializer.serializeMap(
+				initMap.entrySet(),
+				BasicTypeInfo.LONG_TYPE_INFO.createSerializer(new ExecutionConfig()),
+				BasicTypeInfo.LONG_TYPE_INFO.createSerializer(new ExecutionConfig()));
+
+		mapState = ImmutableMapState.createState(mapStateDesc, initSer);
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testPut() {
+		assertTrue(mapState.contains(1L));
+		long value = mapState.get(1L);
+		assertEquals(5L, value);
+
+		assertTrue(mapState.contains(2L));
+		value = mapState.get(2L);
+		assertEquals(5L, value);
+
+		mapState.put(2L, 54L);
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testPutAll() {
+		assertTrue(mapState.contains(1L));
+		long value = mapState.get(1L);
+		assertEquals(5L, value);
+
+		assertTrue(mapState.contains(2L));
+		value = mapState.get(2L);
+		assertEquals(5L, value);
+
+		Map<Long, Long> nMap = new HashMap<>();
+		nMap.put(1L, 7L);
+		nMap.put(2L, 7L);
+
+		mapState.putAll(nMap);
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testUpdate() {
+		assertTrue(mapState.contains(1L));
+		long value = mapState.get(1L);
+		assertEquals(5L, value);
+
+		assertTrue(mapState.contains(2L));
+		value = mapState.get(2L);
+		assertEquals(5L, value);
+
+		mapState.put(2L, 54L);
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testIterator() {
+		assertTrue(mapState.contains(1L));
+		long value = mapState.get(1L);
+		assertEquals(5L, value);
+
+		assertTrue(mapState.contains(2L));
+		value = mapState.get(2L);
+		assertEquals(5L, value);
+
+		Iterator<Map.Entry<Long, Long>> iterator = mapState.iterator();
+		while (iterator.hasNext()) {
+			iterator.remove();
+		}
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testIterable() {
+		assertTrue(mapState.contains(1L));
+		long value = mapState.get(1L);
+		assertEquals(5L, value);
+
+		assertTrue(mapState.contains(2L));
+		value = mapState.get(2L);
+		assertEquals(5L, value);
+
+		Iterable<Map.Entry<Long, Long>> iterable = mapState.entries();
+		Iterator<Map.Entry<Long, Long>> iterator = iterable.iterator();
+		while (iterator.hasNext()) {
+			assertEquals(5L, (long) iterator.next().getValue());
+			iterator.remove();
+		}
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testKeys() {
+		assertTrue(mapState.contains(1L));
+		long value = mapState.get(1L);
+		assertEquals(5L, value);
+
+		assertTrue(mapState.contains(2L));
+		value = mapState.get(2L);
+		assertEquals(5L, value);
+
+		Iterator<Long> iterator = mapState.keys().iterator();
+		while (iterator.hasNext()) {
+			iterator.remove();
+		}
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testValues() {
+		assertTrue(mapState.contains(1L));
+		long value = mapState.get(1L);
+		assertEquals(5L, value);
+
+		assertTrue(mapState.contains(2L));
+		value = mapState.get(2L);
+		assertEquals(5L, value);
+
+		Iterator<Long> iterator = mapState.values().iterator();
+		while (iterator.hasNext()) {
+			iterator.remove();
+		}
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testClear() {
+		assertTrue(mapState.contains(1L));
+		long value = mapState.get(1L);
+		assertEquals(5L, value);
+
+		assertTrue(mapState.contains(2L));
+		value = mapState.get(2L);
+		assertEquals(5L, value);
+
+		mapState.clear();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableReducingStateTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableReducingStateTest.java b/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableReducingStateTest.java
new file mode 100644
index 0000000..9694f55
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableReducingStateTest.java
@@ -0,0 +1,83 @@
+/*
+ * 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.state;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.state.ReducingStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests the {@link ImmutableReducingState}.
+ */
+public class ImmutableReducingStateTest {
+
+	private final ReducingStateDescriptor<Long> reducingStateDesc =
+			new ReducingStateDescriptor<>("test", new SumReduce(), BasicTypeInfo.LONG_TYPE_INFO);
+
+	private ImmutableReducingState<Long> reduceState;
+
+	@Before
+	public void setUp() throws Exception {
+		if (!reducingStateDesc.isSerializerInitialized()) {
+			reducingStateDesc.initializeSerializerUnlessSet(new ExecutionConfig());
+		}
+
+		reduceState = ImmutableReducingState.createState(
+				reducingStateDesc,
+				ByteBuffer.allocate(Long.BYTES).putLong(42L).array()
+		);
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testUpdate() {
+		long value = reduceState.get();
+		assertEquals(42L, value);
+
+		reduceState.add(54L);
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testClear() {
+		long value = reduceState.get();
+		assertEquals(42L, value);
+
+		reduceState.clear();
+	}
+
+	/**
+	 * Test {@link ReduceFunction} summing up its two arguments.
+	 */
+	private static class SumReduce implements ReduceFunction<Long> {
+
+		private static final long serialVersionUID = 6041237513913189144L;
+
+		@Override
+		public Long reduce(Long value1, Long value2) throws Exception {
+			return value1 + value2;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableValueStateTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableValueStateTest.java b/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableValueStateTest.java
new file mode 100644
index 0000000..a0da43d
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableValueStateTest.java
@@ -0,0 +1,69 @@
+/*
+ * 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.state;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests the {@link ImmutableValueState}.
+ */
+public class ImmutableValueStateTest {
+
+	private final ValueStateDescriptor<Long> valueStateDesc =
+			new ValueStateDescriptor<>("test", BasicTypeInfo.LONG_TYPE_INFO);
+
+	private ImmutableValueState<Long> valueState;
+
+	@Before
+	public void setUp() throws Exception {
+		if (!valueStateDesc.isSerializerInitialized()) {
+			valueStateDesc.initializeSerializerUnlessSet(new ExecutionConfig());
+		}
+
+		valueState = ImmutableValueState.createState(
+				valueStateDesc,
+				ByteBuffer.allocate(Long.BYTES).putLong(42L).array()
+		);
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testUpdate() {
+		long value = valueState.value();
+		assertEquals(42L, value);
+
+		valueState.update(54L);
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testClear() {
+		long value = valueState.value();
+		assertEquals(42L, value);
+
+		valueState.clear();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/test/resources/log4j-test.properties b/flink-queryable-state/flink-queryable-state-client-java/src/test/resources/log4j-test.properties
new file mode 100644
index 0000000..10792cd
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-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/0c771505/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
deleted file mode 100644
index e60c6f3..0000000
--- a/flink-queryable-state/flink-queryable-state-java/pom.xml
+++ /dev/null
@@ -1,137 +0,0 @@
-<?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>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/0c771505/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
deleted file mode 100644
index fa2604b..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownJobManagerException.java
+++ /dev/null
@@ -1,36 +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.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/0c771505/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
deleted file mode 100644
index c497a72..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKeyOrNamespaceException.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.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/0c771505/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
deleted file mode 100644
index 59ba081..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateIdException.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.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/0c771505/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
deleted file mode 100644
index 0d6588a..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateKeyGroupLocationException.java
+++ /dev/null
@@ -1,41 +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.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.");
-	}
-}


[11/13] flink git commit: [FLINK-7908][QS] Restructure the queryable state module.

Posted by kk...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/serialization/DataOutputSerializer.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/serialization/DataOutputSerializer.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/serialization/DataOutputSerializer.java
new file mode 100644
index 0000000..5811c91
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/serialization/DataOutputSerializer.java
@@ -0,0 +1,344 @@
+/*
+ * 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.state.serialization;
+
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.MemoryUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UTFDataFormatException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+
+/**
+ * A simple and efficient serializer for the {@link java.io.DataOutput} interface.
+ *
+ * <p><b>THIS WAS COPIED FROM RUNTIME SO THAT WE AVOID THE DEPENDENCY.</b>
+ */
+public class DataOutputSerializer implements DataOutputView {
+
+	private static final Logger LOG = LoggerFactory.getLogger(DataOutputSerializer.class);
+
+	private static final int PRUNE_BUFFER_THRESHOLD = 5 * 1024 * 1024;
+
+	// ------------------------------------------------------------------------
+
+	private final byte[] startBuffer;
+
+	private byte[] buffer;
+
+	private int position;
+
+	private ByteBuffer wrapper;
+
+	// ------------------------------------------------------------------------
+
+	public DataOutputSerializer(int startSize) {
+		if (startSize < 1) {
+			throw new IllegalArgumentException();
+		}
+
+		this.startBuffer = new byte[startSize];
+		this.buffer = this.startBuffer;
+		this.wrapper = ByteBuffer.wrap(buffer);
+	}
+
+	public ByteBuffer wrapAsByteBuffer() {
+		this.wrapper.position(0);
+		this.wrapper.limit(this.position);
+		return this.wrapper;
+	}
+
+	public byte[] getByteArray() {
+		return buffer;
+	}
+
+	public byte[] getCopyOfBuffer() {
+		return Arrays.copyOf(buffer, position);
+	}
+
+	public void clear() {
+		this.position = 0;
+	}
+
+	public int length() {
+		return this.position;
+	}
+
+	public void pruneBuffer() {
+		if (this.buffer.length > PRUNE_BUFFER_THRESHOLD) {
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("Releasing serialization buffer of " + this.buffer.length + " bytes.");
+			}
+
+			this.buffer = this.startBuffer;
+			this.wrapper = ByteBuffer.wrap(this.buffer);
+		}
+	}
+
+	@Override
+	public String toString() {
+		return String.format("[pos=%d cap=%d]", this.position, this.buffer.length);
+	}
+
+	// ----------------------------------------------------------------------------------------
+	//                               Data Output
+	// ----------------------------------------------------------------------------------------
+
+	@Override
+	public void write(int b) throws IOException {
+		if (this.position >= this.buffer.length) {
+			resize(1);
+		}
+		this.buffer[this.position++] = (byte) (b & 0xff);
+	}
+
+	@Override
+	public void write(byte[] b) throws IOException {
+		write(b, 0, b.length);
+	}
+
+	@Override
+	public void write(byte[] b, int off, int len) throws IOException {
+		if (len < 0 || off > b.length - len) {
+			throw new ArrayIndexOutOfBoundsException();
+		}
+		if (this.position > this.buffer.length - len) {
+			resize(len);
+		}
+		System.arraycopy(b, off, this.buffer, this.position, len);
+		this.position += len;
+	}
+
+	@Override
+	public void writeBoolean(boolean v) throws IOException {
+		write(v ? 1 : 0);
+	}
+
+	@Override
+	public void writeByte(int v) throws IOException {
+		write(v);
+	}
+
+	@Override
+	public void writeBytes(String s) throws IOException {
+		final int sLen = s.length();
+		if (this.position >= this.buffer.length - sLen) {
+			resize(sLen);
+		}
+
+		for (int i = 0; i < sLen; i++) {
+			writeByte(s.charAt(i));
+		}
+		this.position += sLen;
+	}
+
+	@Override
+	public void writeChar(int v) throws IOException {
+		if (this.position >= this.buffer.length - 1) {
+			resize(2);
+		}
+		this.buffer[this.position++] = (byte) (v >> 8);
+		this.buffer[this.position++] = (byte) v;
+	}
+
+	@Override
+	public void writeChars(String s) throws IOException {
+		final int sLen = s.length();
+		if (this.position >= this.buffer.length - 2 * sLen) {
+			resize(2 * sLen);
+		}
+
+		for (int i = 0; i < sLen; i++) {
+			writeChar(s.charAt(i));
+		}
+	}
+
+	@Override
+	public void writeDouble(double v) throws IOException {
+		writeLong(Double.doubleToLongBits(v));
+	}
+
+	@Override
+	public void writeFloat(float v) throws IOException {
+		writeInt(Float.floatToIntBits(v));
+	}
+
+	@SuppressWarnings("restriction")
+	@Override
+	public void writeInt(int v) throws IOException {
+		if (this.position >= this.buffer.length - 3) {
+			resize(4);
+		}
+		if (LITTLE_ENDIAN) {
+			v = Integer.reverseBytes(v);
+		}
+		UNSAFE.putInt(this.buffer, BASE_OFFSET + this.position, v);
+		this.position += 4;
+	}
+
+	@SuppressWarnings("restriction")
+	@Override
+	public void writeLong(long v) throws IOException {
+		if (this.position >= this.buffer.length - 7) {
+			resize(8);
+		}
+		if (LITTLE_ENDIAN) {
+			v = Long.reverseBytes(v);
+		}
+		UNSAFE.putLong(this.buffer, BASE_OFFSET + this.position, v);
+		this.position += 8;
+	}
+
+	@Override
+	public void writeShort(int v) throws IOException {
+		if (this.position >= this.buffer.length - 1) {
+			resize(2);
+		}
+		this.buffer[this.position++] = (byte) ((v >>> 8) & 0xff);
+		this.buffer[this.position++] = (byte) ((v >>> 0) & 0xff);
+	}
+
+	@Override
+	public void writeUTF(String str) throws IOException {
+		int strlen = str.length();
+		int utflen = 0;
+		int c;
+
+		/* use charAt instead of copying String to char array */
+		for (int i = 0; i < strlen; i++) {
+			c = str.charAt(i);
+			if ((c >= 0x0001) && (c <= 0x007F)) {
+				utflen++;
+			} else if (c > 0x07FF) {
+				utflen += 3;
+			} else {
+				utflen += 2;
+			}
+		}
+
+		if (utflen > 65535) {
+			throw new UTFDataFormatException("Encoded string is too long: " + utflen);
+		}
+		else if (this.position > this.buffer.length - utflen - 2) {
+			resize(utflen + 2);
+		}
+
+		byte[] bytearr = this.buffer;
+		int count = this.position;
+
+		bytearr[count++] = (byte) ((utflen >>> 8) & 0xFF);
+		bytearr[count++] = (byte) ((utflen >>> 0) & 0xFF);
+
+		int i = 0;
+		for (i = 0; i < strlen; i++) {
+			c = str.charAt(i);
+			if (!((c >= 0x0001) && (c <= 0x007F))) {
+				break;
+			}
+			bytearr[count++] = (byte) c;
+		}
+
+		for (; i < strlen; i++) {
+			c = str.charAt(i);
+			if ((c >= 0x0001) && (c <= 0x007F)) {
+				bytearr[count++] = (byte) c;
+
+			} else if (c > 0x07FF) {
+				bytearr[count++] = (byte) (0xE0 | ((c >> 12) & 0x0F));
+				bytearr[count++] = (byte) (0x80 | ((c >> 6) & 0x3F));
+				bytearr[count++] = (byte) (0x80 | ((c >> 0) & 0x3F));
+			} else {
+				bytearr[count++] = (byte) (0xC0 | ((c >> 6) & 0x1F));
+				bytearr[count++] = (byte) (0x80 | ((c >> 0) & 0x3F));
+			}
+		}
+
+		this.position = count;
+	}
+
+	private void resize(int minCapacityAdd) throws IOException {
+		int newLen = Math.max(this.buffer.length * 2, this.buffer.length + minCapacityAdd);
+		byte[] nb;
+		try {
+			nb = new byte[newLen];
+		}
+		catch (NegativeArraySizeException e) {
+			throw new IOException("Serialization failed because the record length would exceed 2GB (max addressable array size in Java).");
+		}
+		catch (OutOfMemoryError e) {
+			// this was too large to allocate, try the smaller size (if possible)
+			if (newLen > this.buffer.length + minCapacityAdd) {
+				newLen = this.buffer.length + minCapacityAdd;
+				try {
+					nb = new byte[newLen];
+				}
+				catch (OutOfMemoryError ee) {
+					// still not possible. give an informative exception message that reports the size
+					throw new IOException("Failed to serialize element. Serialized size (> "
+							+ newLen + " bytes) exceeds JVM heap space", ee);
+				}
+			} else {
+				throw new IOException("Failed to serialize element. Serialized size (> "
+						+ newLen + " bytes) exceeds JVM heap space", e);
+			}
+		}
+
+		System.arraycopy(this.buffer, 0, nb, 0, this.position);
+		this.buffer = nb;
+		this.wrapper = ByteBuffer.wrap(this.buffer);
+	}
+
+	@Override
+	public void skipBytesToWrite(int numBytes) throws IOException {
+		if (buffer.length - this.position < numBytes){
+			throw new EOFException("Could not skip " + numBytes + " bytes.");
+		}
+
+		this.position += numBytes;
+	}
+
+	@Override
+	public void write(DataInputView source, int numBytes) throws IOException {
+		if (buffer.length - this.position < numBytes){
+			throw new EOFException("Could not write " + numBytes + " bytes. Buffer overflow.");
+		}
+
+		source.readFully(this.buffer, this.position, numBytes);
+		this.position += numBytes;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	@SuppressWarnings("restriction")
+	private static final sun.misc.Unsafe UNSAFE = MemoryUtils.UNSAFE;
+
+	@SuppressWarnings("restriction")
+	private static final long BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
+
+	private static final boolean LITTLE_ENDIAN = (MemoryUtils.NATIVE_BYTE_ORDER == ByteOrder.LITTLE_ENDIAN);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/serialization/KvStateSerializer.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/serialization/KvStateSerializer.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/serialization/KvStateSerializer.java
new file mode 100644
index 0000000..4c69483
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/serialization/KvStateSerializer.java
@@ -0,0 +1,265 @@
+/*
+ * 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.state.serialization;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+
+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/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownJobManagerException.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownJobManagerException.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownJobManagerException.java
new file mode 100644
index 0000000..19063c2
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/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.exceptions;
+
+import org.apache.flink.annotation.Internal;
+
+/**
+ * Exception to fail Future if the Task Manager on which the
+ * {@code Client Proxy} 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/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownKeyOrNamespaceException.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownKeyOrNamespaceException.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownKeyOrNamespaceException.java
new file mode 100644
index 0000000..08e3324
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/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.exceptions;
+
+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/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownKvStateIdException.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownKvStateIdException.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownKvStateIdException.java
new file mode 100644
index 0000000..81ea177
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/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.exceptions;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.queryablestate.KvStateID;
+import org.apache.flink.queryablestate.network.BadRequestException;
+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/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownKvStateKeyGroupLocationException.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownKvStateKeyGroupLocationException.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownKvStateKeyGroupLocationException.java
new file mode 100644
index 0000000..d8d34f7
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownKvStateKeyGroupLocationException.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.exceptions;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.queryablestate.network.BadRequestException;
+
+/**
+ * Exception thrown if there is no location information available for the given key group.
+ */
+@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/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequest.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequest.java
new file mode 100644
index 0000000..8169d48
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequest.java
@@ -0,0 +1,140 @@
+/*
+ * 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.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;
+
+/**
+ * The request to be sent by the {@link org.apache.flink.queryablestate.client.QueryableStateClient
+ * Queryable State Client} to the Client Proxy requesting a given state.
+ */
+@Internal
+public class KvStateRequest extends MessageBody {
+
+	private final JobID jobId;
+	private final String stateName;
+	private final int keyHashCode;
+	private final byte[] serializedKeyAndNamespace;
+
+	public KvStateRequest(
+			final JobID jobId,
+			final String stateName,
+			final int keyHashCode,
+			final byte[] serializedKeyAndNamespace) {
+
+		this.jobId = Preconditions.checkNotNull(jobId);
+		this.stateName = Preconditions.checkNotNull(stateName);
+		this.keyHashCode = keyHashCode;
+		this.serializedKeyAndNamespace = Preconditions.checkNotNull(serializedKeyAndNamespace);
+	}
+
+	public JobID getJobId() {
+		return jobId;
+	}
+
+	public String getStateName() {
+		return stateName;
+	}
+
+	public int getKeyHashCode() {
+		return keyHashCode;
+	}
+
+	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{" +
+				"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/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateResponse.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateResponse.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateResponse.java
new file mode 100644
index 0000000..6bf14a7
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateResponse.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.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 {@code State Server} to the {@code 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/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java
new file mode 100644
index 0000000..487020a
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.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.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.util.FlinkRuntimeException;
+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.ChannelFuture;
+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.BindException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+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;
+
+	/** The name of the server, useful for debugging. */
+	private final String serverName;
+
+	/** The {@link InetAddress address} to listen to. */
+	private final InetAddress bindAddress;
+
+	/** A port range on which to try to connect. */
+	private final Set<Integer> bindPortRange;
+
+	/** The number of threads to be allocated to the event loop. */
+	private final int numEventLoopThreads;
+
+	/** The number of threads to be used for query serving. */
+	private final int numQueryThreads;
+
+	/** Netty's ServerBootstrap. */
+	private ServerBootstrap bootstrap;
+
+	/** Query executor thread pool. */
+	private ExecutorService queryExecutor;
+
+	/** Address of this server. */
+	private InetSocketAddress 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()}.
+	 *
+	 * @param serverName the name of the server
+	 * @param bindAddress address to bind to
+	 * @param bindPortIterator port to bind to
+	 * @param numEventLoopThreads number of event loop threads
+	 */
+	protected AbstractServerBase(
+			final String serverName,
+			final InetAddress bindAddress,
+			final Iterator<Integer> bindPortIterator,
+			final Integer numEventLoopThreads,
+			final Integer numQueryThreads) {
+
+		Preconditions.checkNotNull(bindPortIterator);
+		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.bindAddress = Preconditions.checkNotNull(bindAddress);
+		this.numEventLoopThreads = numEventLoopThreads;
+		this.numQueryThreads = numQueryThreads;
+
+		this.bindPortRange = new HashSet<>();
+		while (bindPortIterator.hasNext()) {
+			int port = bindPortIterator.next();
+			Preconditions.checkArgument(port >= 0 && port <= 65535,
+					"Invalid port configuration. Port must be between 0 and 65535, but was " + port + ".");
+			bindPortRange.add(port);
+		}
+	}
+
+	/**
+	 * Creates a thread pool for the query execution.
+	 * @return Thread pool for query execution
+	 */
+	private ExecutorService createQueryExecutor() {
+		ThreadFactory threadFactory = new ThreadFactoryBuilder()
+				.setDaemon(true)
+				.setNameFormat("Flink " + getServerName() + " Thread %d")
+				.build();
+		return Executors.newFixedThreadPool(numQueryThreads, threadFactory);
+	}
+
+	/**
+	 * Returns the thread-pool responsible for processing incoming requests.
+	 */
+	protected ExecutorService getQueryExecutor() {
+		return queryExecutor;
+	}
+
+	/**
+	 * Gets the name of the server. This is useful for debugging.
+	 * @return The name of the server.
+	 */
+	public String getServerName() {
+		return serverName;
+	}
+
+	/**
+	 * Returns the {@link AbstractServerHandler handler} to be used for
+	 * serving the incoming requests.
+	 */
+	public abstract AbstractServerHandler<REQ, RESP> initializeHandler();
+
+	/**
+	 * Returns the address of this server.
+	 *
+	 * @return AbstractServerBase address
+	 * @throws IllegalStateException If server has not been started yet
+	 */
+	public InetSocketAddress getServerAddress() {
+		Preconditions.checkState(serverAddress != null, "Server " + serverName + " has not been started.");
+		return serverAddress;
+	}
+
+	/**
+	 * Starts the server by binding to the configured bind address (blocking).
+	 * @throws Exception If something goes wrong during the bind operation.
+	 */
+	public void start() throws Throwable {
+		Preconditions.checkState(serverAddress == null,
+				"Server " + serverName + " already running @ " + serverAddress + '.');
+
+		Iterator<Integer> portIterator = bindPortRange.iterator();
+		while (portIterator.hasNext() && !attemptToBind(portIterator.next())) {}
+
+		if (serverAddress != null) {
+			LOG.info("Started server {} @ {}.", serverName, serverAddress);
+		} else {
+			LOG.info("Unable to start server {}. All ports in provided range are occupied.", serverName);
+			throw new FlinkRuntimeException("Unable to start server " + serverName + ". All ports in provided range are occupied.");
+		}
+	}
+
+	/**
+	 * Tries to start the server at the provided port.
+	 *
+	 * <p>This, in conjunction with {@link #start()}, try to start the
+	 * server on a free port among the port range provided at the constructor.
+	 *
+	 * @param port the port to try to bind the server to.
+	 * @throws Exception If something goes wrong during the bind operation.
+	 */
+	private boolean attemptToBind(final int port) throws Throwable {
+		LOG.debug("Attempting to start server {} on port {}.", serverName, port);
+
+		this.queryExecutor = createQueryExecutor();
+		this.handler = initializeHandler();
+
+		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);
+
+		this.bootstrap = new ServerBootstrap()
+				.localAddress(bindAddress, port)
+				.group(nioGroup)
+				.channel(NioServerSocketChannel.class)
+				.option(ChannelOption.ALLOCATOR, bufferPool)
+				.childOption(ChannelOption.ALLOCATOR, bufferPool)
+				.childOption(ChannelOption.WRITE_BUFFER_HIGH_WATER_MARK, HIGH_WATER_MARK)
+				.childOption(ChannelOption.WRITE_BUFFER_LOW_WATER_MARK, LOW_WATER_MARK)
+				.childHandler(new ServerChannelInitializer<>(handler));
+
+		try {
+			final ChannelFuture future = bootstrap.bind().sync();
+			if (future.isSuccess()) {
+				final InetSocketAddress localAddress = (InetSocketAddress) future.channel().localAddress();
+				serverAddress = new InetSocketAddress(localAddress.getAddress(), localAddress.getPort());
+				return true;
+			}
+
+			// the following throw is to bypass Netty's "optimization magic"
+			// and catch the bind exception.
+			// the exception is thrown by the sync() call above.
+
+			throw future.cause();
+		} catch (BindException e) {
+			LOG.debug("Failed to start server {} on port {}: {}.", serverName, port, e.getMessage());
+			shutdown();
+		}
+		// any other type of exception we let it bubble up.
+		return false;
+	}
+
+	/**
+	 * Shuts down the server and all related thread pools.
+	 */
+	public void shutdown() {
+		LOG.info("Shutting down server {} @ {}", serverName, serverAddress);
+
+		if (handler != null) {
+			handler.shutdown();
+			handler = null;
+		}
+
+		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/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerHandler.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerHandler.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerHandler.java
new file mode 100644
index 0000000..9e02291
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerHandler.java
@@ -0,0 +1,305 @@
+/*
+ * 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.stats.KvStateRequestStats;
+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.
+				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 state 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/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/BadRequestException.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/BadRequestException.java b/flink-queryable-state/flink-queryable-state-client-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-client-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/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/ChunkedByteBuf.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/ChunkedByteBuf.java b/flink-queryable-state/flink-queryable-state-client-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-client-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 +
+				'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/Client.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/Client.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/Client.java
new file mode 100644
index 0000000..13d34fb
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/Client.java
@@ -0,0 +1,536 @@
+/*
+ * 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.FutureUtils;
+import org.apache.flink.queryablestate.network.messages.MessageBody;
+import org.apache.flink.queryablestate.network.messages.MessageSerializer;
+import org.apache.flink.queryablestate.network.stats.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.net.InetSocketAddress;
+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<InetSocketAddress, EstablishedConnection> establishedConnections = new ConcurrentHashMap<>();
+
+	/** Pending connections. */
+	private final Map<InetSocketAddress, 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 InetSocketAddress 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.getAddress(), 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<InetSocketAddress, EstablishedConnection> conn : establishedConnections.entrySet()) {
+				if (establishedConnections.remove(conn.getKey(), conn.getValue())) {
+					conn.getValue().close();
+				}
+			}
+
+			for (Map.Entry<InetSocketAddress, 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 InetSocketAddress 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 InetSocketAddress 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 InetSocketAddress 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 InetSocketAddress 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;
+			}
+		}
+	}
+}


[09/13] flink git commit: [FLINK-7908][QS] Restructure the queryable state module.

Posted by kk...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/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
deleted file mode 100644
index 70bccf0..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java
+++ /dev/null
@@ -1,260 +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.annotation.PublicEvolving;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.state.State;
-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.queryablestate.client.state.ImmutableStateBinder;
-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.netty.DisabledKvStateRequestStats;
-import org.apache.flink.runtime.query.netty.message.KvStateSerializer;
-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 org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.concurrent.CompletableFuture;
-
-/**
- * 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 Task Managers and the location will be reported to the Job Manager.
- *
- * <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);
-
-	/** The client that forwards the requests to the proxy. */
-	private final Client<KvStateRequest, KvStateResponse> client;
-
-	/** 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;
-
-	/**
-	 * 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(final String remoteHostname, final int remotePort) throws UnknownHostException {
-		this(InetAddress.getByName(Preconditions.checkNotNull(remoteHostname)), remotePort);
-	}
-
-	/**
-	 * 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.
-	 */
-	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).");
-
-		this.remoteAddress = new KvStateServerAddress(remoteAddress, remotePort);
-
-		final MessageSerializer<KvStateRequest, KvStateResponse> messageSerializer =
-				new MessageSerializer<>(
-						new KvStateRequest.KvStateRequestDeserializer(),
-						new KvStateResponse.KvStateResponseDeserializer());
-
-		this.client = new Client<>(
-				"Queryable State Client",
-				Hardware.getNumberCPUCores(),
-				messageSerializer,
-				new DisabledKvStateRequestStats());
-	}
-
-	/** Shuts down the client. */
-	public void shutdown() {
-		client.shutdown();
-	}
-
-	/**
-	 * Gets the {@link ExecutionConfig}.
-	 */
-	public ExecutionConfig getExecutionConfig() {
-		return executionConfig;
-	}
-
-	/**
-	 * 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.	 *
-	 * @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 immutable {@link State} object containing the result.
-	 */
-	@PublicEvolving
-	public <K, S extends State, V> CompletableFuture<S> getKvState(
-			final JobID jobId,
-			final String queryableStateName,
-			final K key,
-			final TypeHint<K> keyTypeHint,
-			final StateDescriptor<S, V> stateDescriptor) {
-
-		Preconditions.checkNotNull(keyTypeHint);
-
-		TypeInformation<K> keyTypeInfo = keyTypeHint.getTypeInfo();
-		return getKvState(jobId, queryableStateName, key, keyTypeInfo, stateDescriptor);
-	}
-
-	/**
-	 * 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.
-	 * @param keyTypeInfo				The {@link TypeInformation} of the key.
-	 * @param stateDescriptor			The {@link StateDescriptor} of the state we want to query.
-	 * @return Future holding the immutable {@link State} object containing the result.
-	 */
-	@PublicEvolving
-	public <K, S extends State, V> CompletableFuture<S> getKvState(
-			final JobID jobId,
-			final String queryableStateName,
-			final K key,
-			final TypeInformation<K> keyTypeInfo,
-			final StateDescriptor<S, V> stateDescriptor) {
-
-		return getKvState(jobId, queryableStateName, key, VoidNamespace.INSTANCE,
-				keyTypeInfo, VoidNamespaceTypeInfo.INSTANCE, stateDescriptor);
-	}
-
-	/**
-	 * 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 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 immutable {@link State} object containing the result.
-	 */
-	@PublicEvolving
-	public <K, N, S extends State, V> CompletableFuture<S> getKvState(
-			final JobID jobId,
-			final String queryableStateName,
-			final K key,
-			final N namespace,
-			final TypeInformation<K> keyTypeInfo,
-			final TypeInformation<N> namespaceTypeInfo,
-			final StateDescriptor<S, V> stateDescriptor) {
-
-		Preconditions.checkNotNull(jobId);
-		Preconditions.checkNotNull(queryableStateName);
-		Preconditions.checkNotNull(key);
-		Preconditions.checkNotNull(namespace);
-
-		Preconditions.checkNotNull(keyTypeInfo);
-		Preconditions.checkNotNull(namespaceTypeInfo);
-		Preconditions.checkNotNull(stateDescriptor);
-
-		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 FutureUtils.getFailedFuture(e);
-		}
-
-		return getKvState(jobId, queryableStateName, key.hashCode(), serializedKeyAndNamespace).thenApply(
-				stateResponse -> {
-					try {
-						return stateDescriptor.bind(new ImmutableStateBinder(stateResponse.getContent()));
-					} catch (Exception e) {
-						throw new FlinkRuntimeException(e);
-					}
-				});
-	}
-
-	/**
-	 * 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
-	 * @return Future holding the serialized result
-	 */
-	private CompletableFuture<KvStateResponse> getKvState(
-			final JobID jobId,
-			final String queryableStateName,
-			final int keyHashCode,
-			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/0c771505/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
deleted file mode 100644
index d7191b6..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyHandler.java
+++ /dev/null
@@ -1,225 +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.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/0c771505/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
deleted file mode 100644
index 196641d..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java
+++ /dev/null
@@ -1,128 +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.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.Iterator;
-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, Iterator, int, int, KvStateRequestStats)
-	 * QueryableStateUtils.createKvStateClientProxy(InetAddress, Iterator, 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 bindPortIterator the port range to try to bind 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 Iterator<Integer> bindPortIterator,
-			final Integer numEventLoopThreads,
-			final Integer numQueryThreads,
-			final KvStateRequestStats stats) {
-
-		super("Queryable State Proxy Server", bindAddress, bindPortIterator, 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 Throwable {
-		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/0c771505/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableAggregatingState.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableAggregatingState.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableAggregatingState.java
deleted file mode 100644
index b853cfc..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableAggregatingState.java
+++ /dev/null
@@ -1,71 +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.state;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.api.common.state.AggregatingState;
-import org.apache.flink.api.common.state.AggregatingStateDescriptor;
-import org.apache.flink.runtime.query.netty.message.KvStateSerializer;
-import org.apache.flink.util.Preconditions;
-
-import java.io.IOException;
-
-/**
- * A read-only {@link AggregatingState} that <b>does not</b> allow for modifications.
- *
- * <p>This is the type of the result returned when querying Flink's keyed state using the
- * {@link org.apache.flink.queryablestate.client.QueryableStateClient Queryable State Client} and
- * providing an {@link AggregatingStateDescriptor}.
- */
-@PublicEvolving
-public final class ImmutableAggregatingState<IN, OUT> extends ImmutableState implements AggregatingState<IN, OUT> {
-
-	private final OUT value;
-
-	private ImmutableAggregatingState(OUT value) {
-		this.value = Preconditions.checkNotNull(value);
-	}
-
-	@Override
-	public OUT get() {
-		return value;
-	}
-
-	@Override
-	public void add(Object newValue) {
-		throw MODIFICATION_ATTEMPT_ERROR;
-	}
-
-	@Override
-	public void clear() {
-		throw MODIFICATION_ATTEMPT_ERROR;
-	}
-
-	public static <IN, ACC, OUT> ImmutableAggregatingState<IN, OUT> createState(
-			final AggregatingStateDescriptor<IN, ACC, OUT> stateDescriptor,
-			final byte[] serializedValue) throws IOException {
-
-		final ACC accumulator = KvStateSerializer.deserializeValue(
-				serializedValue,
-				stateDescriptor.getSerializer());
-
-		final OUT state = stateDescriptor.getAggregateFunction().getResult(accumulator);
-		return new ImmutableAggregatingState<>(state);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableFoldingState.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableFoldingState.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableFoldingState.java
deleted file mode 100644
index a12adaa..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableFoldingState.java
+++ /dev/null
@@ -1,70 +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.state;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.api.common.state.FoldingState;
-import org.apache.flink.api.common.state.FoldingStateDescriptor;
-import org.apache.flink.runtime.query.netty.message.KvStateSerializer;
-import org.apache.flink.util.Preconditions;
-
-import java.io.IOException;
-
-/**
- * A read-only {@link FoldingState} that does not allow for modifications.
- *
- * <p>This is the result returned when querying Flink's keyed state using the
- * {@link org.apache.flink.queryablestate.client.QueryableStateClient Queryable State Client} and
- * providing an {@link FoldingStateDescriptor}.
- */
-@PublicEvolving
-@Deprecated
-public final class ImmutableFoldingState<IN, ACC> extends ImmutableState implements FoldingState<IN, ACC> {
-
-	private final ACC value;
-
-	private ImmutableFoldingState(ACC value) {
-		this.value = Preconditions.checkNotNull(value);
-	}
-
-	@Override
-	public ACC get() {
-		return value;
-	}
-
-	@Override
-	public void add(Object newValue) {
-		throw MODIFICATION_ATTEMPT_ERROR;
-	}
-
-	@Override
-	public void clear() {
-		throw MODIFICATION_ATTEMPT_ERROR;
-	}
-
-	public static <IN, ACC> ImmutableFoldingState<IN, ACC> createState(
-			final FoldingStateDescriptor<IN, ACC> stateDescriptor,
-			final byte[] serializedState) throws IOException {
-
-		final ACC state = KvStateSerializer.deserializeValue(
-				serializedState,
-				stateDescriptor.getSerializer());
-		return new ImmutableFoldingState<>(state);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableListState.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableListState.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableListState.java
deleted file mode 100644
index 8416905..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableListState.java
+++ /dev/null
@@ -1,70 +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.state;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.api.common.state.ListState;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.runtime.query.netty.message.KvStateSerializer;
-import org.apache.flink.util.Preconditions;
-
-import java.io.IOException;
-import java.util.List;
-
-/**
- * A read-only {@link ListState} that does not allow for modifications.
- *
- * <p>This is the result returned when querying Flink's keyed state using the
- * {@link org.apache.flink.queryablestate.client.QueryableStateClient Queryable State Client} and
- * providing an {@link ListStateDescriptor}.
- */
-@PublicEvolving
-public final class ImmutableListState<V> extends ImmutableState implements ListState<V> {
-
-	private final List<V> listState;
-
-	private ImmutableListState(final List<V> state) {
-		this.listState = Preconditions.checkNotNull(state);
-	}
-
-	@Override
-	public Iterable<V> get() {
-		return listState;
-	}
-
-	@Override
-	public void add(V value) {
-		throw MODIFICATION_ATTEMPT_ERROR;
-	}
-
-	@Override
-	public void clear() {
-		throw MODIFICATION_ATTEMPT_ERROR;
-	}
-
-	public static <V> ImmutableListState<V> createState(
-			final ListStateDescriptor<V> stateDescriptor,
-			final byte[] serializedState) throws IOException {
-
-		final List<V> state = KvStateSerializer.deserializeList(
-				serializedState,
-				stateDescriptor.getElementSerializer());
-		return new ImmutableListState<>(state);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableMapState.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableMapState.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableMapState.java
deleted file mode 100644
index c216d5d..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableMapState.java
+++ /dev/null
@@ -1,139 +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.state;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.api.common.state.MapState;
-import org.apache.flink.api.common.state.MapStateDescriptor;
-import org.apache.flink.runtime.query.netty.message.KvStateSerializer;
-import org.apache.flink.util.Preconditions;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * A read-only {@link MapState} that does not allow for modifications.
- *
- * <p>This is the result returned when querying Flink's keyed state using the
- * {@link org.apache.flink.queryablestate.client.QueryableStateClient Queryable State Client} and
- * providing an {@link MapStateDescriptor}.
- */
-@PublicEvolving
-public final class ImmutableMapState<K, V> extends ImmutableState implements MapState<K, V> {
-
-	private final Map<K, V> state;
-
-	private ImmutableMapState(final Map<K, V> mapState) {
-		this.state = Preconditions.checkNotNull(mapState);
-	}
-
-	@Override
-	public V get(K key) {
-		return state.get(key);
-	}
-
-	@Override
-	public void put(K key, V value) {
-		throw MODIFICATION_ATTEMPT_ERROR;
-	}
-
-	@Override
-	public void putAll(Map<K, V> map) {
-		throw MODIFICATION_ATTEMPT_ERROR;
-	}
-
-	@Override
-	public void remove(K key) {
-		throw MODIFICATION_ATTEMPT_ERROR;
-	}
-
-	@Override
-	public boolean contains(K key) {
-		return state.containsKey(key);
-	}
-
-	/**
-	 * Returns all the mappings in the state in a {@link Collections#unmodifiableSet(Set)}.
-	 *
-	 * @return A read-only iterable view of all the key-value pairs in the state.
-	 *
-	 * @throws Exception Thrown if the system cannot access the state.
-	 */
-	@Override
-	public Iterable<Map.Entry<K, V>> entries() {
-		return Collections.unmodifiableSet(state.entrySet());
-	}
-
-	/**
-	 * Returns all the keys in the state in a {@link Collections#unmodifiableSet(Set)}.
-	 *
-	 * @return A read-only iterable view of all the keys in the state.
-	 *
-	 * @throws Exception Thrown if the system cannot access the state.
-	 */
-	@Override
-	public Iterable<K> keys() {
-		return Collections.unmodifiableSet(state.keySet());
-	}
-
-	/**
-	 * Returns all the values in the state in a {@link Collections#unmodifiableCollection(Collection)}.
-	 *
-	 * @return A read-only iterable view of all the values in the state.
-	 *
-	 * @throws Exception Thrown if the system cannot access the state.
-	 */
-	@Override
-	public Iterable<V> values() {
-		return Collections.unmodifiableCollection(state.values());
-	}
-
-	/**
-	 * Iterates over all the mappings in the state. The iterator cannot
-	 * remove elements.
-	 *
-	 * @return A read-only iterator over all the mappings in the state
-	 *
-	 * @throws Exception Thrown if the system cannot access the state.
-	 */
-	@Override
-	public Iterator<Map.Entry<K, V>> iterator() {
-		return Collections.unmodifiableSet(state.entrySet()).iterator();
-	}
-
-	@Override
-	public void clear() {
-		throw MODIFICATION_ATTEMPT_ERROR;
-	}
-
-	public static <K, V> ImmutableMapState<K, V> createState(
-			final MapStateDescriptor<K, V> stateDescriptor,
-			final byte[] serializedState) throws IOException {
-
-		final Map<K, V> state = KvStateSerializer.deserializeMap(
-				serializedState,
-				stateDescriptor.getKeySerializer(),
-				stateDescriptor.getValueSerializer());
-		return new ImmutableMapState<>(state);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableReducingState.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableReducingState.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableReducingState.java
deleted file mode 100644
index da08c53..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableReducingState.java
+++ /dev/null
@@ -1,69 +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.state;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.api.common.state.ReducingState;
-import org.apache.flink.api.common.state.ReducingStateDescriptor;
-import org.apache.flink.runtime.query.netty.message.KvStateSerializer;
-import org.apache.flink.util.Preconditions;
-
-import java.io.IOException;
-
-/**
- * A read-only {@link ReducingState} that does not allow for modifications.
- *
- * <p>This is the result returned when querying Flink's keyed state using the
- * {@link org.apache.flink.queryablestate.client.QueryableStateClient Queryable State Client} and
- * providing an {@link ReducingStateDescriptor}.
- */
-@PublicEvolving
-public final class ImmutableReducingState<V> extends ImmutableState implements ReducingState<V> {
-
-	private final V value;
-
-	private ImmutableReducingState(V value) {
-		this.value = Preconditions.checkNotNull(value);
-	}
-
-	@Override
-	public V get() {
-		return value;
-	}
-
-	@Override
-	public void add(V newValue) {
-		throw MODIFICATION_ATTEMPT_ERROR;
-	}
-
-	@Override
-	public void clear() {
-		throw MODIFICATION_ATTEMPT_ERROR;
-	}
-
-	public static <V> ImmutableReducingState<V> createState(
-			final ReducingStateDescriptor<V> stateDescriptor,
-			final byte[] serializedState) throws IOException {
-
-		final V state = KvStateSerializer.deserializeValue(
-				serializedState,
-				stateDescriptor.getSerializer());
-		return new ImmutableReducingState<>(state);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableState.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableState.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableState.java
deleted file mode 100644
index 863f07b..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableState.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.queryablestate.client.state;
-
-/**
- * A base class for the <b>read-only</b> types of state returned
- * as results from the Queryable State Client.
- */
-abstract class ImmutableState {
-
-	protected static final UnsupportedOperationException MODIFICATION_ATTEMPT_ERROR =
-			new UnsupportedOperationException("State is read-only. No modifications allowed.");
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableStateBinder.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableStateBinder.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableStateBinder.java
deleted file mode 100644
index 6ce2787..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableStateBinder.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.queryablestate.client.state;
-
-import org.apache.flink.api.common.state.AggregatingState;
-import org.apache.flink.api.common.state.AggregatingStateDescriptor;
-import org.apache.flink.api.common.state.FoldingState;
-import org.apache.flink.api.common.state.FoldingStateDescriptor;
-import org.apache.flink.api.common.state.ListState;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.state.MapState;
-import org.apache.flink.api.common.state.MapStateDescriptor;
-import org.apache.flink.api.common.state.ReducingState;
-import org.apache.flink.api.common.state.ReducingStateDescriptor;
-import org.apache.flink.api.common.state.StateBinder;
-import org.apache.flink.api.common.state.ValueState;
-import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.util.Preconditions;
-
-/**
- * A {@link StateBinder} used to deserialize the results returned by the
- * {@link org.apache.flink.queryablestate.client.QueryableStateClient}.
- *
- * <p>The result is an immutable {@link org.apache.flink.api.common.state.State State}
- * object containing the requested result.
- */
-public class ImmutableStateBinder implements StateBinder {
-
-	private final byte[] serializedState;
-
-	public ImmutableStateBinder(final byte[] content) {
-		serializedState = Preconditions.checkNotNull(content);
-	}
-
-	@Override
-	public <T> ValueState<T> createValueState(ValueStateDescriptor<T> stateDesc) throws Exception {
-		return ImmutableValueState.createState(stateDesc, serializedState);
-	}
-
-	@Override
-	public <T> ListState<T> createListState(ListStateDescriptor<T> stateDesc) throws Exception {
-		return ImmutableListState.createState(stateDesc, serializedState);
-	}
-
-	@Override
-	public <T> ReducingState<T> createReducingState(ReducingStateDescriptor<T> stateDesc) throws Exception {
-		return ImmutableReducingState.createState(stateDesc, serializedState);
-	}
-
-	@Override
-	public <IN, ACC, OUT> AggregatingState<IN, OUT> createAggregatingState(AggregatingStateDescriptor<IN, ACC, OUT> stateDesc) throws Exception {
-		return ImmutableAggregatingState.createState(stateDesc, serializedState);
-	}
-
-	@Override
-	public <T, ACC> FoldingState<T, ACC> createFoldingState(FoldingStateDescriptor<T, ACC> stateDesc) throws Exception {
-		return ImmutableFoldingState.createState(stateDesc, serializedState);
-	}
-
-	@Override
-	public <MK, MV> MapState<MK, MV> createMapState(MapStateDescriptor<MK, MV> stateDesc) throws Exception {
-		return ImmutableMapState.createState(stateDesc, serializedState);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableValueState.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableValueState.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableValueState.java
deleted file mode 100644
index 7fd6457..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableValueState.java
+++ /dev/null
@@ -1,69 +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.state;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.api.common.state.ValueState;
-import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.runtime.query.netty.message.KvStateSerializer;
-import org.apache.flink.util.Preconditions;
-
-import java.io.IOException;
-
-/**
- * A read-only {@link ValueState} that does not allow for modifications.
- *
- * <p>This is the result returned when querying Flink's keyed state using the
- * {@link org.apache.flink.queryablestate.client.QueryableStateClient Queryable State Client} and
- * providing an {@link ValueStateDescriptor}.
- */
-@PublicEvolving
-public final class ImmutableValueState<V> extends ImmutableState implements ValueState<V> {
-
-	private final V value;
-
-	private ImmutableValueState(V value) {
-		this.value = Preconditions.checkNotNull(value);
-	}
-
-	@Override
-	public V value() {
-		return value;
-	}
-
-	@Override
-	public void update(V newValue) {
-		throw MODIFICATION_ATTEMPT_ERROR;
-	}
-
-	@Override
-	public void clear() {
-		throw MODIFICATION_ATTEMPT_ERROR;
-	}
-
-	public static <V> ImmutableValueState<V> createState(
-			final ValueStateDescriptor<V> stateDescriptor,
-			final byte[] serializedState) throws IOException {
-
-		final V state = KvStateSerializer.deserializeValue(
-				serializedState,
-				stateDescriptor.getSerializer());
-		return new ImmutableValueState<>(state);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/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
deleted file mode 100644
index eedc2a1..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateInternalRequest.java
+++ /dev/null
@@ -1,93 +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.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/0c771505/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
deleted file mode 100644
index 7eb39c7..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequest.java
+++ /dev/null
@@ -1,141 +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.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;
-
-/**
- * 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.
- */
-@Internal
-public class KvStateRequest extends MessageBody {
-
-	private final JobID jobId;
-	private final String stateName;
-	private final int keyHashCode;
-	private final byte[] serializedKeyAndNamespace;
-
-	public KvStateRequest(
-			final JobID jobId,
-			final String stateName,
-			final int keyHashCode,
-			final byte[] serializedKeyAndNamespace) {
-
-		this.jobId = Preconditions.checkNotNull(jobId);
-		this.stateName = Preconditions.checkNotNull(stateName);
-		this.keyHashCode = keyHashCode;
-		this.serializedKeyAndNamespace = Preconditions.checkNotNull(serializedKeyAndNamespace);
-	}
-
-	public JobID getJobId() {
-		return jobId;
-	}
-
-	public String getStateName() {
-		return stateName;
-	}
-
-	public int getKeyHashCode() {
-		return keyHashCode;
-	}
-
-	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{" +
-				"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/0c771505/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
deleted file mode 100644
index 462135f..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateResponse.java
+++ /dev/null
@@ -1,75 +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.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/0c771505/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
deleted file mode 100644
index be852fb..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java
+++ /dev/null
@@ -1,310 +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.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.FlinkRuntimeException;
-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.ChannelFuture;
-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.BindException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Set;
-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;
-
-	/** The name of the server, useful for debugging. */
-	private final String serverName;
-
-	/** The {@link InetAddress address} to listen to. */
-	private final InetAddress bindAddress;
-
-	/** A port range on which to try to connect. */
-	private final Set<Integer> bindPortRange;
-
-	/** The number of threads to be allocated to the event loop. */
-	private final int numEventLoopThreads;
-
-	/** The number of threads to be used for query serving. */
-	private final int numQueryThreads;
-
-	/** Netty's ServerBootstrap. */
-	private ServerBootstrap bootstrap;
-
-	/** Query executor thread pool. */
-	private 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()}.
-	 *
-	 * @param serverName the name of the server
-	 * @param bindAddress address to bind to
-	 * @param bindPortIterator port to bind to
-	 * @param numEventLoopThreads number of event loop threads
-	 */
-	protected AbstractServerBase(
-			final String serverName,
-			final InetAddress bindAddress,
-			final Iterator<Integer> bindPortIterator,
-			final Integer numEventLoopThreads,
-			final Integer numQueryThreads) {
-
-		Preconditions.checkNotNull(bindPortIterator);
-		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.bindAddress = Preconditions.checkNotNull(bindAddress);
-		this.numEventLoopThreads = numEventLoopThreads;
-		this.numQueryThreads = numQueryThreads;
-
-		this.bindPortRange = new HashSet<>();
-		while (bindPortIterator.hasNext()) {
-			int port = bindPortIterator.next();
-			Preconditions.checkArgument(port >= 0 && port <= 65535,
-					"Invalid port configuration. Port must be between 0 and 65535, but was " + port + ".");
-			bindPortRange.add(port);
-		}
-	}
-
-	/**
-	 * Creates a thread pool for the query execution.
-	 * @return Thread pool for query execution
-	 */
-	private ExecutorService createQueryExecutor() {
-		ThreadFactory threadFactory = new ThreadFactoryBuilder()
-				.setDaemon(true)
-				.setNameFormat("Flink " + getServerName() + " Thread %d")
-				.build();
-		return Executors.newFixedThreadPool(numQueryThreads, threadFactory);
-	}
-
-	/**
-	 * Returns the thread-pool responsible for processing incoming requests.
-	 */
-	protected ExecutorService getQueryExecutor() {
-		return queryExecutor;
-	}
-
-	/**
-	 * Gets the name of the server. This is useful for debugging.
-	 * @return The name of the server.
-	 */
-	public String getServerName() {
-		return serverName;
-	}
-
-	/**
-	 * Returns the {@link AbstractServerHandler handler} to be used for
-	 * serving the incoming requests.
-	 */
-	public abstract AbstractServerHandler<REQ, RESP> initializeHandler();
-
-	/**
-	 * 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;
-	}
-
-	/**
-	 * Starts the server by binding to the configured bind address (blocking).
-	 * @throws Exception If something goes wrong during the bind operation.
-	 */
-	public void start() throws Throwable {
-		Preconditions.checkState(serverAddress == null,
-				"Server " + serverName + " already running @ " + serverAddress + '.');
-
-		Iterator<Integer> portIterator = bindPortRange.iterator();
-		while (portIterator.hasNext() && !attemptToBind(portIterator.next())) {}
-
-		if (serverAddress != null) {
-			LOG.info("Started server {} @ {}.", serverName, serverAddress);
-		} else {
-			LOG.info("Unable to start server {}. All ports in provided range are occupied.", serverName);
-			throw new FlinkRuntimeException("Unable to start server " + serverName + ". All ports in provided range are occupied.");
-		}
-	}
-
-	/**
-	 * Tries to start the server at the provided port.
-	 *
-	 * <p>This, in conjunction with {@link #start()}, try to start the
-	 * server on a free port among the port range provided at the constructor.
-	 *
-	 * @param port the port to try to bind the server to.
-	 * @throws Exception If something goes wrong during the bind operation.
-	 */
-	private boolean attemptToBind(final int port) throws Throwable {
-		LOG.debug("Attempting to start server {} on port {}.", serverName, port);
-
-		this.queryExecutor = createQueryExecutor();
-		this.handler = initializeHandler();
-
-		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);
-
-		this.bootstrap = new ServerBootstrap()
-				.localAddress(bindAddress, port)
-				.group(nioGroup)
-				.channel(NioServerSocketChannel.class)
-				.option(ChannelOption.ALLOCATOR, bufferPool)
-				.childOption(ChannelOption.ALLOCATOR, bufferPool)
-				.childOption(ChannelOption.WRITE_BUFFER_HIGH_WATER_MARK, HIGH_WATER_MARK)
-				.childOption(ChannelOption.WRITE_BUFFER_LOW_WATER_MARK, LOW_WATER_MARK)
-				.childHandler(new ServerChannelInitializer<>(handler));
-
-		try {
-			final ChannelFuture future = bootstrap.bind().sync();
-			if (future.isSuccess()) {
-				final InetSocketAddress localAddress = (InetSocketAddress) future.channel().localAddress();
-				serverAddress = new KvStateServerAddress(localAddress.getAddress(), localAddress.getPort());
-				return true;
-			}
-
-			// the following throw is to bypass Netty's "optimization magic"
-			// and catch the bind exception.
-			// the exception is thrown by the sync() call above.
-
-			throw future.cause();
-		} catch (BindException e) {
-			LOG.debug("Failed to start server {} on port {}: {}.", serverName, port, e.getMessage());
-			shutdown();
-		}
-		// any other type of exception we let it bubble up.
-		return false;
-	}
-
-	/**
-	 * Shuts down the server and all related thread pools.
-	 */
-	public void shutdown() {
-		LOG.info("Shutting down server {} @ {}", serverName, serverAddress);
-
-		if (handler != null) {
-			handler.shutdown();
-			handler = null;
-		}
-
-		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();
-	}
-}


[13/13] flink git commit: [FLINK-7824][QS] Put the QS modules in the opt folder.

Posted by kk...@apache.org.
[FLINK-7824][QS] Put the QS modules in the opt folder.

Now the user can find the jars in the opt/ folder and
he can activate QS by putting the core jar in the lib/
folder and program against the client jar.


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

Branch: refs/heads/master
Commit: 2fd8721d0375bfa60a190bb206a65287c167a43a
Parents: 0c77150
Author: kkloudas <kk...@gmail.com>
Authored: Thu Oct 26 15:07:47 2017 +0200
Committer: kkloudas <kk...@gmail.com>
Committed: Thu Oct 26 18:57:35 2017 +0200

----------------------------------------------------------------------
 flink-dist/src/main/assemblies/opt.xml | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/2fd8721d/flink-dist/src/main/assemblies/opt.xml
----------------------------------------------------------------------
diff --git a/flink-dist/src/main/assemblies/opt.xml b/flink-dist/src/main/assemblies/opt.xml
index 58aee3d..14ec174 100644
--- a/flink-dist/src/main/assemblies/opt.xml
+++ b/flink-dist/src/main/assemblies/opt.xml
@@ -138,5 +138,13 @@
 			<destName>flink-s3-fs-presto-${project.version}.jar</destName>
 			<fileMode>0644</fileMode>
 		</file>
+
+		<!-- Queryable State -->
+		<file>
+			<source>../flink-queryable-state/flink-queryable-state-runtime/target/flink-queryable-state-runtime_${scala.binary.version}-${project.version}.jar</source>
+			<outputDirectory>opt/</outputDirectory>
+			<destName>flink-queryable-state-runtime_${scala.binary.version}-${project.version}.jar</destName>
+			<fileMode>0644</fileMode>
+		</file>
 	</files>
 </assembly>


[08/13] flink git commit: [FLINK-7908][QS] Restructure the queryable state module.

Posted by kk...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/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
deleted file mode 100644
index 18a88da..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerHandler.java
+++ /dev/null
@@ -1,306 +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.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.
-				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/0c771505/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
deleted file mode 100644
index 3c0c484..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/BadRequestException.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.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/0c771505/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
deleted file mode 100644
index 9c56025..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/ChunkedByteBuf.java
+++ /dev/null
@@ -1,100 +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.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 +
-				'}';
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/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
deleted file mode 100644
index e6d59de..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/Client.java
+++ /dev/null
@@ -1,537 +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.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/0c771505/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
deleted file mode 100644
index fc9b1d4..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/ClientHandler.java
+++ /dev/null
@@ -1,122 +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.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/0c771505/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
deleted file mode 100644
index 00ce1ed..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/ClientHandlerCallback.java
+++ /dev/null
@@ -1,56 +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.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/0c771505/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
deleted file mode 100644
index f26c267..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageBody.java
+++ /dev/null
@@ -1,38 +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.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/0c771505/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
deleted file mode 100644
index 436fb82..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageDeserializer.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.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/0c771505/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
deleted file mode 100644
index c0a0d32..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageSerializer.java
+++ /dev/null
@@ -1,320 +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.messages;
-
-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.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.network.Client client} and
- * {@link org.apache.flink.queryablestate.network.AbstractServerBase 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}.
- *
- * @param <REQ>		Type of the requests of the protocol.
- * @param <RESP>	Type of the responses of the protocol.
- */
-@Internal
-public final class MessageSerializer<REQ extends MessageBody, RESP extends MessageBody> {
-
-	/** 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;
-
-	/** 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
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Serializes the request sent to the
-	 * {@link org.apache.flink.queryablestate.network.AbstractServerBase}.
-	 *
-	 * @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 <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());
-	}
-
-	/**
-	 * Serializes the response sent to the
-	 * {@link org.apache.flink.queryablestate.network.Client}.
-	 *
-	 * @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 <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.network.Client} 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 serializeRequestFailure(
-			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.network.Client} 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());
-	}
-
-	/**
-	 * 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
-	// ------------------------------------------------------------------------
-
-	/**
-	 * 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];
-	}
-
-	/**
-	 * 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 long getRequestId(final ByteBuf buf) {
-		return buf.readLong();
-	}
-
-	/**
-	 * 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 REQ deserializeRequest(final ByteBuf buf) {
-		Preconditions.checkNotNull(buf);
-		return requestDeserializer.deserializeMessage(buf);
-	}
-
-	/**
-	 * 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 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>
-	 * </pre>
-	 * @param buf	The {@link ByteBuf} containing the serialized failure message.
-	 * @return		The failure message.
-	 */
-	public static RequestFailure deserializeRequestFailure(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 RequestFailure(requestId, cause);
-	}
-
-	/**
-	 * De-serializes the failure message sent to the
-	 * {@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>
-	 * </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/0c771505/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
deleted file mode 100644
index 562ce93..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageType.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.queryablestate.network.messages;
-
-import org.apache.flink.annotation.Internal;
-
-/**
- * Expected message types during the communication between
- * {@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. */
-	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/0c771505/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
deleted file mode 100644
index 106199f..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/RequestFailure.java
+++ /dev/null
@@ -1,71 +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.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/0c771505/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
deleted file mode 100644
index 055a5d0..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.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.server;
-
-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.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.channel.ChannelHandler;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.CompletableFuture;
-
-/**
- * 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.).
- */
-@Internal
-@ChannelHandler.Sharable
-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;
-
-	/**
-	 * Create the handler used by the {@link KvStateServerImpl}.
-	 *
-	 * @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(
-			final KvStateServerImpl server,
-			final KvStateRegistry kvStateRegistry,
-			final MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer,
-			final KvStateRequestStats stats) {
-
-		super(server, serializer, stats);
-		this.registry = Preconditions.checkNotNull(kvStateRegistry);
-	}
-
-	@Override
-	public CompletableFuture<KvStateResponse> handleRequest(final long requestId, final KvStateInternalRequest request) {
-		final CompletableFuture<KvStateResponse> responseFuture = new CompletableFuture<>();
-
-		try {
-			final InternalKvState<?> kvState = registry.getKvState(request.getKvStateId());
-			if (kvState == null) {
-				responseFuture.completeExceptionally(new UnknownKvStateIdException(getServerName(), request.getKvStateId()));
-			} else {
-				byte[] serializedKeyAndNamespace = request.getSerializedKeyAndNamespace();
-
-				byte[] serializedResult = kvState.getSerializedValue(serializedKeyAndNamespace);
-				if (serializedResult != null) {
-					responseFuture.complete(new KvStateResponse(serializedResult));
-				} else {
-					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 void shutdown() {
-		// do nothing
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c771505/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
deleted file mode 100644
index dfca915..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java
+++ /dev/null
@@ -1,111 +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.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.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.InetAddress;
-import java.util.Iterator;
-
-/**
- * The default implementation of the {@link KvStateServer}.
- */
-@Internal
-public class KvStateServerImpl extends AbstractServerBase<KvStateInternalRequest, KvStateResponse> implements KvStateServer {
-
-	private static final Logger LOG = LoggerFactory.getLogger(KvStateServerImpl.class);
-
-	/** The {@link KvStateRegistry} to query for state instances. */
-	private final KvStateRegistry kvStateRegistry;
-
-	private final KvStateRequestStats stats;
-
-	private MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer;
-
-	/**
-	 * Creates the state server.
-	 *
-	 * <p>The server is instantiated using reflection by the
-	 * {@link org.apache.flink.runtime.query.QueryableStateUtils#createKvStateServer(InetAddress, Iterator, int, int, KvStateRegistry, KvStateRequestStats)
-	 * QueryableStateUtils.createKvStateServer(InetAddress, Iterator, 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 the address to listen to.
-	 * @param bindPortIterator the port range to try to bind 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(
-			final InetAddress bindAddress,
-			final Iterator<Integer> bindPortIterator,
-			final Integer numEventLoopThreads,
-			final Integer numQueryThreads,
-			final KvStateRegistry kvStateRegistry,
-			final KvStateRequestStats stats) {
-
-		super("Queryable State Server", bindAddress, bindPortIterator, numEventLoopThreads, numQueryThreads);
-		this.stats = Preconditions.checkNotNull(stats);
-		this.kvStateRegistry = Preconditions.checkNotNull(kvStateRegistry);
-	}
-
-	@Override
-	public AbstractServerHandler<KvStateInternalRequest, KvStateResponse> initializeHandler() {
-		this.serializer = new MessageSerializer<>(
-				new KvStateInternalRequest.KvStateInternalRequestDeserializer(),
-				new KvStateResponse.KvStateResponseDeserializer());
-		return new KvStateServerHandler(this, kvStateRegistry, serializer, stats);
-	}
-
-	public MessageSerializer<KvStateInternalRequest, KvStateResponse> getSerializer() {
-		Preconditions.checkState(serializer != null, "Server " + getServerName() + " has not been started.");
-		return serializer;
-	}
-
-	@Override
-	public void start() throws Throwable {
-		super.start();
-	}
-
-	@Override
-	public KvStateServerAddress getServerAddress() {
-		return super.getServerAddress();
-	}
-
-	@Override
-	public void shutdown() {
-		super.shutdown();
-	}
-}