jectpro7 commented on code in PR #24681: URL: https://github.com/apache/flink/pull/24681#discussion_r1573855255
########## flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStGeneralMultiGetOperation.java: ########## @@ -0,0 +1,105 @@ +/* + * 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.rocksdb.RocksDB; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * The general-purpose multiGet operation implementation for ForStDB, which simulates multiGet by + * calling the Get API multiple times with multiple threads. + * + * @param <K> The type of key in get access request. + * @param <V> The type of value in get access request. + */ +public class ForStGeneralMultiGetOperation<K, V> implements ForStDBOperation<List<V>> { + + private static final Logger LOG = LoggerFactory.getLogger(ForStGeneralMultiGetOperation.class); + + private final RocksDB db; + + private final List<Request<K, V>> batchRequest; + + private final Executor executor; + + ForStGeneralMultiGetOperation(RocksDB db, List<Request<K, V>> batchRequest, Executor executor) { + this.db = db; + this.batchRequest = batchRequest; + this.executor = executor; + } + + @Override + public CompletableFuture<List<V>> process() throws IOException { + + CompletableFuture<List<V>> future = new CompletableFuture<>(); + @SuppressWarnings("unchecked") + V[] result = (V[]) new Object[batchRequest.size()]; + Arrays.fill(result, null); + + AtomicInteger counter = new AtomicInteger(batchRequest.size()); + for (int i = 0; i < batchRequest.size(); i++) { + Request<K, V> request = batchRequest.get(i); + final int index = i; + executor.execute( + () -> { + try { + ForStInnerTable<K, V> table = request.table; + byte[] key = table.serializeKey(request.key); + byte[] value = db.get(table.getColumnFamilyHandle(), key); Review Comment: Hi @ljz2051, it creates many rpc request here, as FLIP-426 mentioned the rpc round-trip overhead is the bottleneck. It might be better by using `multiGetAsList` ########## flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStWriteBatchOperation.java: ########## @@ -0,0 +1,96 @@ +/* + * 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.rocksdb.RocksDB; +import org.rocksdb.RocksDBException; +import org.rocksdb.WriteBatch; +import org.rocksdb.WriteOptions; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +/** + * The writeBatch operation implementation for ForStDB. + * + * @param <K> The type of key in put access request. + * @param <V> The type of value in put access request. + */ +public class ForStWriteBatchOperation<K, V> implements ForStDBOperation<Void> { + + private static final int PER_RECORD_ESTIMATE_BYTES = 100; + + private final RocksDB db; + + private final List<Request<K, V>> batchRequest; + + private final WriteOptions writeOptions; + + ForStWriteBatchOperation( + RocksDB db, List<Request<K, V>> batchRequest, WriteOptions writeOptions) { + this.db = db; + this.batchRequest = batchRequest; + this.writeOptions = writeOptions; + } + + @Override + public CompletableFuture<Void> process() throws IOException { + CompletableFuture<Void> result = new CompletableFuture<>(); + try (WriteBatch writeBatch = + new WriteBatch(batchRequest.size() * PER_RECORD_ESTIMATE_BYTES)) { + for (Request<K, V> request : batchRequest) { + ForStInnerTable<K, V> table = request.table; + if (request.value == null) { + // put(key, null) == delete(key) + writeBatch.delete( + table.getColumnFamilyHandle(), table.serializeKey(request.key)); + } else { + writeBatch.put( + table.getColumnFamilyHandle(), + table.serializeKey(request.key), + table.serializeValue(request.value)); + } + } + db.write(writeOptions, writeBatch); + result.complete(null); + } catch (RocksDBException e) { Review Comment: is there any case that the exception is not a `RocksDBException`? If yes, the `future` might not be completed. So `complete` it in `finally` might be a better choice. -- 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]
