Zakelly commented on code in PR #25363:
URL: https://github.com/apache/flink/pull/25363#discussion_r1774571210
##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStGeneralMultiGetOperation.java:
##########
@@ -33,39 +37,169 @@
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(
+ () -> {
+ ReadOptions readOptions = new ReadOptions();
+ readOptions.setReadaheadSize(0);
+ List<byte[]> keys = new
ArrayList<>(getRequests.size());
+ List<ColumnFamilyHandle> columnFamilyHandles =
+ new ArrayList<>(getRequests.size());
+
+ for (int i = 0; i < getRequests.size(); i++) {
+ ForStDBGetRequest<?, ?, ?, ?> request =
getRequests.get(i);
+ try {
+ if (error.get() == null) {
+ byte[] key = request.buildSerializedKey();
+ keys.add(key);
+
columnFamilyHandles.add(request.getColumnFamilyHandle());
+ } else {
+ completeExceptionallyRequest(
+ request,
+ "Error already occurred in other
state request of the same group, failed the state request directly",
+ error.get());
+ }
+ } catch (IOException e) {
+ error.set(e);
+ completeExceptionallyRequest(
+ request,
+ "Error when execute ForStDb serialized
get key",
+ e);
+ future.completeExceptionally(e);
+ }
+ }
+ if (error.get() != null) {
+ if (subProcessFinished != null) {
+ subProcessFinished.run();
+ }
+ return;
+ }
+ List<byte[]> values = null;
+ try {
+ values = db.multiGetAsList(readOptions,
columnFamilyHandles, keys);
+ } catch (Exception e) {
+ error.set(e);
+ future.completeExceptionally(e);
+ for (int i = 0; i < getRequests.size(); i++) {
+ completeExceptionallyRequest(
+ getRequests.get(i), "Error occurred
when multiGet", e);
+ }
+ }
+ if (error.get() != null) {
+ if (subProcessFinished != null) {
+ subProcessFinished.run();
+ }
+ return;
+ }
+ for (int i = 0; i < getRequests.size(); i++) {
+ ForStDBGetRequest<?, ?, ?, ?> request =
getRequests.get(i);
+ try {
+ if (error.get() == null) {
+ request.completeStateFuture(values.get(i));
+ } else {
+ completeExceptionallyRequest(
+ request,
+ "Error already occurred in other
state request of the same "
+ + "group, failed the state
request directly",
+ error.get());
+ }
+ } catch (Exception e) {
+ error.set(e);
+ completeExceptionallyRequest(
+ request, "Error when complete get
future.", e);
+ future.completeExceptionally(e);
+ }
+ }
+
+ if (counter.addAndGet(-getRequests.size()) == 0
+ && !future.isCompletedExceptionally()) {
+ future.complete(null);
+ }
+ if (subProcessFinished != null) {
+ subProcessFinished.run();
+ }
+ });
+ }
+ return future;
+ }
+
+ private void completeExceptionallyRequest(
+ ForStDBGetRequest<?, ?, ?, ?> request, String message, Exception
e) {
+ request.completeStateFutureExceptionally(message, e);
+ }
+
+ private void classifyAndSplitRequests(
+ List<List<ForStDBGetRequest<?, ?, ?, ?>>> splitRequests,
+ List<ForStDBGetRequest<?, ?, ?, ?>> mapCheckRequests) {
+ List<ForStDBGetRequest<?, ?, ?, ?>> getRequests = new ArrayList<>();
for (int i = 0; i < batchRequest.size(); i++) {
ForStDBGetRequest<?, ?, ?, ?> request = batchRequest.get(i);
+ if (request instanceof ForStDBMapCheckRequest) {
+ mapCheckRequests.add(request);
+ } else {
+ getRequests.add(request);
Review Comment:
OK. Let's keep this
--
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]