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


##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStOptions.java:
##########
@@ -157,17 +157,27 @@ public class ForStOptions {
                                             + "of rocksdb timer service, but 
consumes more heap memory at the same time.",
                                     TIMER_SERVICE_FACTORY.key(), 
ForStDB.name()));
 
+    public static final ConfigOption<Boolean> EXECUTOR_WRITE_IO_INLINE =

Review Comment:
   Add this option to `ForStConfigurableOptions#CANDIDATE_CONFIGS` to make it 
configurable?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/BatchCallbackRunner.java:
##########
@@ -21,20 +21,19 @@
 import org.apache.flink.api.common.operators.MailboxExecutor;
 import org.apache.flink.util.function.ThrowingRunnable;
 
-import javax.annotation.concurrent.GuardedBy;
+import net.jcip.annotations.GuardedBy;

Review Comment:
   Use `javax.annotation.concurrent.GuardedBy`?



##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStStateExecutor.java:
##########
@@ -98,39 +130,51 @@ public CompletableFuture<Void> executeBatchRequests(
         Preconditions.checkArgument(stateRequestContainer instanceof 
ForStStateRequestClassifier);
         ForStStateRequestClassifier stateRequestClassifier =
                 (ForStStateRequestClassifier) stateRequestContainer;
+        // Calculate ongoing sub-processes. Only count read ones.
+        // The fully loaded only consider read requests for now, since the 
write ones are quick.
+        final List<ForStDBGetRequest<?, ?, ?, ?>> getRequests =
+                stateRequestClassifier.pollDbGetRequests();
+        final List<ForStDBIterRequest<?, ?, ?, ?, ?>> iterRequests =
+                stateRequestClassifier.pollDbIterRequests();
+        if (!getRequests.isEmpty()) {
+            ongoing.addAndGet(1);
+        }
+        if (!iterRequests.isEmpty()) {
+            ongoing.addAndGet(1);
+        }
         CompletableFuture<Void> resultFuture = new CompletableFuture<>();
         coordinatorThread.execute(
                 () -> {
                     long startTime = System.currentTimeMillis();
                     List<CompletableFuture<Void>> futures = new ArrayList<>(3);
-                    List<ForStDBPutRequest<?, ?, ?>> putRequests =
-                            stateRequestClassifier.pollDbPutRequests();
-                    if (!putRequests.isEmpty()) {
-                        ForStWriteBatchOperation writeOperations =
-                                new ForStWriteBatchOperation(
-                                        db,
-                                        putRequests,
-                                        writeOptions,
-                                        writeThreads == null ? readThreads : 
writeThreads);
-                        futures.add(writeOperations.process());
-                    }
 
-                    List<ForStDBGetRequest<?, ?, ?, ?>> getRequests =
-                            stateRequestClassifier.pollDbGetRequests();
                     if (!getRequests.isEmpty()) {
                         ForStGeneralMultiGetOperation getOperations =
-                                new ForStGeneralMultiGetOperation(db, 
getRequests, readThreads);
+                                new ForStGeneralMultiGetOperation(
+                                        db, getRequests, readThreads, 
ongoing::decrementAndGet);
+                        // sub process count should -1, since we have added 1 
on top.
+                        ongoing.addAndGet(getOperations.subProcessCount() - 1);
                         futures.add(getOperations.process());
                     }
 
-                    List<ForStDBIterRequest<?, ?, ?, ?, ?>> iterRequests =
-                            stateRequestClassifier.pollDbIterRequests();
                     if (!iterRequests.isEmpty()) {
                         ForStIterateOperation iterOperations =
-                                new ForStIterateOperation(db, iterRequests, 
readThreads);
+                                new ForStIterateOperation(
+                                        db, iterRequests, readThreads, 
ongoing::decrementAndGet);
+                        // sub process count should -1, since we have added 1 
on top.
+                        ongoing.addAndGet(iterOperations.subProcessCount() - 
1);
                         futures.add(iterOperations.process());
                     }
 
+                    List<ForStDBPutRequest<?, ?, ?>> putRequests =

Review Comment:
   When `isWriteInline=false`, would the `putRequests` affect `ongoing`?



-- 
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