fredia commented on code in PR #24671:
URL: https://github.com/apache/flink/pull/24671#discussion_r1568129118
##
flink-runtime/src/main/java/org/apache/flink/runtime/state/v2/InternalSyncState.java:
##
@@ -0,0 +1,51 @@
+/*
+ * 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 java.io.IOException;
+
+/**
+ * Internal state interface for the stateBackend layer, where state requests
are executed
+ * synchronously. Unlike the user-facing state interface, {@code
InternalSyncState}'s interfaces
+ * provide the key directly in the method signature, so there is no need to
pass the keyContext
+ * information for these interfaces.
+ *
+ * @param Type of the key in the state.
+ */
+@Internal
+public interface InternalSyncState {
Review Comment:
`SyncState` is a little bit strange, I'm not sure if
`InternalUnderlyingState` would be better, maybe we can seek other people’s
opinions.
##
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/state/AbstractForStState.java:
##
@@ -0,0 +1,115 @@
+/*
+ * 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.state;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataInputDeserializer;
+import org.apache.flink.core.memory.DataOutputSerializer;
+import org.apache.flink.runtime.state.CompositeKeySerializationUtils;
+import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
+import org.apache.flink.runtime.state.SerializedCompositeKeyBuilder;
+import org.apache.flink.runtime.state.v2.InternalSyncState;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.WriteOptions;
+
+import java.io.IOException;
+
+/**
+ * Base class for {@link InternalSyncState} implementations that store state
in a ForSt database.
+ *
+ * @param Type of the key in the state.
+ * @param Type of the value in the state.
+ */
+public class AbstractForStState implements InternalSyncState {
+
+protected final RocksDB db;
+
+protected final ColumnFamilyHandle columnFamily;
+
+private final int maxParallelism;
+
+protected final WriteOptions writeOptions;
+
+protected final TypeSerializer keySerializer;
Review Comment:
Is`namespaceSerializer` needed here?
##
flink-runtime/src/main/java/org/apache/flink/runtime/state/v2/InternalSyncState.java:
##
@@ -0,0 +1,51 @@
+/*
+ * 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