From 0c771505b84cdacf7a359c3be0efe38a30f9e660 Mon Sep 17 00:00:00 2001 From: kkloudas Date: Tue, 24 Oct 2017 12:16:08 +0200 Subject: [PATCH 1/2] [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. --- .../streaming/state/AbstractRocksDBState.java | 2 +- .../streaming/state/RocksDBMapState.java | 2 +- .../flink-queryable-state-client-java/pom.xml | 90 ++++ .../flink/queryablestate/FutureUtils.java | 43 ++ .../flink/queryablestate}/KvStateID.java | 7 +- .../client/QueryableStateClient.java | 33 +- .../queryablestate/client/VoidNamespace.java | 75 ++++ .../client/VoidNamespaceSerializer.java | 96 +++++ .../client/VoidNamespaceTypeInfo.java | 92 ++++ .../state/ImmutableAggregatingState.java | 2 +- .../client/state/ImmutableFoldingState.java | 2 +- .../client/state/ImmutableListState.java | 2 +- .../client/state/ImmutableMapState.java | 2 +- .../client/state/ImmutableReducingState.java | 2 +- .../client/state/ImmutableState.java | 0 .../client/state/ImmutableStateBinder.java | 0 .../client/state/ImmutableValueState.java | 2 +- .../serialization/DataInputDeserializer.java | 392 ++++++++++++++++++ .../serialization/DataOutputSerializer.java | 344 +++++++++++++++ .../serialization}/KvStateSerializer.java | 4 +- .../UnknownJobManagerException.java | 6 +- .../UnknownKeyOrNamespaceException.java | 2 +- .../UnknownKvStateIdException.java | 4 +- ...knownKvStateKeyGroupLocationException.java | 6 +- .../messages/KvStateRequest.java | 3 +- .../messages/KvStateResponse.java | 7 +- .../network/AbstractServerBase.java | 8 +- .../network/AbstractServerHandler.java | 5 +- .../network/BadRequestException.java | 0 .../network/ChunkedByteBuf.java | 0 .../flink/queryablestate/network/Client.java | 27 +- .../queryablestate/network/ClientHandler.java | 0 .../network/ClientHandlerCallback.java | 0 .../network/NettyBufferPool.java | 171 ++++++++ .../network/messages/MessageBody.java | 0 .../network/messages/MessageDeserializer.java | 0 .../network/messages/MessageSerializer.java | 0 .../network/messages/MessageType.java | 0 .../network/messages/RequestFailure.java | 0 .../stats}/AtomicKvStateRequestStats.java | 2 +- .../stats}/DisabledKvStateRequestStats.java | 2 +- .../network/stats}/KvStateRequestStats.java | 7 +- .../client/VoidNamespaceTypeInfoTest.java | 32 ++ .../state/ImmutableAggregatingStateTest.java | 3 +- .../state/ImmutableFoldingStateTest.java | 3 +- .../client}/state/ImmutableListStateTest.java | 6 +- .../client}/state/ImmutableMapStateTest.java | 5 +- .../state/ImmutableReducingStateTest.java | 3 +- .../state/ImmutableValueStateTest.java | 3 +- .../src/test/resources/log4j-test.properties | 0 .../pom.xml | 34 +- .../proxy/KvStateClientProxyHandler.java | 14 +- .../client/proxy/KvStateClientProxyImpl.java | 8 +- .../messages/KvStateInternalRequest.java | 2 +- .../server/KvStateServerHandler.java | 6 +- .../server/KvStateServerImpl.java | 6 +- .../AbstractQueryableStateTestBase.java | 8 +- .../HAAbstractQueryableStateTestBase.java | 0 .../HAQueryableStateFsBackendITCase.java | 0 .../HAQueryableStateRocksDBBackendITCase.java | 0 .../KVStateRequestSerializerRocksDBTest.java | 6 +- .../NonHAAbstractQueryableStateTestBase.java | 0 .../NonHAQueryableStateFsBackendITCase.java | 0 ...nHAQueryableStateRocksDBBackendITCase.java | 0 .../network/AbstractServerTest.java | 2 +- .../queryablestate/network/ClientTest.java | 30 +- .../network/KvStateClientHandlerTest.java | 0 .../KvStateRequestSerializerTest.java | 7 +- .../network/KvStateServerHandlerTest.java | 18 +- .../network/KvStateServerTest.java | 25 +- .../network/MessageSerializerTest.java | 2 +- .../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 +- .../flink/runtime/query/KvStateLocation.java | 14 +- .../query/KvStateLocationRegistry.java | 4 +- .../flink/runtime/query/KvStateMessage.java | 10 +- .../flink/runtime/query/KvStateRegistry.java | 1 + .../runtime/query/KvStateRegistryGateway.java | 5 +- .../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 + .../runtime/state/heap/AbstractHeapState.java | 6 +- .../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 +- .../runtime/state/StateBackendTestBase.java | 4 +- .../TypeInfoTestCoverageTest.java | 3 +- pom.xml | 2 +- tools/travis_mvn_watchdog.sh | 3 +- 98 files changed, 1601 insertions(+), 335 deletions(-) create mode 100644 flink-queryable-state/flink-queryable-state-client-java/pom.xml create mode 100644 flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/FutureUtils.java rename {flink-runtime/src/main/java/org/apache/flink/runtime/query => flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate}/KvStateID.java (82%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java (88%) create mode 100644 flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/VoidNamespace.java create mode 100644 flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/VoidNamespaceSerializer.java create mode 100644 flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/VoidNamespaceTypeInfo.java rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableAggregatingState.java (96%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableFoldingState.java (96%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableListState.java (96%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableMapState.java (97%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableReducingState.java (96%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableState.java (100%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableStateBinder.java (100%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableValueState.java (96%) create mode 100644 flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/serialization/DataInputDeserializer.java create mode 100644 flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/serialization/DataOutputSerializer.java rename {flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message => flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/serialization}/KvStateSerializer.java (98%) rename flink-queryable-state/{flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate => flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions}/UnknownJobManagerException.java (88%) rename flink-queryable-state/{flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate => flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions}/UnknownKeyOrNamespaceException.java (96%) rename flink-queryable-state/{flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate => flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions}/UnknownKvStateIdException.java (93%) rename flink-queryable-state/{flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate => flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions}/UnknownKvStateKeyGroupLocationException.java (89%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequest.java (96%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/main/java/org/apache/flink/queryablestate/messages/KvStateResponse.java (88%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java (97%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/main/java/org/apache/flink/queryablestate/network/AbstractServerHandler.java (98%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/main/java/org/apache/flink/queryablestate/network/BadRequestException.java (100%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/main/java/org/apache/flink/queryablestate/network/ChunkedByteBuf.java (100%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/main/java/org/apache/flink/queryablestate/network/Client.java (93%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/main/java/org/apache/flink/queryablestate/network/ClientHandler.java (100%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/main/java/org/apache/flink/queryablestate/network/ClientHandlerCallback.java (100%) create mode 100644 flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/NettyBufferPool.java rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/main/java/org/apache/flink/queryablestate/network/messages/MessageBody.java (100%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/main/java/org/apache/flink/queryablestate/network/messages/MessageDeserializer.java (100%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/main/java/org/apache/flink/queryablestate/network/messages/MessageSerializer.java (100%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/main/java/org/apache/flink/queryablestate/network/messages/MessageType.java (100%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/main/java/org/apache/flink/queryablestate/network/messages/RequestFailure.java (100%) rename {flink-runtime/src/main/java/org/apache/flink/runtime/query/netty => flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats}/AtomicKvStateRequestStats.java (97%) rename {flink-runtime/src/main/java/org/apache/flink/runtime/query/netty => flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats}/DisabledKvStateRequestStats.java (95%) rename {flink-runtime/src/main/java/org/apache/flink/runtime/query/netty => flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats}/KvStateRequestStats.java (86%) create mode 100644 flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/VoidNamespaceTypeInfoTest.java rename flink-queryable-state/{flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate => flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client}/state/ImmutableAggregatingStateTest.java (95%) rename flink-queryable-state/{flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate => flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client}/state/ImmutableFoldingStateTest.java (95%) rename flink-queryable-state/{flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate => flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client}/state/ImmutableListStateTest.java (92%) rename flink-queryable-state/{flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate => flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client}/state/ImmutableMapStateTest.java (96%) rename flink-queryable-state/{flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate => flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client}/state/ImmutableReducingStateTest.java (95%) rename flink-queryable-state/{flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate => flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client}/state/ImmutableValueStateTest.java (94%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-client-java}/src/test/resources/log4j-test.properties (100%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-runtime}/pom.xml (82%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-runtime}/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyHandler.java (94%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-runtime}/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java (94%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-runtime}/src/main/java/org/apache/flink/queryablestate/messages/KvStateInternalRequest.java (98%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-runtime}/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java (94%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-runtime}/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java (95%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-runtime}/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java (99%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-runtime}/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java (100%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-runtime}/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateFsBackendITCase.java (100%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-runtime}/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateRocksDBBackendITCase.java (100%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-runtime}/src/test/java/org/apache/flink/queryablestate/itcases/KVStateRequestSerializerRocksDBTest.java (96%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-runtime}/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java (100%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-runtime}/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateFsBackendITCase.java (100%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-runtime}/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateRocksDBBackendITCase.java (100%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-runtime}/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java (98%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-runtime}/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java (95%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-runtime}/src/test/java/org/apache/flink/queryablestate/network/KvStateClientHandlerTest.java (100%) rename {flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/message => flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network}/KvStateRequestSerializerTest.java (98%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-runtime}/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java (97%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-runtime}/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java (91%) rename flink-queryable-state/{flink-queryable-state-java => flink-queryable-state-runtime}/src/test/java/org/apache/flink/queryablestate/network/MessageSerializerTest.java (99%) create mode 100644 flink-queryable-state/flink-queryable-state-runtime/src/test/resources/log4j-test.properties delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateServerAddress.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 cf365b49af0da..969a1fc44e0b3 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.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; 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 421bb2ea402cf..e8c34ccf6f10d 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.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; 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 0000000000000..8a4ff69b9820f --- /dev/null +++ b/flink-queryable-state/flink-queryable-state-client-java/pom.xml @@ -0,0 +1,90 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-queryable-state + 1.4-SNAPSHOT + .. + + + flink-queryable-state-client-java_${scala.binary.version} + flink-queryable-state-client-java + jar + + + + + + + org.apache.flink + flink-core + ${project.version} + provided + + + + org.apache.flink + flink-shaded-netty + + + + org.apache.flink + flink-shaded-guava + + + + org.apache.flink + flink-core + ${project.version} + test + test-jar + + + + org.apache.flink + flink-test-utils-junit + ${project.version} + test + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + + 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 0000000000000..e2af7b15da859 --- /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 CompletableFuture getFailedFuture(Throwable throwable) { + CompletableFuture failedAttempt = new CompletableFuture<>(); + failedAttempt.completeExceptionally(throwable); + return failedAttempt; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateID.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/KvStateID.java similarity index 82% rename from flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateID.java rename to flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/KvStateID.java index c122508fcc718..992b2835fcc78 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateID.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/KvStateID.java @@ -16,15 +16,14 @@ * limitations under the License. */ -package org.apache.flink.runtime.query; +package org.apache.flink.queryablestate; -import org.apache.flink.runtime.state.internal.InternalKvState; import org.apache.flink.util.AbstractID; /** - * Identifier for {@link InternalKvState} instances. + * Identifier for state instances. * - *

Assigned when registering state at the {@link KvStateRegistry}. + *

Assigned when registering the state at the state registry. */ public class KvStateID extends AbstractID { 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-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java similarity index 88% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java rename to flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java index 70bccf0df5d95..304505ac29dde 100644 --- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java @@ -26,18 +26,14 @@ 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.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.queryablestate.network.stats.DisabledKvStateRequestStats; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.Preconditions; @@ -46,6 +42,7 @@ import java.io.IOException; import java.net.InetAddress; +import java.net.InetSocketAddress; import java.net.UnknownHostException; import java.util.concurrent.CompletableFuture; @@ -56,10 +53,10 @@ * 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. * - *

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. + *

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. * *

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 @@ -74,15 +71,14 @@ public class QueryableStateClient { private final Client client; /** The address of the proxy this client is connected to. */ - private final KvStateServerAddress remoteAddress; + 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 {@link org.apache.flink.runtime.query.KvStateClientProxy proxy} - * to connect to. + * @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 { @@ -91,15 +87,14 @@ public QueryableStateClient(final String remoteHostname, final int remotePort) t /** * 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 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 KvStateServerAddress(remoteAddress, remotePort); + this.remoteAddress = new InetSocketAddress(remoteAddress, remotePort); final MessageSerializer messageSerializer = new MessageSerializer<>( @@ -108,7 +103,7 @@ public QueryableStateClient(final InetAddress remoteAddress, final int remotePor this.client = new Client<>( "Queryable State Client", - Hardware.getNumberCPUCores(), + 1, messageSerializer, new DisabledKvStateRequestStats()); } @@ -213,6 +208,8 @@ public CompletableFuture getKvState( TypeSerializer keySerializer = keyTypeInfo.createSerializer(executionConfig); TypeSerializer namespaceSerializer = namespaceTypeInfo.createSerializer(executionConfig); + stateDescriptor.initializeSerializerUnlessSet(executionConfig); + final byte[] serializedKeyAndNamespace; try { serializedKeyAndNamespace = KvStateSerializer 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 0000000000000..0560ec23eb917 --- /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. + * + *

THIS WAS COPIED FROM RUNTIME SO THAT WE AVOID THE DEPENDENCY. + */ +@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; + } +} 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 0000000000000..38db7058898a1 --- /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}. + * + *

THIS WAS COPIED FROM RUNTIME SO THAT WE AVOID THE DEPENDENCY. + */ +@Internal +public final class VoidNamespaceSerializer extends TypeSerializerSingleton { + + 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; + } +} 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 0000000000000..2efb87bd667f2 --- /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}. + * + *

THIS WAS COPIED FROM RUNTIME SO THAT WE AVOID THE DEPENDENCY. + */ +@Internal +public class VoidNamespaceTypeInfo extends TypeInformation { + + 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 getTypeClass() { + return VoidNamespace.class; + } + + @Override + public boolean isKeyType() { + return false; + } + + @Override + public TypeSerializer 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; + } +} 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-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableAggregatingState.java similarity index 96% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableAggregatingState.java rename to flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableAggregatingState.java index b853cfc4b8c5a..8964fbf357f35 100644 --- 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-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableAggregatingState.java @@ -21,7 +21,7 @@ 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.queryablestate.client.state.serialization.KvStateSerializer; import org.apache.flink.util.Preconditions; import java.io.IOException; 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-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableFoldingState.java similarity index 96% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableFoldingState.java rename to flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableFoldingState.java index a12adaad17619..25f31180e2f94 100644 --- 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-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableFoldingState.java @@ -21,7 +21,7 @@ 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.queryablestate.client.state.serialization.KvStateSerializer; import org.apache.flink.util.Preconditions; import java.io.IOException; 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-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableListState.java similarity index 96% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableListState.java rename to flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableListState.java index 841690581236c..3dcd75d1929ec 100644 --- 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-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableListState.java @@ -21,7 +21,7 @@ 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.queryablestate.client.state.serialization.KvStateSerializer; import org.apache.flink.util.Preconditions; import java.io.IOException; 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-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableMapState.java similarity index 97% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableMapState.java rename to flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableMapState.java index c216d5dbcd0c0..bb08cf0dd9191 100644 --- 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-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableMapState.java @@ -21,7 +21,7 @@ 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.queryablestate.client.state.serialization.KvStateSerializer; import org.apache.flink.util.Preconditions; import java.io.IOException; 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-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableReducingState.java similarity index 96% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableReducingState.java rename to flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableReducingState.java index da08c5375f36b..46b477fb8c70c 100644 --- 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-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableReducingState.java @@ -21,7 +21,7 @@ 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.queryablestate.client.state.serialization.KvStateSerializer; import org.apache.flink.util.Preconditions; import java.io.IOException; 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-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableState.java similarity index 100% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableState.java rename to 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-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 similarity index 100% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableStateBinder.java rename to 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-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 similarity index 96% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableValueState.java rename to flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableValueState.java index 7fd6457944b27..f3ddd2b5021c0 100644 --- 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-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableValueState.java @@ -21,7 +21,7 @@ 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.queryablestate.client.state.serialization.KvStateSerializer; import org.apache.flink.util.Preconditions; import java.io.IOException; 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 0000000000000..878df85942865 --- /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. + * + *

THIS WAS COPIED FROM RUNTIME SO THAT WE AVOID THE DEPENDENCY. + */ +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); +} 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 0000000000000..5811c91f3d20e --- /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. + * + *

THIS WAS COPIED FROM RUNTIME SO THAT WE AVOID THE DEPENDENCY. + */ +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); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateSerializer.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/serialization/KvStateSerializer.java similarity index 98% rename from flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateSerializer.java rename to flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/serialization/KvStateSerializer.java index 44ee571ccbb2b..4c69483989be3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateSerializer.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/serialization/KvStateSerializer.java @@ -16,12 +16,10 @@ * limitations under the License. */ -package org.apache.flink.runtime.query.netty.message; +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 org.apache.flink.runtime.util.DataInputDeserializer; -import org.apache.flink.runtime.util.DataOutputSerializer; import java.io.IOException; import java.nio.ByteBuffer; 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-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownJobManagerException.java similarity index 88% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownJobManagerException.java rename to flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownJobManagerException.java index fa2604badaf6e..19063c2cefac5 100644 --- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownJobManagerException.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownJobManagerException.java @@ -16,14 +16,14 @@ * limitations under the License. */ -package org.apache.flink.queryablestate; +package org.apache.flink.queryablestate.exceptions; 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. + * {@code Client Proxy} is running on, does not know the active + * Job Manager. */ @Internal public class UnknownJobManagerException extends Exception { 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-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownKeyOrNamespaceException.java similarity index 96% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKeyOrNamespaceException.java rename to flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownKeyOrNamespaceException.java index c497a729e5ee5..08e3324a46d39 100644 --- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKeyOrNamespaceException.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownKeyOrNamespaceException.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.queryablestate; +package org.apache.flink.queryablestate.exceptions; import org.apache.flink.annotation.Internal; import org.apache.flink.queryablestate.network.BadRequestException; 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-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownKvStateIdException.java similarity index 93% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateIdException.java rename to flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownKvStateIdException.java index 59ba0811b856b..81ea17778e971 100644 --- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateIdException.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownKvStateIdException.java @@ -16,11 +16,11 @@ * limitations under the License. */ -package org.apache.flink.queryablestate; +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.runtime.query.KvStateID; import org.apache.flink.util.Preconditions; /** 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-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownKvStateKeyGroupLocationException.java similarity index 89% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateKeyGroupLocationException.java rename to flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownKvStateKeyGroupLocationException.java index 0d6588af403db..d8d34f7595c9b 100644 --- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateKeyGroupLocationException.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/exceptions/UnknownKvStateKeyGroupLocationException.java @@ -16,15 +16,13 @@ * limitations under the License. */ -package org.apache.flink.queryablestate; +package org.apache.flink.queryablestate.exceptions; 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. + * Exception thrown if there is no location information available for the given key group. */ @Internal public class UnknownKvStateKeyGroupLocationException extends BadRequestException { 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-client-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequest.java similarity index 96% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequest.java rename to flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequest.java index 7eb39c750b897..8169d4821e54d 100644 --- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequest.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequest.java @@ -31,8 +31,7 @@ /** * 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. + * Queryable State Client} to the Client Proxy requesting a given state. */ @Internal public class KvStateRequest extends MessageBody { 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-client-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateResponse.java similarity index 88% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateResponse.java rename to flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateResponse.java index 462135fa0148a..6bf14a7039f00 100644 --- 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-client-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateResponse.java @@ -28,10 +28,9 @@ 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}. + * 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 { 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-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java similarity index 97% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java rename to flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java index be852fbab46bc..487020a1cc92c 100644 --- 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-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java @@ -21,8 +21,6 @@ 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; @@ -92,7 +90,7 @@ public abstract class AbstractServerBase handler; @@ -171,7 +169,7 @@ public String getServerName() { * @return AbstractServerBase address * @throws IllegalStateException If server has not been started yet */ - public KvStateServerAddress getServerAddress() { + public InetSocketAddress getServerAddress() { Preconditions.checkState(serverAddress != null, "Server " + serverName + " has not been started."); return serverAddress; } @@ -233,7 +231,7 @@ private boolean attemptToBind(final int port) throws Throwable { final ChannelFuture future = bootstrap.bind().sync(); if (future.isSuccess()) { final InetSocketAddress localAddress = (InetSocketAddress) future.channel().localAddress(); - serverAddress = new KvStateServerAddress(localAddress.getAddress(), localAddress.getPort()); + serverAddress = new InetSocketAddress(localAddress.getAddress(), localAddress.getPort()); return true; } 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-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerHandler.java similarity index 98% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerHandler.java rename to flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerHandler.java index 18a88dacb9ac6..9e02291b52895 100644 --- 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-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerHandler.java @@ -22,8 +22,7 @@ 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.queryablestate.network.stats.KvStateRequestStats; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; @@ -189,7 +188,7 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws E public abstract void shutdown(); /** - * Task to execute the actual query against the {@link InternalKvState} instance. + * Task to execute the actual query against the state instance. */ private static class AsyncRequestTask implements Runnable { 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-client-java/src/main/java/org/apache/flink/queryablestate/network/BadRequestException.java similarity index 100% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/BadRequestException.java rename to 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-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 similarity index 100% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/ChunkedByteBuf.java rename to 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-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 similarity index 93% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/Client.java rename to flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/Client.java index e6d59deadb262..13d34fb48fadb 100644 --- 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-client-java/src/main/java/org/apache/flink/queryablestate/network/Client.java @@ -19,12 +19,10 @@ 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.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.queryablestate.network.stats.KvStateRequestStats; import org.apache.flink.util.Preconditions; import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -43,6 +41,7 @@ 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; @@ -77,10 +76,10 @@ public class Client { private final KvStateRequestStats stats; /** Established connections. */ - private final Map establishedConnections = new ConcurrentHashMap<>(); + private final Map establishedConnections = new ConcurrentHashMap<>(); /** Pending connections. */ - private final Map pendingConnections = new ConcurrentHashMap<>(); + private final Map pendingConnections = new ConcurrentHashMap<>(); /** Atomic shut down flag. */ private final AtomicBoolean shutDown = new AtomicBoolean(); @@ -132,7 +131,7 @@ public String getClientName() { return clientName; } - public CompletableFuture sendRequest(final KvStateServerAddress serverAddress, final REQ request) { + public CompletableFuture sendRequest(final InetSocketAddress serverAddress, final REQ request) { if (shutDown.get()) { return FutureUtils.getFailedFuture(new IllegalStateException("Shut down")); } @@ -152,7 +151,7 @@ public CompletableFuture sendRequest(final KvStateServerAddress serverAddr if (previous == null) { // OK, we are responsible to connect. - bootstrap.connect(serverAddress.getHost(), serverAddress.getPort()).addListener(pending); + bootstrap.connect(serverAddress.getAddress(), serverAddress.getPort()).addListener(pending); return pending.sendRequest(request); } else { // There was a race, use the existing pending connection. @@ -169,13 +168,13 @@ public CompletableFuture sendRequest(final KvStateServerAddress serverAddr */ public void shutdown() { if (shutDown.compareAndSet(false, true)) { - for (Map.Entry conn : establishedConnections.entrySet()) { + for (Map.Entry conn : establishedConnections.entrySet()) { if (establishedConnections.remove(conn.getKey(), conn.getValue())) { conn.getValue().close(); } } - for (Map.Entry conn : pendingConnections.entrySet()) { + for (Map.Entry conn : pendingConnections.entrySet()) { if (pendingConnections.remove(conn.getKey()) != null) { conn.getValue().close(); } @@ -199,7 +198,7 @@ private class PendingConnection implements ChannelFutureListener { private final Object connectLock = new Object(); /** Address of the server we are connecting to. */ - private final KvStateServerAddress serverAddress; + private final InetSocketAddress serverAddress; private final MessageSerializer serializer; @@ -221,7 +220,7 @@ private class PendingConnection implements ChannelFutureListener { * @param serverAddress Address of the server to connect to. */ private PendingConnection( - final KvStateServerAddress serverAddress, + final InetSocketAddress serverAddress, final MessageSerializer serializer) { this.serverAddress = serverAddress; this.serializer = serializer; @@ -370,7 +369,7 @@ private PendingRequest(REQ request) { private class EstablishedConnection implements ClientHandlerCallback { /** Address of the server we are connected to. */ - private final KvStateServerAddress serverAddress; + private final InetSocketAddress serverAddress; /** The actual TCP channel. */ private final Channel channel; @@ -391,7 +390,7 @@ private class EstablishedConnection implements ClientHandlerCallback { * @param channel The actual TCP channel */ EstablishedConnection( - final KvStateServerAddress serverAddress, + final InetSocketAddress serverAddress, final MessageSerializer serializer, final Channel channel) { 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-client-java/src/main/java/org/apache/flink/queryablestate/network/ClientHandler.java similarity index 100% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/ClientHandler.java rename to 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-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 similarity index 100% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/ClientHandlerCallback.java rename to 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/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 0000000000000..5e014b83f8419 --- /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(); + } +} 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-client-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageBody.java similarity index 100% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageBody.java rename to 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-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 similarity index 100% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageDeserializer.java rename to 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-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 similarity index 100% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageSerializer.java rename to 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-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 similarity index 100% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageType.java rename to 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-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 similarity index 100% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/RequestFailure.java rename to flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/messages/RequestFailure.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/AtomicKvStateRequestStats.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats/AtomicKvStateRequestStats.java similarity index 97% rename from flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/AtomicKvStateRequestStats.java rename to flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats/AtomicKvStateRequestStats.java index 1d80babf24561..9ba5f846c078f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/AtomicKvStateRequestStats.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats/AtomicKvStateRequestStats.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.query.netty; +package org.apache.flink.queryablestate.network.stats; import java.util.concurrent.atomic.AtomicLong; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/DisabledKvStateRequestStats.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats/DisabledKvStateRequestStats.java similarity index 95% rename from flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/DisabledKvStateRequestStats.java rename to flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats/DisabledKvStateRequestStats.java index de8824ddb3a78..b34ac3e371e89 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/DisabledKvStateRequestStats.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats/DisabledKvStateRequestStats.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.query.netty; +package org.apache.flink.queryablestate.network.stats; /** * Disabled {@link KvStateRequestStats} implementation. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateRequestStats.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats/KvStateRequestStats.java similarity index 86% rename from flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateRequestStats.java rename to flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats/KvStateRequestStats.java index 19caf924fa17c..8e9edd8b3d929 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateRequestStats.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/stats/KvStateRequestStats.java @@ -16,12 +16,11 @@ * limitations under the License. */ -package org.apache.flink.runtime.query.netty; +package org.apache.flink.queryablestate.network.stats; /** - * Simple statistics for - * {@link org.apache.flink.runtime.query.KvStateServer} and - * {@link org.apache.flink.runtime.query.KvStateClientProxy} monitoring. + * Simple statistics for monitoring the state server + * and the client proxy. */ public interface KvStateRequestStats { 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 0000000000000..ca11a32b3af46 --- /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 { + + @Override + protected VoidNamespaceTypeInfo[] getTestData() { + return new VoidNamespaceTypeInfo[] { VoidNamespaceTypeInfo.INSTANCE }; + } +} 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-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableAggregatingStateTest.java similarity index 95% rename from flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableAggregatingStateTest.java rename to flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableAggregatingStateTest.java index 2e05f611127db..ebbc896994d17 100644 --- 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-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableAggregatingStateTest.java @@ -16,13 +16,12 @@ * limitations under the License. */ -package org.apache.flink.queryablestate.state; +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.apache.flink.queryablestate.client.state.ImmutableAggregatingState; import org.junit.Before; import org.junit.Test; 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-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableFoldingStateTest.java similarity index 95% rename from flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableFoldingStateTest.java rename to flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableFoldingStateTest.java index d2c95351acd8e..9e8dfc982f59b 100644 --- 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-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableFoldingStateTest.java @@ -16,14 +16,13 @@ * limitations under the License. */ -package org.apache.flink.queryablestate.state; +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.apache.flink.queryablestate.client.state.ImmutableFoldingState; import org.junit.Before; import org.junit.Test; 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-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableListStateTest.java similarity index 92% rename from flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableListStateTest.java rename to flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableListStateTest.java index 3283295187379..a78ed1f6080dc 100644 --- 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-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableListStateTest.java @@ -16,15 +16,13 @@ * limitations under the License. */ -package org.apache.flink.queryablestate.state; +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.apache.flink.queryablestate.client.state.ImmutableListState; -import org.apache.flink.runtime.state.heap.HeapListState; import org.junit.Before; import org.junit.Test; @@ -82,7 +80,7 @@ public void testClear() { } /** - * Copied from {@link HeapListState#getSerializedValue(Object, Object)}. + * Copied from HeapListState.getSerializedValue(Object, Object). */ private byte[] serializeInitValue(List toSerialize) throws IOException { TypeSerializer serializer = listStateDesc.getElementSerializer(); 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-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableMapStateTest.java similarity index 96% rename from flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableMapStateTest.java rename to flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableMapStateTest.java index 30a8a50c64b58..ffeabae00aa36 100644 --- 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-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableMapStateTest.java @@ -16,13 +16,12 @@ * limitations under the License. */ -package org.apache.flink.queryablestate.state; +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.ImmutableMapState; -import org.apache.flink.runtime.query.netty.message.KvStateSerializer; +import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer; import org.junit.Before; import org.junit.Test; 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-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableReducingStateTest.java similarity index 95% rename from flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableReducingStateTest.java rename to flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableReducingStateTest.java index 9b1ecf8d4b6b8..9694f557ab28b 100644 --- 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-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableReducingStateTest.java @@ -16,13 +16,12 @@ * limitations under the License. */ -package org.apache.flink.queryablestate.state; +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.apache.flink.queryablestate.client.state.ImmutableReducingState; import org.junit.Before; import org.junit.Test; 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-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableValueStateTest.java similarity index 94% rename from flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/state/ImmutableValueStateTest.java rename to flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableValueStateTest.java index 5f7032d2eab22..a0da43dcd20a7 100644 --- 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-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableValueStateTest.java @@ -16,12 +16,11 @@ * limitations under the License. */ -package org.apache.flink.queryablestate.state; +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.apache.flink.queryablestate.client.state.ImmutableValueState; import org.junit.Before; import org.junit.Test; diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/resources/log4j-test.properties b/flink-queryable-state/flink-queryable-state-client-java/src/test/resources/log4j-test.properties similarity index 100% rename from flink-queryable-state/flink-queryable-state-java/src/test/resources/log4j-test.properties rename to flink-queryable-state/flink-queryable-state-client-java/src/test/resources/log4j-test.properties diff --git a/flink-queryable-state/flink-queryable-state-java/pom.xml b/flink-queryable-state/flink-queryable-state-runtime/pom.xml similarity index 82% rename from flink-queryable-state/flink-queryable-state-java/pom.xml rename to flink-queryable-state/flink-queryable-state-runtime/pom.xml index e60c6f3cd9c88..f39498e06265f 100644 --- a/flink-queryable-state/flink-queryable-state-java/pom.xml +++ b/flink-queryable-state/flink-queryable-state-runtime/pom.xml @@ -30,8 +30,8 @@ under the License. .. - flink-queryable-state-java_${scala.binary.version} - flink-queryable-state-java + flink-queryable-state-runtime_${scala.binary.version} + flink-queryable-state-runtime jar @@ -47,35 +47,21 @@ under the License. org.apache.flink - flink-streaming-java_${scala.binary.version} + flink-runtime_${scala.binary.version} ${project.version} provided - - org.apache.flink - flink-test-utils_${scala.binary.version} + flink-queryable-state-client-java_${scala.binary.version} ${project.version} - test - - - - org.apache.flink - flink-streaming-java_${scala.binary.version} - ${project.version} - test-jar - test + provided - - org.apache.flink - flink-runtime_${scala.binary.version} - ${project.version} - test-jar - test - + org.apache.flink @@ -84,10 +70,6 @@ under the License. test - - org.apache.flink flink-test-utils-junit 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-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyHandler.java similarity index 94% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyHandler.java rename to flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyHandler.java index d7191b6792616..d4343365ba921 100644 --- 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-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyHandler.java @@ -21,24 +21,23 @@ 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.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.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; @@ -48,6 +47,7 @@ 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; @@ -167,7 +167,7 @@ private CompletableFuture getState( final int keyGroupIndex = KeyGroupRangeAssignment.computeKeyGroupForKeyHash( request.getKeyHashCode(), location.getNumKeyGroups()); - final KvStateServerAddress serverAddress = location.getKvStateServerAddress(keyGroupIndex); + final InetSocketAddress serverAddress = location.getKvStateServerAddress(keyGroupIndex); if (serverAddress == null) { return FutureUtils.getFailedFuture(new UnknownKvStateKeyGroupLocationException(getServerName())); } else { 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-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java similarity index 94% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java rename to flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java index 196641d042fc5..f473443516307 100644 --- 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-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java @@ -19,20 +19,20 @@ package org.apache.flink.queryablestate.client.proxy; import org.apache.flink.annotation.Internal; -import org.apache.flink.queryablestate.UnknownJobManagerException; +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.runtime.query.KvStateServerAddress; -import org.apache.flink.runtime.query.netty.KvStateRequestStats; import org.apache.flink.util.Preconditions; import java.net.InetAddress; +import java.net.InetSocketAddress; import java.util.Iterator; import java.util.concurrent.CompletableFuture; @@ -85,7 +85,7 @@ public KvStateClientProxyImpl( } @Override - public KvStateServerAddress getServerAddress() { + public InetSocketAddress getServerAddress() { return super.getServerAddress(); } 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-runtime/src/main/java/org/apache/flink/queryablestate/messages/KvStateInternalRequest.java similarity index 98% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateInternalRequest.java rename to flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/messages/KvStateInternalRequest.java index eedc2a1f2f7c4..8c8de59a6e66b 100644 --- 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-runtime/src/main/java/org/apache/flink/queryablestate/messages/KvStateInternalRequest.java @@ -19,9 +19,9 @@ 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.runtime.query.KvStateID; import org.apache.flink.util.Preconditions; import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; 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-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java similarity index 94% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java rename to flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java index 055a5d0e41b3c..476f153107894 100644 --- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java @@ -19,14 +19,14 @@ 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.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.query.netty.KvStateRequestStats; import org.apache.flink.runtime.state.internal.InternalKvState; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; 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-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java similarity index 95% rename from flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java rename to flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java index dfca915ddf4d9..fe0768748862a 100644 --- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java @@ -24,16 +24,16 @@ 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.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.net.InetSocketAddress; import java.util.Iterator; /** @@ -100,7 +100,7 @@ public void start() throws Throwable { } @Override - public KvStateServerAddress getServerAddress() { + public InetSocketAddress getServerAddress() { return super.getServerAddress(); } 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-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java similarity index 99% rename from flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java rename to flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java index 4d27da286c5fe..b4bae9ce1d03b 100644 --- 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-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java @@ -45,8 +45,11 @@ 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.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; @@ -57,9 +60,6 @@ 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; 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-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java similarity index 100% rename from flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java rename to 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-java/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 similarity index 100% rename from flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateFsBackendITCase.java rename to 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-java/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 similarity index 100% rename from flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateRocksDBBackendITCase.java rename to 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-java/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 similarity index 96% rename from flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/KVStateRequestSerializerRocksDBTest.java rename to flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/KVStateRequestSerializerRocksDBTest.java index 907e8a3d3c8e4..cb6fb3dc6c875 100644 --- 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-runtime/src/test/java/org/apache/flink/queryablestate/itcases/KVStateRequestSerializerRocksDBTest.java @@ -26,11 +26,11 @@ 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.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; 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-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java similarity index 100% rename from flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java rename to 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-java/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 similarity index 100% rename from flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateFsBackendITCase.java rename to 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-java/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 similarity index 100% rename from flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateRocksDBBackendITCase.java rename to 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-java/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 similarity index 98% rename from flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java rename to flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java index 1fd7012c17d9f..0b2727c2f1feb 100644 --- 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-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java @@ -22,7 +22,7 @@ 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.queryablestate.network.stats.DisabledKvStateRequestStats; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.Preconditions; 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-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java similarity index 95% rename from flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java rename to flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java index 40239257540c5..1fa4deb953675 100644 --- 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-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java @@ -22,23 +22,22 @@ 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.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; @@ -144,7 +143,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception } }); - KvStateServerAddress serverAddress = getKvStateServerAddress(serverChannel); + InetSocketAddress serverAddress = getKvStateServerAddress(serverChannel); long numQueries = 1024L; @@ -248,7 +247,7 @@ public void testRequestUnavailableHost() throws Exception { int availablePort = NetUtils.getAvailablePort(); - KvStateServerAddress serverAddress = new KvStateServerAddress( + InetSocketAddress serverAddress = new InetSocketAddress( InetAddress.getLocalHost(), availablePort); @@ -319,7 +318,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception } }); - final KvStateServerAddress serverAddress = getKvStateServerAddress(serverChannel); + final InetSocketAddress serverAddress = getKvStateServerAddress(serverChannel); final Client finalClient = client; Callable>> queryTask = () -> { @@ -407,7 +406,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception } }); - KvStateServerAddress serverAddress = getKvStateServerAddress(serverChannel); + InetSocketAddress serverAddress = getKvStateServerAddress(serverChannel); // Requests List> futures = new ArrayList<>(); @@ -512,7 +511,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception } }); - KvStateServerAddress serverAddress = getKvStateServerAddress(serverChannel); + InetSocketAddress serverAddress = getKvStateServerAddress(serverChannel); // Requests KvStateInternalRequest request = new KvStateInternalRequest(new KvStateID(), new byte[0]); @@ -633,7 +632,8 @@ public void testClientServerIntegration() throws Throwable { backend.setCurrentKey(1010 + i); // Value per server - ValueState state = backend.getPartitionedState(VoidNamespace.INSTANCE, + ValueState state = backend.getPartitionedState( + VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, desc); @@ -776,9 +776,7 @@ protected void initChannel(SocketChannel ch) throws Exception { return bootstrap.bind().sync().channel(); } - private KvStateServerAddress getKvStateServerAddress(Channel serverChannel) { - InetSocketAddress localAddress = (InetSocketAddress) serverChannel.localAddress(); - - return new KvStateServerAddress(localAddress.getAddress(), localAddress.getPort()); + private InetSocketAddress getKvStateServerAddress(Channel serverChannel) { + return (InetSocketAddress) serverChannel.localAddress(); } } 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-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateClientHandlerTest.java similarity index 100% rename from flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateClientHandlerTest.java rename to flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateClientHandlerTest.java diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/message/KvStateRequestSerializerTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateRequestSerializerTest.java similarity index 98% rename from flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/message/KvStateRequestSerializerTest.java rename to flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateRequestSerializerTest.java index aa4e6d80a6af5..d3314abad1a6a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/message/KvStateRequestSerializerTest.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateRequestSerializerTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.query.netty.message; +package org.apache.flink.queryablestate.network; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.state.ListStateDescriptor; @@ -25,10 +25,11 @@ 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.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; 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-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java similarity index 97% rename from flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java rename to flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java index 217d0b515e38c..041544d845bf3 100644 --- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java @@ -24,30 +24,30 @@ 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.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.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; 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-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java similarity index 91% rename from flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java rename to flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java index 7abc84efaca09..debd190504d51 100644 --- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java @@ -22,23 +22,22 @@ 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.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; @@ -58,6 +57,7 @@ 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; @@ -94,11 +94,16 @@ public void testSimpleRequest() throws Throwable { KvStateRegistry registry = new KvStateRegistry(); KvStateRequestStats stats = new AtomicKvStateRequestStats(); - server = new KvStateServerImpl(InetAddress.getLocalHost(), - Collections.singletonList(0).iterator(), 1, 1, registry, stats); + server = new KvStateServerImpl( + InetAddress.getLocalHost(), + Collections.singletonList(0).iterator(), + 1, + 1, + registry, + stats); server.start(); - KvStateServerAddress serverAddress = server.getServerAddress(); + InetSocketAddress serverAddress = server.getServerAddress(); int numKeyGroups = 1; AbstractStateBackend abstractBackend = new MemoryStateBackend(); DummyEnvironment dummyEnv = new DummyEnvironment("test", 1, 0); @@ -151,7 +156,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception }); Channel channel = bootstrap - .connect(serverAddress.getHost(), serverAddress.getPort()) + .connect(serverAddress.getAddress(), serverAddress.getPort()) .sync().channel(); long requestId = Integer.MAX_VALUE + 182828L; 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-runtime/src/test/java/org/apache/flink/queryablestate/network/MessageSerializerTest.java similarity index 99% rename from flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/MessageSerializerTest.java rename to flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/MessageSerializerTest.java index 32a0c9b82b96f..acaa0671bc056 100644 --- 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-runtime/src/test/java/org/apache/flink/queryablestate/network/MessageSerializerTest.java @@ -18,12 +18,12 @@ 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.runtime.query.KvStateID; import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator; 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 0000000000000..10792cd2949cc --- /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 diff --git a/flink-queryable-state/pom.xml b/flink-queryable-state/pom.xml index e2579f694423d..9300fb3be7b20 100644 --- a/flink-queryable-state/pom.xml +++ b/flink-queryable-state/pom.xml @@ -35,8 +35,9 @@ under the License. pom - flink-queryable-state-java - + flink-queryable-state-runtime + flink-queryable-state-client-java + diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml index 83ac781799f14..134c41468e8fe 100644 --- a/flink-runtime/pom.xml +++ b/flink-runtime/pom.xml @@ -50,6 +50,12 @@ under the License. ${project.version} + + org.apache.flink + flink-queryable-state-client-java_${scala.binary.version} + ${project.version} + + 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 f60f56110f76f..4535290725193 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.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.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 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 void notifyKvStateRegistered( 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 {}.", 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 d59feedee85d9..2c7e438f92105 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.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.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 @@ void notifyKvStateRegistered( final KeyGroupRange keyGroupRange, final String registrationName, final KvStateID kvStateId, - final KvStateServerAddress kvStateServerAddress); + final InetSocketAddress kvStateServerAddress); /** * Notifies that queryable state has been unregistered. 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 03e82389ba9cb..e4fdda54d0975 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 KvStateLocation(JobID jobId, JobVertexID jobVertexId, int numKeyGroups, S 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 KvStateID getKvStateID(int keyGroupIndex) { } /** - * Returns the registered KvStateServerAddress for the key group index or + * Returns the registered server address for the key group index or * null if none is registered yet. * * @param keyGroupIndex Key group index to get server address for. - * @return KvStateServerAddress for the key group index or null + * @return the server address for the key group index or null * 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 KvStateServerAddress getKvStateServerAddress(int keyGroupIndex) { * @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"); 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 cb619054adad3..05ee017e3b176 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.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 void notifyKvStateRegistered( KeyGroupRange keyGroupRange, String registrationName, KvStateID kvStateId, - KvStateServerAddress kvStateServerAddress) { + InetSocketAddress kvStateServerAddress) { KvStateLocation location = lookupTable.get(registrationName); 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 04684eedb521f..e94d2f8348c15 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 @@ class NotifyKvStateRegistered implements KvStateMessage { 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 NotifyKvStateRegistered( 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 NotifyKvStateRegistered( 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 KvStateID getKvStateId() { * * @return Server address where to find the KvState instance */ - public KvStateServerAddress getKvStateServerAddress() { + public InetSocketAddress getKvStateServerAddress() { return kvStateServerAddress; } 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 90fa5ccabee51..af19d8161ab74 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; 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 13862c98d1443..4b9834a28f0b4 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 @@ void notifyKvStateRegistered( KeyGroupRange keyGroupRange, String registrationName, KvStateID kvStateId, - KvStateServerAddress kvStateServerAddress); + InetSocketAddress kvStateServerAddress); /** * Notifies the listener about an unregistered KvState instance. 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 29bee9a92c9a2..dc90c96d854a1 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; 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 17ffe0d1de333..ae587144faec4 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 @@ 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. */ 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 259985550a0ad..0000000000000 --- 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; - } -} 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 fa021dff08cc1..adbe15d90675b 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 static KvStateClientProxy createKvStateClientProxy( 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 static KvStateServer createKvStateServer( 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) { 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 8d0eede59610a..f799b5a49174c 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; 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 97b6bcd2281be..66360e4e9fffb 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.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 @@ public abstract class AbstractHeapState> implements InternalKvState { - /** Map containing the actual key/value pairs */ + /** Map containing the actual key/value pairs. */ protected final StateTable stateTable; /** This holds the name of the state and can create an initial default value for the state. */ @@ -118,4 +118,4 @@ public byte[] getSerializedValue(K key, N namespace) throws Exception { public StateTable getStateTable() { return stateTable; } -} \ No newline at end of file +} 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 f981b9ffdb0bf..206f10ac2507d 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 @@ 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; 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 1cc94d2a61822..2baf644b61a8c 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 @@ 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.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; 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 3692a71c7dd74..6312d085943cf 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); } 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 4404867acfa11..63bda99b29316 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 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 68da36215d461..889191f1bd10d 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.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.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 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 void testKvStateMessages() throws Exception { 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 void testKvStateMessages() throws Exception { 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 void testKvStateMessages() throws Exception { 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 void testKvStateMessages() throws Exception { new KeyGroupRange(0, 0), "duplicate-me", // ...same name new KvStateID(), - new KvStateServerAddress(InetAddress.getLocalHost(), 1293)); + new InetSocketAddress(InetAddress.getLocalHost(), 1293)); Future failedFuture = jobManager .ask(new NotifyWhenJobStatus(jobGraph.getJobID(), JobStatus.FAILED), deadline.timeLeft()) 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 7bf9ee73507d9..74e16a02aa2b9 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.junit.Test; import java.net.InetAddress; +import java.net.InetSocketAddress; import java.util.HashMap; import java.util.Map; @@ -63,7 +65,7 @@ public void testRegisterAndLookup() throws Exception { } } - KvStateServerAddress server = new KvStateServerAddress(InetAddress.getLocalHost(), 12032); + InetSocketAddress server = new InetSocketAddress(InetAddress.getLocalHost(), 12032); // Create registry Map vertexMap = createVertexMap(vertices); @@ -129,7 +131,7 @@ public void testRegisterDuplicateName() throws Exception { 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 void testRegisterDuplicateName() throws Exception { 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 void testUnregisterFailures() throws Exception { new KeyGroupRange(0, 0), name, new KvStateID(), - mock(KvStateServerAddress.class)); + mock(InetSocketAddress.class)); try { // Unregister not registered keyGroupIndex 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 116deea78d215..3c79948ecd155 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 void testRegisterAndLookup() throws Exception { 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 void testRegisterAndLookup() throws Exception { 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 void testRegisterAndLookup() throws Exception { // 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()); 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 8f803ed660198..0a4b5bfec1d81 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.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; 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 91566af84f547..95e159820d731 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 void testTypeInfoTestCoverage() { 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; diff --git a/pom.xml b/pom.xml index 9384a48fe8472..e4a49d4d7b99a 100644 --- a/pom.xml +++ b/pom.xml @@ -68,6 +68,7 @@ under the License. flink-connectors flink-examples flink-clients + flink-queryable-state flink-tests flink-end-to-end-tests flink-test-utils-parent @@ -81,7 +82,6 @@ under the License. flink-yarn flink-yarn-tests flink-fs-tests - flink-queryable-state diff --git a/tools/travis_mvn_watchdog.sh b/tools/travis_mvn_watchdog.sh index f608e5140d879..b7e3709f90b9f 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,\ From 2fd8721d0375bfa60a190bb206a65287c167a43a Mon Sep 17 00:00:00 2001 From: kkloudas Date: Thu, 26 Oct 2017 15:07:47 +0200 Subject: [PATCH 2/2] [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. --- flink-dist/src/main/assemblies/opt.xml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/flink-dist/src/main/assemblies/opt.xml b/flink-dist/src/main/assemblies/opt.xml index 58aee3d2fe6d6..14ec174ad3442 100644 --- a/flink-dist/src/main/assemblies/opt.xml +++ b/flink-dist/src/main/assemblies/opt.xml @@ -138,5 +138,13 @@ flink-s3-fs-presto-${project.version}.jar 0644 + + + + ../flink-queryable-state/flink-queryable-state-runtime/target/flink-queryable-state-runtime_${scala.binary.version}-${project.version}.jar + opt/ + flink-queryable-state-runtime_${scala.binary.version}-${project.version}.jar + 0644 +