Skip to content

Commit

Permalink
[FLINK-35389][State/ForSt] Async list state (apache#25126)
Browse files Browse the repository at this point in the history
  • Loading branch information
Zakelly authored Sep 3, 2024
1 parent 2c16d88 commit c58a6eb
Show file tree
Hide file tree
Showing 18 changed files with 666 additions and 58 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -91,4 +91,8 @@ public StateDescriptor<V> getStateDescriptor() {
public TypeSerializer<V> getValueSerializer() {
return stateDescriptor.getSerializer();
}

public StateRequestHandler getStateRequestHandler() {
return stateRequestHandler;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,16 +28,18 @@
* The Get access request for ForStDB.
*
* @param <K> The type of key in get access request.
* @param <N> The type of namespace in put access request.
* @param <V> The type of value returned by get request.
* @param <R> The type of returned value in state future.
*/
public class ForStDBGetRequest<K, N, V> {
public abstract class ForStDBGetRequest<K, N, V, R> {

private final ContextKey<K, N> key;
private final ForStInnerTable<K, N, V> table;
private final InternalStateFuture<V> future;
final ContextKey<K, N> key;
final ForStInnerTable<K, N, V> table;
final InternalStateFuture<R> future;

private ForStDBGetRequest(
ContextKey<K, N> key, ForStInnerTable<K, N, V> table, InternalStateFuture<V> future) {
ForStDBGetRequest(
ContextKey<K, N> key, ForStInnerTable<K, N, V> table, InternalStateFuture<R> future) {
this.key = key;
this.table = table;
this.future = future;
Expand All @@ -51,21 +53,9 @@ public ColumnFamilyHandle getColumnFamilyHandle() {
return table.getColumnFamilyHandle();
}

public void completeStateFuture(byte[] bytesValue) throws IOException {
if (bytesValue == null) {
future.complete(null);
return;
}
V value = table.deserializeValue(bytesValue);
future.complete(value);
}
public abstract void completeStateFuture(byte[] bytesValue) throws IOException;

public void completeStateFutureExceptionally(String message, Throwable ex) {
future.completeExceptionally(message, ex);
}

static <K, N, V> ForStDBGetRequest<K, N, V> of(
ContextKey<K, N> key, ForStInnerTable<K, N, V> table, InternalStateFuture<V> future) {
return new ForStDBGetRequest<>(key, table, future);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
/*
* 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.state.forst;

import org.apache.flink.api.common.state.v2.StateIterator;
import org.apache.flink.core.state.InternalStateFuture;
import org.apache.flink.runtime.asyncprocessing.StateRequestType;

import java.io.IOException;
import java.util.List;

/**
* The Get access request for ForStDB.
*
* @param <K> The type of key in get access request.
* @param <V> The type of value returned by get request.
*/
public class ForStDBListGetRequest<K, N, V>
extends ForStDBGetRequest<K, N, List<V>, StateIterator<V>> {

ForStDBListGetRequest(
ContextKey<K, N> key,
ForStInnerTable<K, N, List<V>> table,
InternalStateFuture<StateIterator<V>> future) {
super(key, table, future);
}

@Override
public void completeStateFuture(byte[] bytesValue) throws IOException {
if (bytesValue == null) {
future.complete(null);
return;
}
List<V> value = table.deserializeValue(bytesValue);
future.complete(
new ForStListIterator<>(
(ForStListState) table,
StateRequestType.LIST_GET,
((ForStListState) table).getStateRequestHandler(),
value));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,22 +30,26 @@
* The Put access request for ForStDB.
*
* @param <K> The type of key in put access request.
* @param <N> The type of namespace in put access request.
* @param <V> The type of value in put access request.
*/
public class ForStDBPutRequest<K, N, V> {

private final ContextKey<K, N> key;
@Nullable private final V value;
private final ForStInnerTable<K, N, V> table;
private final InternalStateFuture<Void> future;
final ContextKey<K, N> key;
@Nullable final V value;
final boolean isMerge;
final ForStInnerTable<K, N, V> table;
final InternalStateFuture<Void> future;

private ForStDBPutRequest(
ForStDBPutRequest(
ContextKey<K, N> key,
V value,
boolean isMerge,
ForStInnerTable<K, N, V> table,
InternalStateFuture<Void> future) {
this.key = key;
this.value = value;
this.isMerge = isMerge;
this.table = table;
this.future = future;
}
Expand All @@ -54,6 +58,10 @@ public boolean valueIsNull() {
return value == null;
}

public boolean isMerge() {
return isMerge;
}

public ColumnFamilyHandle getColumnFamilyHandle() {
return table.getColumnFamilyHandle();
}
Expand Down Expand Up @@ -84,6 +92,14 @@ static <K, N, V> ForStDBPutRequest<K, N, V> of(
@Nullable V value,
ForStInnerTable<K, N, V> table,
InternalStateFuture<Void> future) {
return new ForStDBPutRequest<>(key, value, table, future);
return new ForStDBPutRequest<>(key, value, false, table, future);
}

static <K, N, V> ForStDBPutRequest<K, N, V> ofMerge(
ContextKey<K, N> key,
@Nullable V value,
ForStInnerTable<K, N, V> table,
InternalStateFuture<Void> future) {
return new ForStDBPutRequest<>(key, value, true, table, future);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.state.forst;

import org.apache.flink.core.state.InternalStateFuture;

import java.io.IOException;

/**
* The Get access request for ForStDB.
*
* @param <K> The type of key in get access request.
* @param <V> The type of value returned by get request.
*/
public class ForStDBSingleGetRequest<K, N, V> extends ForStDBGetRequest<K, N, V, V> {

ForStDBSingleGetRequest(
ContextKey<K, N> key, ForStInnerTable<K, N, V> table, InternalStateFuture<V> future) {
super(key, table, future);
}

@Override
public void completeStateFuture(byte[] bytesValue) throws IOException {
if (bytesValue == null) {
future.complete(null);
return;
}
V value = table.deserializeValue(bytesValue);
future.complete(value);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -38,12 +38,12 @@ public class ForStGeneralMultiGetOperation implements ForStDBOperation {

private final RocksDB db;

private final List<ForStDBGetRequest<?, ?, ?>> batchRequest;
private final List<ForStDBGetRequest<?, ?, ?, ?>> batchRequest;

private final Executor executor;

ForStGeneralMultiGetOperation(
RocksDB db, List<ForStDBGetRequest<?, ?, ?>> batchRequest, Executor executor) {
RocksDB db, List<ForStDBGetRequest<?, ?, ?, ?>> batchRequest, Executor executor) {
this.db = db;
this.batchRequest = batchRequest;
this.executor = executor;
Expand All @@ -58,7 +58,7 @@ public CompletableFuture<Void> process() {
AtomicReference<Exception> error = new AtomicReference<>();
AtomicInteger counter = new AtomicInteger(batchRequest.size());
for (int i = 0; i < batchRequest.size(); i++) {
ForStDBGetRequest<?, ?, ?> request = batchRequest.get(i);
ForStDBGetRequest<?, ?, ?, ?> request = batchRequest.get(i);
executor.execute(
() -> {
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,7 @@ public interface ForStInnerTable<K, N, V> {
* @param stateRequest The given stateRequest.
* @return The corresponding ForSt GetRequest.
*/
ForStDBGetRequest<K, N, V> buildDBGetRequest(StateRequest<?, ?, ?> stateRequest);
ForStDBGetRequest<?, ?, ?, ?> buildDBGetRequest(StateRequest<?, ?, ?> stateRequest);

/**
* Build a {@link ForStDBPutRequest} that belong to {@code ForStInnerTable} with the given
Expand All @@ -82,5 +82,5 @@ public interface ForStInnerTable<K, N, V> {
* @param stateRequest The given stateRequest.
* @return The corresponding ForSt PutRequest.
*/
ForStDBPutRequest<K, N, V> buildDBPutRequest(StateRequest<?, ?, ?> stateRequest);
ForStDBPutRequest<?, ?, ?> buildDBPutRequest(StateRequest<?, ?, ?> stateRequest);
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.flink.runtime.asyncprocessing.StateRequestHandler;
import org.apache.flink.runtime.state.AsyncKeyedStateBackend;
import org.apache.flink.runtime.state.SerializedCompositeKeyBuilder;
import org.apache.flink.runtime.state.v2.ListStateDescriptor;
import org.apache.flink.runtime.state.v2.StateDescriptor;
import org.apache.flink.runtime.state.v2.ValueStateDescriptor;
import org.apache.flink.util.FlinkRuntimeException;
Expand Down Expand Up @@ -149,20 +150,33 @@ public <N, S extends State, SV> S createState(
ColumnFamilyHandle columnFamilyHandle =
ForStOperationUtils.createColumnFamilyHandle(
stateDesc.getStateId(), db, columnFamilyOptionsFactory);
if (stateDesc.getType() == StateDescriptor.Type.VALUE) {
return (S)
new ForStValueState<>(
stateRequestHandler,
columnFamilyHandle,
(ValueStateDescriptor<SV>) stateDesc,
serializedKeyBuilder,
defaultNamespace,
namespaceSerializer::duplicate,
valueSerializerView,
valueDeserializerView);
switch (stateDesc.getType()) {
case VALUE:
return (S)
new ForStValueState<>(
stateRequestHandler,
columnFamilyHandle,
(ValueStateDescriptor<SV>) stateDesc,
serializedKeyBuilder,
defaultNamespace,
namespaceSerializer::duplicate,
valueSerializerView,
valueDeserializerView);
case LIST:
return (S)
new ForStListState<>(
stateRequestHandler,
columnFamilyHandle,
(ListStateDescriptor<SV>) stateDesc,
serializedKeyBuilder,
defaultNamespace,
namespaceSerializer::duplicate,
valueSerializerView,
valueDeserializerView);
default:
throw new UnsupportedOperationException(
String.format("Unsupported state type: %s", stateDesc.getType()));
}
throw new UnsupportedOperationException(
String.format("Unsupported state type: %s", stateDesc.getType()));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
/*
* 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.state.forst;

import org.apache.flink.api.common.state.v2.State;
import org.apache.flink.runtime.asyncprocessing.AbstractStateIterator;
import org.apache.flink.runtime.asyncprocessing.StateRequestHandler;
import org.apache.flink.runtime.asyncprocessing.StateRequestType;

import java.util.Collection;

/** The forst implementation for list iterator. */
public class ForStListIterator<V> extends AbstractStateIterator<V> {

public ForStListIterator(
State originalState,
StateRequestType requestType,
StateRequestHandler stateHandler,
Collection<V> partialResult) {
super(originalState, requestType, stateHandler, partialResult);
}

@Override
protected boolean hasNext() {
return false;
}

@Override
protected Object nextPayloadForContinuousLoading() {
return null;
}
}
Loading

0 comments on commit c58a6eb

Please sign in to comment.