Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
8 changes: 8 additions & 0 deletions flink-dist/src/main/assemblies/opt.xml
Original file line number Diff line number Diff line change
Expand Up @@ -138,5 +138,13 @@
<destName>flink-s3-fs-presto-${project.version}.jar</destName>
<fileMode>0644</fileMode>
</file>

<!-- Queryable State -->
<file>
<source>../flink-queryable-state/flink-queryable-state-runtime/target/flink-queryable-state-runtime_${scala.binary.version}-${project.version}.jar</source>
<outputDirectory>opt/</outputDirectory>
<destName>flink-queryable-state-runtime_${scala.binary.version}-${project.version}.jar</destName>
<fileMode>0644</fileMode>
</file>
</files>
</assembly>
90 changes: 90 additions & 0 deletions flink-queryable-state/flink-queryable-state-client-java/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,90 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
Licensed to the Apache Software Foundation (ASF) under one
or more contributor license agreements. See the NOTICE file
distributed with this work for additional information
regarding copyright ownership. The ASF licenses this file
to you under the Apache License, Version 2.0 (the
"License"); you may not use this file except in compliance
with the License. You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing,
software distributed under the License is distributed on an
"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
KIND, either express or implied. See the License for the
specific language governing permissions and limitations
under the License.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">

<modelVersion>4.0.0</modelVersion>

<parent>
<groupId>org.apache.flink</groupId>
<artifactId>flink-queryable-state</artifactId>
<version>1.4-SNAPSHOT</version>
<relativePath>..</relativePath>
</parent>

<artifactId>flink-queryable-state-client-java_${scala.binary.version}</artifactId>
<name>flink-queryable-state-client-java</name>
<packaging>jar</packaging>

<dependencies>

<!-- core dependencies -->

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-core</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-shaded-netty</artifactId>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-shaded-guava</artifactId>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-core</artifactId>
<version>${project.version}</version>
<scope>test</scope>
<type>test-jar</type>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>

</dependencies>

<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>test-jar</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.queryablestate;

import java.util.concurrent.CompletableFuture;

/**
* Utility class for {@link java.util.concurrent.Future Java Futures}.
*/
public class FutureUtils {

// ------------------------------------------------------------------------
// Future Completed with an exception.
// ------------------------------------------------------------------------

/**
* Returns a {@link CompletableFuture} that has failed with the exception
* provided as argument.
* @param throwable the exception to fail the future with.
* @return The failed future.
*/
public static <T> CompletableFuture<T> getFailedFuture(Throwable throwable) {
CompletableFuture<T> failedAttempt = new CompletableFuture<>();
failedAttempt.completeExceptionally(throwable);
return failedAttempt;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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.
*
* <p>Assigned when registering state at the {@link KvStateRegistry}.
* <p>Assigned when registering the state at the state registry.
*/
public class KvStateID extends AbstractID {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand All @@ -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;

Expand All @@ -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.
*
* <p>The client connects to a {@link org.apache.flink.runtime.query.KvStateClientProxy Client Proxy}
* running on a given Task Manager. The proxy is the entry point of the client to the Flink cluster.
* It forwards the requests of the client to the Job Manager and the required Task Manager, and forwards
* the final response back the client.
* <p>The client connects to a {@code Client Proxy} running on a given Task Manager. The
* proxy is the entry point of the client to the Flink cluster. It forwards the requests
* of the client to the Job Manager and the required Task Manager, and forwards the final
* response back the client.
*
* <p>The proxy, initially resolves the location of the requested KvState via the JobManager. Resolved
* locations are cached. When the server address of the requested KvState instance is determined, the
Expand All @@ -74,15 +71,14 @@ public class QueryableStateClient {
private final Client<KvStateRequest, KvStateResponse> 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 {
Expand All @@ -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<KvStateRequest, KvStateResponse> messageSerializer =
new MessageSerializer<>(
Expand All @@ -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());
}
Expand Down Expand Up @@ -213,6 +208,8 @@ public <K, N, S extends State, V> CompletableFuture<S> getKvState(
TypeSerializer<K> keySerializer = keyTypeInfo.createSerializer(executionConfig);
TypeSerializer<N> namespaceSerializer = namespaceTypeInfo.createSerializer(executionConfig);

stateDescriptor.initializeSerializerUnlessSet(executionConfig);

final byte[] serializedKeyAndNamespace;
try {
serializedKeyAndNamespace = KvStateSerializer
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.queryablestate.client;

import org.apache.flink.annotation.Internal;

import java.io.ObjectStreamException;

/**
* Singleton placeholder class for state without a namespace.
*
* <p><b>THIS WAS COPIED FROM RUNTIME SO THAT WE AVOID THE DEPENDENCY.</b>
*/
@Internal
public final class VoidNamespace {

// ------------------------------------------------------------------------
// Singleton instance.
// ------------------------------------------------------------------------

/** The singleton instance. */
public static final VoidNamespace INSTANCE = new VoidNamespace();

/** Getter for the singleton instance. */
public static VoidNamespace get() {
return INSTANCE;
}

/** This class should not be instantiated. */
private VoidNamespace() {}

// ------------------------------------------------------------------------
// Standard Utilities
// ------------------------------------------------------------------------

@Override
public int hashCode() {
return 99;
}

@Override
public boolean equals(Object obj) {
return obj == this;
}

@Override
public String toString() {
return getClass().getSimpleName();
}

// ------------------------------------------------------------------------
// Singleton serialization
// ------------------------------------------------------------------------

// make sure that we preserve the singleton properly on serialization
private Object readResolve() throws ObjectStreamException {
return INSTANCE;
}
}
Loading