rkhachatryan commented on code in PR #25363:
URL: https://github.com/apache/flink/pull/25363#discussion_r3287549573
##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStGeneralMultiGetOperation.java:
##########
@@ -33,39 +37,167 @@
public class ForStGeneralMultiGetOperation implements ForStDBOperation {
private final RocksDB db;
-
private final List<ForStDBGetRequest<?, ?, ?, ?>> batchRequest;
+ List<List<ForStDBGetRequest<?, ?, ?, ?>>> splitRequests;
+ List<ForStDBGetRequest<?, ?, ?, ?>> mapCheckRequests;
+
private final Executor executor;
private final Runnable subProcessFinished;
+ private final int readIoParallelism;
+
ForStGeneralMultiGetOperation(
RocksDB db, List<ForStDBGetRequest<?, ?, ?, ?>> batchRequest,
Executor executor) {
- this(db, batchRequest, executor, null);
+ this(db, batchRequest, executor, 1, null);
}
ForStGeneralMultiGetOperation(
RocksDB db,
List<ForStDBGetRequest<?, ?, ?, ?>> batchRequest,
Executor executor,
+ int readIoParallelism,
Runnable subProcessFinished) {
this.db = db;
this.batchRequest = batchRequest;
this.executor = executor;
this.subProcessFinished = subProcessFinished;
+ this.readIoParallelism = readIoParallelism;
+ this.splitRequests = new ArrayList<>();
+ this.mapCheckRequests = new ArrayList<>();
+ classifyAndSplitRequests(splitRequests, mapCheckRequests);
}
@Override
public CompletableFuture<Void> process() {
- // TODO: Use MultiGet to optimize this implement
CompletableFuture<Void> future = new CompletableFuture<>();
-
AtomicReference<Exception> error = new AtomicReference<>();
AtomicInteger counter = new AtomicInteger(batchRequest.size());
+
+ processOneByOne(mapCheckRequests, error, counter, future);
+ for (List<ForStDBGetRequest<?, ?, ?, ?>> getRequests : splitRequests) {
+ executor.execute(
+ () -> {
+ try {
+ ReadOptions readOptions = new ReadOptions();
Review Comment:
Should this object be closed?
--
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]