Repository: flink Updated Branches: refs/heads/master a0838de79 -> a4d869759
[FLINK-8062][QS] Make getKvState() with namespace private. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/ff7e3cf6 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/ff7e3cf6 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/ff7e3cf6 Branch: refs/heads/master Commit: ff7e3cf6749a6b6bc898fde871c36661c8629c23 Parents: a0838de Author: kkloudas <kklou...@gmail.com> Authored: Wed Nov 15 15:32:42 2017 +0100 Committer: kkloudas <kklou...@gmail.com> Committed: Fri Nov 17 10:46:08 2017 +0100 ---------------------------------------------------------------------- .../flink/queryablestate/client/QueryableStateClient.java | 3 +-- .../itcases/AbstractQueryableStateTestBase.java | 7 +------ 2 files changed, 2 insertions(+), 8 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/flink/blob/ff7e3cf6/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 index 304505a..03e02e1 100644 --- 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 @@ -186,8 +186,7 @@ public class QueryableStateClient { * @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( + private <K, N, S extends State, V> CompletableFuture<S> getKvState( final JobID jobId, final String queryableStateName, final K key, http://git-wip-us.apache.org/repos/asf/flink/blob/ff7e3cf6/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 index c1cbb61..a789dbd 100644 --- 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 @@ -47,7 +47,6 @@ 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; @@ -491,9 +490,7 @@ public abstract class AbstractQueryableStateTestBase extends TestLogger { jobId, "wrong-hankuna", // this is the wrong name. 0, - VoidNamespace.INSTANCE, BasicTypeInfo.INT_TYPE_INFO, - VoidNamespaceTypeInfo.INSTANCE, valueState); try { @@ -572,9 +569,7 @@ public abstract class AbstractQueryableStateTestBase extends TestLogger { jobId, queryableState.getQueryableStateName(), 0, - VoidNamespace.INSTANCE, BasicTypeInfo.INT_TYPE_INFO, - VoidNamespaceTypeInfo.INSTANCE, valueState); cluster.submitJobDetached(jobGraph); @@ -1486,7 +1481,7 @@ public abstract class AbstractQueryableStateTestBase extends TestLogger { if (!resultFuture.isDone()) { Thread.sleep(100L); - CompletableFuture<S> expected = client.getKvState(jobId, queryName, key, VoidNamespace.INSTANCE, keyTypeInfo, VoidNamespaceTypeInfo.INSTANCE, stateDescriptor); + CompletableFuture<S> expected = client.getKvState(jobId, queryName, key, keyTypeInfo, stateDescriptor); expected.whenCompleteAsync((result, throwable) -> { if (throwable != null) { if (