Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-35262] Bridge between AsyncKeyedStateBackend and AsyncExecutionController #24740

Closed
wants to merge 2 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -320,8 +320,8 @@ public void drainInflightRecords(int targetNum) {
}

@VisibleForTesting
public void setStateExecutor(StateExecutor stateExecutor) {
this.stateExecutor = stateExecutor;
public StateExecutor getStateExecutor() {
return stateExecutor;
}

@VisibleForTesting
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,15 +19,40 @@
package org.apache.flink.runtime.state;

import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.state.v2.State;
import org.apache.flink.runtime.asyncprocessing.StateExecutor;
import org.apache.flink.runtime.asyncprocessing.StateRequestHandler;
import org.apache.flink.runtime.state.v2.StateDescriptor;
import org.apache.flink.util.Disposable;

import javax.annotation.Nonnull;

import java.io.Closeable;

/**
* An async keyed state backend provides methods supporting to access keyed state asynchronously and
* in batch.
*/
@Internal
public interface AsyncKeyedStateBackend extends Disposable {
public interface AsyncKeyedStateBackend extends Disposable, Closeable {

/**
* Initializes with some contexts.
*
* @param stateRequestHandler which handles state request.
*/
void setup(@Nonnull StateRequestHandler stateRequestHandler);

/**
* Creates and returns a new state.
*
* @param stateDesc The {@code StateDescriptor} that contains the name of the state.
* @param <SV> The type of the stored state value.
* @param <S> The type of the public API state.
* @throws Exception Exceptions may occur during initialization of the state.
*/
@Nonnull
<SV, S extends State> S createState(@Nonnull StateDescriptor<SV> stateDesc) throws Exception;

/**
* Creates a {@code StateExecutor} which supports to execute a batch of state requests
Expand All @@ -36,5 +61,9 @@ public interface AsyncKeyedStateBackend extends Disposable {
* @return a {@code StateExecutor} which supports to execute a batch of state requests
* asynchronously.
*/
@Nonnull
StateExecutor createStateExecutor();

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

package org.apache.flink.runtime.state.v2;

import org.apache.flink.api.common.state.v2.ValueState;
import org.apache.flink.runtime.state.AsyncKeyedStateBackend;
import org.apache.flink.util.Preconditions;

import javax.annotation.Nonnull;

/** Default implementation of KeyedStateStoreV2. */
public class DefaultKeyedStateStoreV2 implements KeyedStateStoreV2 {

private final AsyncKeyedStateBackend asyncKeyedStateBackend;

public DefaultKeyedStateStoreV2(@Nonnull AsyncKeyedStateBackend asyncKeyedStateBackend) {
this.asyncKeyedStateBackend = Preconditions.checkNotNull(asyncKeyedStateBackend);
}

@Override
public <T> ValueState<T> getValueState(@Nonnull ValueStateDescriptor<T> stateProperties) {
Preconditions.checkNotNull(stateProperties, "The state properties must not be null");
try {
return asyncKeyedStateBackend.createState(stateProperties);
} catch (Exception e) {
throw new RuntimeException("Error while getting state", e);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
/*
* 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.state.v2;

import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.state.v2.State;
import org.apache.flink.api.common.state.v2.ValueState;

import javax.annotation.Nonnull;

/** This interface contains methods for registering {@link State}. */
@Internal
public interface KeyedStateStoreV2 {

/**
* Gets a handle to the system's {@link ValueState}. The key/value state is only accessible if
* the function is executed on a KeyedStream. On each access, the state exposes the value for
* the key of the element currently processed by the function. Each function may have multiple
* partitioned states, addressed with different names.
*
* <p>Because the scope of each value is the key of the currently processed element, and the
* elements are distributed by the Flink runtime, the system can transparently scale out and
* redistribute the state and KeyedStream.
*
* @param stateProperties The descriptor defining the properties of the state.
* @param <T> The type of value stored in the state.
* @return The partitioned state object.
*/
<T> ValueState<T> getValueState(@Nonnull ValueStateDescriptor<T> stateProperties);
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,15 +18,15 @@

package org.apache.flink.runtime.asyncprocessing;

import org.apache.flink.api.common.state.v2.State;
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
import org.apache.flink.api.common.typeutils.base.IntSerializer;
import org.apache.flink.core.state.StateFutureUtils;
import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutorService;
import org.apache.flink.runtime.mailbox.SyncMailboxExecutor;
import org.apache.flink.runtime.state.AsyncKeyedStateBackend;
import org.apache.flink.runtime.state.CheckpointableKeyedStateBackend;
import org.apache.flink.runtime.state.OperatorStateBackend;
import org.apache.flink.runtime.state.StateBackend;
import org.apache.flink.runtime.state.StateBackendTestUtils;
import org.apache.flink.runtime.state.v2.InternalValueState;
import org.apache.flink.runtime.state.v2.ValueStateDescriptor;
import org.apache.flink.util.Preconditions;
Expand All @@ -37,13 +37,13 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Supplier;

import static org.assertj.core.api.AssertionsForClassTypes.assertThat;

/** Test for {@link AsyncExecutionController}. */
class AsyncExecutionControllerTest {
AsyncExecutionController aec;
TestUnderlyingState underlyingState;
AtomicInteger output;
TestValueState valueState;

Expand All @@ -63,16 +63,36 @@ class AsyncExecutionControllerTest {
};

void setup(int batchSize, long timeout, int maxInFlight) {
StateExecutor stateExecutor = new TestStateExecutor();
ValueStateDescriptor<Integer> stateDescriptor =
new ValueStateDescriptor<>("test-value-state", BasicTypeInfo.INT_TYPE_INFO);
Supplier<State> stateSupplier =
() -> new TestValueState(aec, new TestUnderlyingState(), stateDescriptor);
StateBackend testAsyncStateBackend =
StateBackendTestUtils.buildAsyncStateBackend(stateSupplier, stateExecutor);
assertThat(testAsyncStateBackend.supportsAsyncKeyedStateBackend()).isTrue();
AsyncKeyedStateBackend asyncKeyedStateBackend;
try {
asyncKeyedStateBackend = testAsyncStateBackend.createAsyncKeyedStateBackend(null);
} catch (Exception e) {
throw new RuntimeException(e);
}
aec =
new AsyncExecutionController<>(
new SyncMailboxExecutor(),
createStateExecutor(),
stateExecutor,
128,
batchSize,
timeout,
maxInFlight);
underlyingState = new TestUnderlyingState();
valueState = new TestValueState(aec, underlyingState);
asyncKeyedStateBackend.setup(aec);

try {
valueState = asyncKeyedStateBackend.createState(stateDescriptor);
} catch (Exception e) {
throw new RuntimeException(e);
}

output = new AtomicInteger();
}

Expand Down Expand Up @@ -466,12 +486,6 @@ void testBufferTimeoutSkip() {
assertThat(aec.stateRequestsBuffer.currentScheduledFuture.isDone()).isTrue();
}

private StateExecutor createStateExecutor() {
TestAsyncStateBackend testAsyncStateBackend = new TestAsyncStateBackend();
assertThat(testAsyncStateBackend.supportsAsyncKeyedStateBackend()).isTrue();
return testAsyncStateBackend.createAsyncKeyedStateBackend(null).createStateExecutor();
}

/** Simulate the underlying state that is actually used to execute the request. */
static class TestUnderlyingState {

Expand All @@ -495,53 +509,15 @@ static class TestValueState extends InternalValueState<String, Integer> {
private final TestUnderlyingState underlyingState;

public TestValueState(
AsyncExecutionController<String> aec, TestUnderlyingState underlyingState) {
super(aec, new ValueStateDescriptor<>("test-value-state", BasicTypeInfo.INT_TYPE_INFO));
StateRequestHandler stateRequestHandler,
TestUnderlyingState underlyingState,
ValueStateDescriptor<Integer> stateDescriptor) {
super(stateRequestHandler, stateDescriptor);
this.underlyingState = underlyingState;
assertThat(this.getValueSerializer()).isEqualTo(IntSerializer.INSTANCE);
}
}

/**
* A brief implementation of {@link StateBackend} which illustrates the interaction between AEC
* and StateBackend.
*/
static class TestAsyncStateBackend implements StateBackend {

@Override
public <K> CheckpointableKeyedStateBackend<K> createKeyedStateBackend(
KeyedStateBackendParameters<K> parameters) throws Exception {
throw new UnsupportedOperationException("Don't support createKeyedStateBackend yet");
}

@Override
public OperatorStateBackend createOperatorStateBackend(
OperatorStateBackendParameters parameters) throws Exception {
throw new UnsupportedOperationException("Don't support createOperatorStateBackend yet");
}

@Override
public boolean supportsAsyncKeyedStateBackend() {
return true;
}

@Override
public <K> AsyncKeyedStateBackend createAsyncKeyedStateBackend(
KeyedStateBackendParameters<K> parameters) {
return new AsyncKeyedStateBackend() {
@Override
public StateExecutor createStateExecutor() {
return new TestStateExecutor();
}

@Override
public void dispose() {
// do nothing
}
};
}
}

/**
* A brief implementation of {@link StateExecutor}, to illustrate the interaction between AEC
* and StateExecutor.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,12 +16,8 @@
* limitations under the License.
*/

package org.apache.flink.streaming.runtime.operators.asyncprocessing;
package org.apache.flink.runtime.asyncprocessing;

import org.apache.flink.runtime.asyncprocessing.MockStateRequestContainer;
import org.apache.flink.runtime.asyncprocessing.StateExecutor;
import org.apache.flink.runtime.asyncprocessing.StateRequest;
import org.apache.flink.runtime.asyncprocessing.StateRequestContainer;
import org.apache.flink.util.Preconditions;

import java.util.concurrent.CompletableFuture;
Expand Down