fredia commented on code in PR #24739:
URL: https://github.com/apache/flink/pull/24739#discussion_r1584188915


##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStStateExecutor.java:
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.runtime.asyncprocessing.StateExecutor;
+import org.apache.flink.runtime.asyncprocessing.StateRequest;
+import org.apache.flink.runtime.asyncprocessing.StateRequestContainer;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+/**
+ * The {@link StateExecutor} implementation which executing batch {@link 
StateRequest}s for
+ * ForStStateBackend.
+ */
+public class ForStStateExecutor implements StateExecutor {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(ForStStateExecutor.class);
+
+    /**
+     * The coordinator thread which schedules the execution of multiple 
batches of stateRequests.
+     * The number of coordinator threads is 1 to ensure that multiple batches 
of stateRequests can
+     * be executed sequentially.
+     */
+    private final Executor coordinatorThread;

Review Comment:
   It would be better to close `coordinatorThread` when the task dispose.



##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStValueState.java:
##########
@@ -95,4 +100,32 @@ public V deserializeValue(byte[] valueBytes) throws 
IOException {
         inputView.setBuffer(valueBytes);
         return getValueSerializer().deserialize(inputView);
     }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public ForStDBGetRequest<ContextKey<K>, V> buildDBGetRequest(
+            StateRequest<?, ?, ?> stateRequest) {
+        Preconditions.checkArgument(stateRequest.getRequestType() == 
StateRequestType.VALUE_GET);
+        ContextKey<K> contextKey =
+                new ContextKey<>((RecordContext<K>) 
stateRequest.getRecordContext());
+        return ForStDBGetRequest.of(
+                contextKey, this, (InternalStateFuture<V>) 
stateRequest.getFuture());
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public ForStDBPutRequest<ContextKey<K>, V> buildDBPutRequest(
+            StateRequest<?, ?, ?> stateRequest) {
+        Preconditions.checkArgument(
+                stateRequest.getRequestType() == StateRequestType.VALUE_UPDATE
+                        || stateRequest.getRequestType() == 
StateRequestType.CLEAR);
+        ContextKey<K> contextKey =
+                new ContextKey<>((RecordContext<K>) 
stateRequest.getRecordContext());
+        V value =
+                (stateRequest.getRequestType() == StateRequestType.CLEAR)
+                        ? null  // "Delete(key)" is equivalent to "Put(key, 
null)"
+                        : (V) stateRequest.getPayload();

Review Comment:
   What's the content in `stateRequest.getPayload()`, I remember it was the 
serialized key in [last 
PR](https://github.com/apache/flink/pull/24681#discussion_r1576167215).



##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStStateRequestClassifier.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.runtime.asyncprocessing.StateRequest;
+import org.apache.flink.runtime.asyncprocessing.StateRequestContainer;
+import org.apache.flink.runtime.asyncprocessing.StateRequestType;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * The ForSt {@link StateRequestContainer} which can classify the state 
requests by ForStDB
+ * requestType (Get、Put or Iterator).
+ */
+public class ForStStateRequestClassifier implements StateRequestContainer {
+
+    private final List<ForStDBGetRequest<?, ?>> dbGetRequests;
+
+    private final List<ForStDBPutRequest<?, ?>> dbPutRequests;
+
+    public ForStStateRequestClassifier() {
+        this.dbGetRequests = new ArrayList<>();
+        this.dbPutRequests = new ArrayList<>();
+    }
+
+    @Override
+    public void offer(StateRequest<?, ?, ?> stateRequest) {
+        if (stateRequest.getState() == null) {

Review Comment:
   when does `stateRequest.getState() == null`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to