Skip to content

[FLINK-37552] Support switching between ForStKeyedStatebackend and ForStSyncKeyedStatebackend #26466

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

Merged
merged 3 commits into from
May 11, 2025
Merged
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
@@ -102,7 +102,7 @@ public RegisteredKeyValueStateBackendMetaInfo(@Nonnull StateMetaInfoSnapshot sna
StateMetaInfoSnapshot.BackendStateType.KEY_VALUE == snapshot.getBackendStateType());
}

private RegisteredKeyValueStateBackendMetaInfo(
public RegisteredKeyValueStateBackendMetaInfo(
@Nonnull StateDescriptor.Type stateType,
@Nonnull String name,
@Nonnull StateSerializerProvider<N> namespaceSerializerProvider,
@@ -287,4 +287,19 @@ private StateMetaInfoSnapshot computeSnapshot() {
serializerConfigSnapshotsMap,
serializerMap);
}

public static RegisteredStateMetaInfoBase fromMetaInfoSnapshot(
@Nonnull StateMetaInfoSnapshot snapshot) {

final StateMetaInfoSnapshot.BackendStateType backendStateType =
snapshot.getBackendStateType();
switch (backendStateType) {
case KEY_VALUE:
return new RegisteredKeyValueStateBackendMetaInfo<>(snapshot);
case KEY_VALUE_V2:
return RegisteredStateMetaInfoUtils.createMetaInfoV1FromV2Snapshot(snapshot);
default:
return RegisteredStateMetaInfoBase.fromMetaInfoSnapshot(snapshot);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,174 @@
/*
* 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;

import org.apache.flink.api.common.state.v2.StateDescriptor;
import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
import org.apache.flink.api.common.typeutils.base.ListSerializer;
import org.apache.flink.api.common.typeutils.base.MapSerializer;
import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot;
import org.apache.flink.util.Preconditions;

/**
* Utilities for transforming {@link RegisteredKeyValueStateBackendMetaInfo} to {@link
* org.apache.flink.runtime.state.v2.RegisteredKeyValueStateBackendMetaInfo}.
*/
public class RegisteredStateMetaInfoUtils {

private RegisteredStateMetaInfoUtils() {}

public static org.apache.flink.runtime.state.v2.RegisteredKeyValueStateBackendMetaInfo
createMetaInfoV2FromV1Snapshot(StateMetaInfoSnapshot stateMeta) {

Preconditions.checkArgument(
stateMeta
.getBackendStateType()
.equals(StateMetaInfoSnapshot.BackendStateType.KEY_VALUE));

StateDescriptor.Type newStateType;
TypeSerializerSnapshot oldValueSerializerSnapshot =
stateMeta.getTypeSerializerSnapshot(
StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER);
TypeSerializerSnapshot newUserKeySerializerSnapshot = null;
TypeSerializerSnapshot newStateSerializerSnapshot = oldValueSerializerSnapshot;

switch (org.apache.flink.api.common.state.StateDescriptor.Type.valueOf(
stateMeta.getOption(StateMetaInfoSnapshot.CommonOptionsKeys.KEYED_STATE_TYPE))) {
case VALUE:
newStateType = StateDescriptor.Type.VALUE;
break;
case REDUCING:
newStateType = StateDescriptor.Type.REDUCING;
break;
case AGGREGATING:
newStateType = StateDescriptor.Type.AGGREGATING;
break;
case LIST:
newStateType = StateDescriptor.Type.LIST;
newStateSerializerSnapshot =
((ListSerializer) oldValueSerializerSnapshot.restoreSerializer())
.getElementSerializer()
.snapshotConfiguration();
break;
case MAP:
newStateType = StateDescriptor.Type.MAP;
newStateSerializerSnapshot =
((MapSerializer) oldValueSerializerSnapshot.restoreSerializer())
.getValueSerializer()
.snapshotConfiguration();
newUserKeySerializerSnapshot =
((MapSerializer) oldValueSerializerSnapshot.restoreSerializer())
.getKeySerializer()
.snapshotConfiguration();
break;
default:
throw new UnsupportedOperationException(
"Unsupported state type to transform from org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo to org.apache.flink.runtime.state.v2.RegisteredKeyValueStateBackendMetaInfo");
}

if (!newStateType.equals(StateDescriptor.Type.MAP)) {
return new org.apache.flink.runtime.state.v2.RegisteredKeyValueStateBackendMetaInfo<>(
stateMeta.getName(),
newStateType,
StateSerializerProvider.fromPreviousSerializerSnapshot(
stateMeta.getTypeSerializerSnapshot(
StateMetaInfoSnapshot.CommonSerializerKeys
.NAMESPACE_SERIALIZER)),
StateSerializerProvider.fromPreviousSerializerSnapshot(
newStateSerializerSnapshot),
StateSnapshotTransformer.StateSnapshotTransformFactory.noTransform());
} else {
return new org.apache.flink.runtime.state.v2
.RegisteredKeyAndUserKeyValueStateBackendMetaInfo<>(
stateMeta.getName(),
newStateType,
StateSerializerProvider.fromPreviousSerializerSnapshot(
stateMeta.getTypeSerializerSnapshot(
StateMetaInfoSnapshot.CommonSerializerKeys
.NAMESPACE_SERIALIZER)),
StateSerializerProvider.fromPreviousSerializerSnapshot(
newStateSerializerSnapshot),
StateSerializerProvider.fromPreviousSerializerSnapshot(
newUserKeySerializerSnapshot),
StateSnapshotTransformer.StateSnapshotTransformFactory.noTransform());
}
}

public static RegisteredKeyValueStateBackendMetaInfo createMetaInfoV1FromV2Snapshot(
StateMetaInfoSnapshot stateMeta) {

Preconditions.checkArgument(
stateMeta
.getBackendStateType()
.equals(StateMetaInfoSnapshot.BackendStateType.KEY_VALUE_V2));
org.apache.flink.api.common.state.StateDescriptor.Type newStateType;
TypeSerializerSnapshot oldValueSerializerSnapshot =
stateMeta.getTypeSerializerSnapshot(
StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER);
TypeSerializerSnapshot oldUserKeySerializerSnapshot =
stateMeta.getTypeSerializerSnapshot(
StateMetaInfoSnapshot.CommonSerializerKeys.USER_KEY_SERIALIZER);
TypeSerializerSnapshot newStateSerializerSnapshot = oldValueSerializerSnapshot;

switch (org.apache.flink.api.common.state.StateDescriptor.Type.valueOf(
stateMeta.getOption(StateMetaInfoSnapshot.CommonOptionsKeys.KEYED_STATE_TYPE))) {
case VALUE:
newStateType = org.apache.flink.api.common.state.StateDescriptor.Type.VALUE;
break;
case REDUCING:
newStateType = org.apache.flink.api.common.state.StateDescriptor.Type.REDUCING;
break;
case AGGREGATING:
newStateType = org.apache.flink.api.common.state.StateDescriptor.Type.AGGREGATING;
break;
case LIST:
newStateType = org.apache.flink.api.common.state.StateDescriptor.Type.LIST;
newStateSerializerSnapshot =
new ListSerializer<>(oldValueSerializerSnapshot.restoreSerializer())
.snapshotConfiguration();
break;
case MAP:
newStateType = org.apache.flink.api.common.state.StateDescriptor.Type.MAP;
// UserKeySerializerSnapshot may be null since it may be restored from an old
// version.
if (oldUserKeySerializerSnapshot == null) {
throw new UnsupportedOperationException(
"RegisteredKeyAndUserKeyValueStateBackendMetaInfo can not transform to org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo since oldUserKeySerializerSnapshot is null");
} else {
newStateSerializerSnapshot =
new MapSerializer<>(
oldUserKeySerializerSnapshot.restoreSerializer(),
oldValueSerializerSnapshot.restoreSerializer())
.snapshotConfiguration();
}
break;
default:
throw new UnsupportedOperationException(
"Unsupported state type to transform from org.apache.flink.runtime.state.v2.RegisteredKeyValueStateBackendMetaInfo to org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo");
}
return new RegisteredKeyValueStateBackendMetaInfo<>(
newStateType,
stateMeta.getName(),
StateSerializerProvider.fromPreviousSerializerSnapshot(
stateMeta.getTypeSerializerSnapshot(
StateMetaInfoSnapshot.CommonSerializerKeys.NAMESPACE_SERIALIZER)),
StateSerializerProvider.fromPreviousSerializerSnapshot(newStateSerializerSnapshot),
StateSnapshotTransformer.StateSnapshotTransformFactory.noTransform());
}
}
Original file line number Diff line number Diff line change
@@ -103,7 +103,7 @@ public RegisteredKeyAndUserKeyValueStateBackendMetaInfo(
== snapshot.getBackendStateType());
}

private RegisteredKeyAndUserKeyValueStateBackendMetaInfo(
public RegisteredKeyAndUserKeyValueStateBackendMetaInfo(
@Nonnull String name,
@Nonnull StateDescriptor.Type stateType,
@Nonnull StateSerializerProvider<N> namespaceSerializerProvider,
Original file line number Diff line number Diff line change
@@ -23,6 +23,7 @@
import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
import org.apache.flink.runtime.state.RegisteredStateMetaInfoBase;
import org.apache.flink.runtime.state.RegisteredStateMetaInfoUtils;
import org.apache.flink.runtime.state.StateSerializerProvider;
import org.apache.flink.runtime.state.StateSnapshotTransformer.StateSnapshotTransformFactory;
import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot;
@@ -103,7 +104,7 @@ public RegisteredKeyValueStateBackendMetaInfo(@Nonnull StateMetaInfoSnapshot sna
== snapshot.getBackendStateType());
}

protected RegisteredKeyValueStateBackendMetaInfo(
public RegisteredKeyValueStateBackendMetaInfo(
@Nonnull String name,
@Nonnull StateDescriptor.Type stateType,
@Nonnull StateSerializerProvider<N> namespaceSerializerProvider,
@@ -261,4 +262,25 @@ private StateMetaInfoSnapshot computeSnapshot() {
serializerConfigSnapshotsMap,
serializerMap);
}

public static RegisteredStateMetaInfoBase fromMetaInfoSnapshot(
@Nonnull StateMetaInfoSnapshot snapshot) {

final StateMetaInfoSnapshot.BackendStateType backendStateType =
snapshot.getBackendStateType();
switch (backendStateType) {
case KEY_VALUE:
return RegisteredStateMetaInfoUtils.createMetaInfoV2FromV1Snapshot(snapshot);
case KEY_VALUE_V2:
if (snapshot.getOption(
StateMetaInfoSnapshot.CommonOptionsKeys.KEYED_STATE_TYPE.toString())
.equals(StateDescriptor.Type.MAP.toString())) {
return new RegisteredKeyAndUserKeyValueStateBackendMetaInfo<>(snapshot);
} else {
return new RegisteredKeyValueStateBackendMetaInfo<>(snapshot);
}
default:
return RegisteredStateMetaInfoBase.fromMetaInfoSnapshot(snapshot);
}
}
}
Original file line number Diff line number Diff line change
@@ -402,7 +402,9 @@ private ForStRestoreOperation getForStRestoreOperation(
overlapFractionThreshold,
useIngestDbRestoreMode,
rescalingUseDeleteFilesInRange,
recoveryClaimMode);
recoveryClaimMode,
org.apache.flink.runtime.state.v2.RegisteredKeyValueStateBackendMetaInfo
::fromMetaInfoSnapshot);
} else if (priorityQueueConfig.getPriorityQueueStateType()
== ForStStateBackend.PriorityQueueStateType.HEAP) {
// Note: This branch can be touched after ForSt Support canonical savepoint,
Original file line number Diff line number Diff line change
@@ -68,6 +68,8 @@ class ForStHandle implements AutoCloseable {
private ColumnFamilyHandle defaultColumnFamilyHandle;
@Nullable private ForStNativeMetricMonitor nativeMetricMonitor;

private final Function<StateMetaInfoSnapshot, RegisteredStateMetaInfoBase> stateMetaInfoFactory;

protected ForStHandle(
Map<String, ForStOperationUtils.ForStKvStateInfo> kvStateInformation,
Path instanceRocksDBPath,
@@ -77,6 +79,28 @@ protected ForStHandle(
MetricGroup metricGroup,
ForStDBTtlCompactFiltersManager ttlCompactFiltersManager,
Long writeBufferManagerCapacity) {
this(
kvStateInformation,
instanceRocksDBPath,
dbOptions,
columnFamilyOptionsFactory,
nativeMetricOptions,
metricGroup,
ttlCompactFiltersManager,
writeBufferManagerCapacity,
RegisteredStateMetaInfoBase::fromMetaInfoSnapshot);
}

protected ForStHandle(
Map<String, ForStOperationUtils.ForStKvStateInfo> kvStateInformation,
Path instanceRocksDBPath,
DBOptions dbOptions,
Function<String, ColumnFamilyOptions> columnFamilyOptionsFactory,
ForStNativeMetricOptions nativeMetricOptions,
MetricGroup metricGroup,
ForStDBTtlCompactFiltersManager ttlCompactFiltersManager,
Long writeBufferManagerCapacity,
Function<StateMetaInfoSnapshot, RegisteredStateMetaInfoBase> stateMetaInfoFactory) {
this.kvStateInformation = kvStateInformation;
this.dbPath = instanceRocksDBPath.getPath();
this.dbOptions = dbOptions;
@@ -87,6 +111,7 @@ protected ForStHandle(
this.columnFamilyDescriptors = Collections.emptyList();
this.ttlCompactFiltersManager = ttlCompactFiltersManager;
this.writeBufferManagerCapacity = writeBufferManagerCapacity;
this.stateMetaInfoFactory = stateMetaInfoFactory;
}

void openDB() throws IOException {
@@ -141,7 +166,7 @@ ForStOperationUtils.ForStKvStateInfo getOrRegisterStateColumnFamilyHandle(
// create a meta info for the state on restore;
// this allows us to retain the state in future snapshots even if it wasn't accessed
RegisteredStateMetaInfoBase stateMetaInfo =
RegisteredStateMetaInfoBase.fromMetaInfoSnapshot(stateMetaInfoSnapshot);
stateMetaInfoFactory.apply(stateMetaInfoSnapshot);
if (columnFamilyHandle == null) {
registeredStateMetaInfoEntry =
createStateInfo(
Original file line number Diff line number Diff line change
@@ -129,6 +129,8 @@ public class ForStIncrementalRestoreOperation<K> implements ForStRestoreOperatio

private final RecoveryClaimMode recoveryClaimMode;

private final Function<StateMetaInfoSnapshot, RegisteredStateMetaInfoBase> stateMetaInfoFactory;

public ForStIncrementalRestoreOperation(
String operatorIdentifier,
KeyGroupRange keyGroupRange,
@@ -152,7 +154,8 @@ public ForStIncrementalRestoreOperation(
double overlapFractionThreshold,
boolean useIngestDbRestoreMode,
boolean useDeleteFilesInRange,
RecoveryClaimMode recoveryClaimMode) {
RecoveryClaimMode recoveryClaimMode,
Function<StateMetaInfoSnapshot, RegisteredStateMetaInfoBase> stateMetaInfoFactory) {

this.forstHandle =
new ForStHandle(
@@ -163,7 +166,8 @@ public ForStIncrementalRestoreOperation(
nativeMetricOptions,
metricGroup,
ttlCompactFiltersManager,
writeBufferManagerCapacity);
writeBufferManagerCapacity,
stateMetaInfoFactory);
this.operatorIdentifier = operatorIdentifier;
this.restoredSstFiles = new TreeMap<>();
this.lastCompletedCheckpointId = -1L;
@@ -182,6 +186,7 @@ public ForStIncrementalRestoreOperation(
this.useIngestDbRestoreMode = useIngestDbRestoreMode;
this.useDeleteFilesInRange = useDeleteFilesInRange;
this.recoveryClaimMode = recoveryClaimMode;
this.stateMetaInfoFactory = stateMetaInfoFactory;
}

/**
@@ -410,7 +415,7 @@ private List<ColumnFamilyDescriptor> createColumnFamilyDescriptors(

for (StateMetaInfoSnapshot stateMetaInfoSnapshot : stateMetaInfoSnapshots) {
RegisteredStateMetaInfoBase metaInfoBase =
RegisteredStateMetaInfoBase.fromMetaInfoSnapshot(stateMetaInfoSnapshot);
stateMetaInfoFactory.apply(stateMetaInfoSnapshot);

ColumnFamilyDescriptor columnFamilyDescriptor =
ForStOperationUtils.createColumnFamilyDescriptor(
@@ -884,7 +889,7 @@ private KeyGroupRange exportColumnFamiliesWithSstDataInKeyGroupsRange(

List<RegisteredStateMetaInfoBase> registeredStateMetaInfoBases =
tmpRestoreDBInfo.stateMetaInfoSnapshots.stream()
.map(RegisteredStateMetaInfoBase::fromMetaInfoSnapshot)
.map(stateMetaInfoFactory)
.collect(Collectors.toList());

// Export all the Column Families and store the result in
Original file line number Diff line number Diff line change
@@ -542,7 +542,9 @@ private ForStRestoreOperation getForStDBRestoreOperation(
overlapFractionThreshold,
useIngestDbRestoreMode,
rescalingUseDeleteFilesInRange,
recoveryClaimMode);
recoveryClaimMode,
org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo
::fromMetaInfoSnapshot);
} else if (priorityQueueConfig.getPriorityQueueStateType()
== ForStStateBackend.PriorityQueueStateType.HEAP) {
// Note: This branch can be touched after ForSt Support canonical savepoint,
Original file line number Diff line number Diff line change
@@ -19,8 +19,12 @@
package org.apache.flink.state.forst;

import org.apache.flink.api.common.operators.MailboxExecutor;
import org.apache.flink.api.common.state.v2.ListState;
import org.apache.flink.api.common.state.v2.ListStateDescriptor;
import org.apache.flink.api.common.state.v2.MapState;
import org.apache.flink.api.common.state.v2.MapStateDescriptor;
import org.apache.flink.api.common.state.v2.ValueState;
import org.apache.flink.api.common.state.v2.ValueStateDescriptor;
import org.apache.flink.api.common.typeutils.base.IntSerializer;
import org.apache.flink.api.common.typeutils.base.StringSerializer;
import org.apache.flink.configuration.Configuration;
@@ -47,6 +51,7 @@

import java.io.File;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.concurrent.RunnableFuture;
@@ -55,7 +60,6 @@
import static org.apache.flink.state.forst.ForStTestUtils.createKeyedStateBackend;
import static org.apache.flink.state.forst.ForStTestUtils.createSyncKeyedStateBackend;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.fail;

/** Compatibility test for {@link ForStKeyedStateBackend} and {@link ForStSyncKeyedStateBackend}. */
class ForStAsyncAndSyncCompatibilityTest {
@@ -85,17 +89,29 @@ public void setup(@TempDir File temporaryFolder) throws IOException {
void testForStTransFromAsyncToSync() throws Exception {
ForStKeyedStateBackend<String> keyedBackend =
setUpAsyncKeyedStateBackend(Collections.emptyList());
MapStateDescriptor<Integer, String> descriptor =
MapStateDescriptor<Integer, String> mapDescriptor =
new MapStateDescriptor<>(
"testState", IntSerializer.INSTANCE, StringSerializer.INSTANCE);

MapState<Integer, String> asyncMapState =
keyedBackend.createState(1, IntSerializer.INSTANCE, descriptor);
keyedBackend.createState(1, IntSerializer.INSTANCE, mapDescriptor);

ValueStateDescriptor<Integer> valueDescriptor =
new ValueStateDescriptor<>("valueState", IntSerializer.INSTANCE);
ValueState<Integer> asyncValueState =
keyedBackend.createState(1, IntSerializer.INSTANCE, valueDescriptor);

ListStateDescriptor<Integer> listDescriptor =
new ListStateDescriptor<>("listState", IntSerializer.INSTANCE);
ListState<Integer> asyncListState =
keyedBackend.createState(1, IntSerializer.INSTANCE, listDescriptor);

context = aec.buildContext("testRecord", "testKey");
context.retain();
aec.setCurrentContext(context);
asyncMapState.asyncPut(1, "1");
asyncValueState.asyncUpdate(1);
asyncListState.asyncUpdate(Arrays.asList(40, 50));

context.release();
aec.drainInflightRecords(0);

@@ -125,20 +141,25 @@ void testForStTransFromAsyncToSync() throws Exception {
org.apache.flink.api.common.state.MapState<Integer, String> syncMapState =
syncKeyedStateBackend.getOrCreateKeyedState(
IntSerializer.INSTANCE,
StateDescriptorUtils.transformFromV2ToV1(descriptor));
fail();
StateDescriptorUtils.transformFromV2ToV1(mapDescriptor));

org.apache.flink.api.common.state.ValueState<Integer> syncValueState =
syncKeyedStateBackend.getOrCreateKeyedState(
IntSerializer.INSTANCE,
StateDescriptorUtils.transformFromV2ToV1(valueDescriptor));

org.apache.flink.api.common.state.ListState<Integer> syncListState =
syncKeyedStateBackend.getOrCreateKeyedState(
IntSerializer.INSTANCE,
StateDescriptorUtils.transformFromV2ToV1(listDescriptor));

syncKeyedStateBackend.setCurrentKey("testKey");
((InternalKvState) syncKeyedStateBackend).setCurrentNamespace(1);
((InternalKvState) syncMapState).setCurrentNamespace(1);
assertThat(syncMapState.get(1)).isEqualTo("1");
} catch (Exception e) {
// Currently, ForStStateBackend does not support switching from Async to Sync, so this
// exception will be caught here
assertThat(e).isInstanceOf(ClassCastException.class);
assertThat(e.getMessage())
.contains(
"org.apache.flink.runtime.state.v2.RegisteredKeyAndUserKeyValueStateBackendMetaInfo cannot be cast to class org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo");

((InternalKvState) syncValueState).setCurrentNamespace(1);
assertThat(syncValueState.value()).isEqualTo(1);
((InternalKvState) syncListState).setCurrentNamespace(1);
assertThat(syncListState.get()).isEqualTo(Arrays.asList(40, 50));
} finally {
IOUtils.closeQuietly(syncKeyedStateBackend);
}
@@ -151,12 +172,29 @@ void testForStTransFromSyncToAsync() throws Exception {
forStStateBackend, env, StringSerializer.INSTANCE, Collections.emptyList());
org.apache.flink.api.common.state.MapStateDescriptor<Integer, String> descriptor =
new org.apache.flink.api.common.state.MapStateDescriptor<>(
"testState", IntSerializer.INSTANCE, StringSerializer.INSTANCE);
"mapState", IntSerializer.INSTANCE, StringSerializer.INSTANCE);
org.apache.flink.api.common.state.MapState<Integer, String> mapState =
keyedBackend.getOrCreateKeyedState(IntSerializer.INSTANCE, descriptor);

org.apache.flink.api.common.state.ValueStateDescriptor<Integer> valueDescriptor =
new org.apache.flink.api.common.state.ValueStateDescriptor<>(
"valueState", IntSerializer.INSTANCE);
org.apache.flink.api.common.state.ValueState<Integer> valueState =
keyedBackend.getOrCreateKeyedState(IntSerializer.INSTANCE, valueDescriptor);

org.apache.flink.api.common.state.ListStateDescriptor<Integer> listStateDescriptor =
new org.apache.flink.api.common.state.ListStateDescriptor<>(
"listState", IntSerializer.INSTANCE);
org.apache.flink.api.common.state.ListState<Integer> listState =
keyedBackend.getOrCreateKeyedState(IntSerializer.INSTANCE, listStateDescriptor);

keyedBackend.setCurrentKey("testKey");
((InternalKvState) mapState).setCurrentNamespace(1);
mapState.put(1, "1");
((InternalKvState) valueState).setCurrentNamespace(1);
valueState.update(1);
((InternalKvState) listState).setCurrentNamespace(1);
listState.update(Arrays.asList(1, 2));

RunnableFuture<SnapshotResult<KeyedStateHandle>> snapshot =
keyedBackend.snapshot(
@@ -177,33 +215,41 @@ void testForStTransFromSyncToAsync() throws Exception {
ForStKeyedStateBackend<String> asyncKeyedStateBackend =
setUpAsyncKeyedStateBackend(Collections.singletonList(stateHandle));

MapStateDescriptor<Integer, String> newStateDescriptor =
MapStateDescriptor<Integer, String> newMapDescriptor =
new MapStateDescriptor<>(
"testState", IntSerializer.INSTANCE, StringSerializer.INSTANCE);
"mapState", IntSerializer.INSTANCE, StringSerializer.INSTANCE);
ValueStateDescriptor<Integer> newValueDescriptor =
new ValueStateDescriptor<>("valueState", IntSerializer.INSTANCE);
ListStateDescriptor<Integer> newListDescriptor =
new ListStateDescriptor<>("listState", IntSerializer.INSTANCE);

try {
MapState<Integer, String> asyncMapState =
asyncKeyedStateBackend.createState(1, IntSerializer.INSTANCE, newMapDescriptor);
ValueState<Integer> asyncValueState =
asyncKeyedStateBackend.createState(
1, IntSerializer.INSTANCE, newValueDescriptor);
ListState<Integer> asyncListState =
asyncKeyedStateBackend.createState(
1, IntSerializer.INSTANCE, newStateDescriptor);
fail();
1, IntSerializer.INSTANCE, newListDescriptor);

context = aec.buildContext("testRecord", "testKey");
context.retain();
aec.setCurrentContext(context);
asyncMapState
.asyncGet(1)
.thenCompose(
mapValue -> {
assertThat(mapValue).isEqualTo("1");
return asyncValueState.asyncValue();
})
.thenAccept(
value -> {
assertThat(value).isEqualTo("1");
assertThat(value).isEqualTo(1);
});
assertThat(listState.get()).isEqualTo(Arrays.asList(1, 2));
context.release();
aec.drainInflightRecords(0);
} catch (Exception e) {
// Currently, ForStStateBackend does not support switching from Sync to Async, so this
// exception will be caught here
assertThat(e).isInstanceOf(ClassCastException.class);
assertThat(e.getMessage())
.contains(
"org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo cannot be cast to class org.apache.flink.runtime.state.v2.RegisteredKeyValueStateBackendMetaInfo");
} finally {
IOUtils.closeQuietly(asyncKeyedStateBackend);
}