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


##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStStateExecutor.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.ExecutorService;
+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 ExecutorService coordinatorThread;
+
+    /** The worker thread that actually executes the {@link StateRequest}s. */
+    private final ExecutorService workerThreads;
+
+    private final RocksDB db;
+
+    private final WriteOptions writeOptions;
+
+    public ForStStateExecutor(int ioParallelism, RocksDB db, WriteOptions 
writeOptions) {
+        this.coordinatorThread =
+                Executors.newSingleThreadScheduledExecutor(
+                        new 
ExecutorThreadFactory("ForSt-StateExecutor-Coordinator"));
+        this.workerThreads =
+                Executors.newFixedThreadPool(
+                        ioParallelism, new 
ExecutorThreadFactory("ForSt-StateExecutor-IO"));
+        this.db = db;
+        this.writeOptions = writeOptions;
+    }
+
+    @Override
+    public CompletableFuture<Void> executeBatchRequests(
+            StateRequestContainer stateRequestContainer) {
+        Preconditions.checkArgument(stateRequestContainer instanceof 
ForStStateRequestClassifier);
+        ForStStateRequestClassifier stateRequestClassifier =
+                (ForStStateRequestClassifier) stateRequestContainer;
+        return CompletableFuture.supplyAsync(
+                () -> {
+                    long startTime = System.currentTimeMillis();
+                    List<CompletableFuture<Void>> futures = new ArrayList<>(2);
+                    List<ForStDBPutRequest<?, ?>> putRequests =
+                            stateRequestClassifier.pollDbPutRequests();
+                    if (!putRequests.isEmpty()) {
+                        ForStWriteBatchOperation writeOperations =
+                                new ForStWriteBatchOperation(
+                                        db, putRequests, writeOptions, 
workerThreads);
+                        futures.add(writeOperations.process());
+                    }
+
+                    List<ForStDBGetRequest<?, ?>> getRequests =
+                            stateRequestClassifier.pollDbGetRequests();
+                    if (!getRequests.isEmpty()) {
+                        ForStGeneralMultiGetOperation getOperations =
+                                new ForStGeneralMultiGetOperation(db, 
getRequests, workerThreads);
+                        futures.add(getOperations.process());
+                    }
+
+                    try {
+                        FutureUtils.waitForAll(futures).get();
+                    } catch (InterruptedException | ExecutionException e) {
+                        throw new FlinkRuntimeException(
+                                "Exception when executing ForSt writeBatch or 
multiGet operation",
+                                e);
+                    }
+                    long duration = System.currentTimeMillis() - startTime;
+                    LOG.info(

Review Comment:
   resolved.



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to