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


##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStWriteBatchOperation.java:
##########
@@ -55,22 +54,44 @@ public class ForStWriteBatchOperation implements 
ForStDBOperation {
     public CompletableFuture<Void> process() {
         return CompletableFuture.runAsync(
                 () -> {
-                    try (WriteBatch writeBatch =
-                            new WriteBatch(batchRequest.size() * 
PER_RECORD_ESTIMATE_BYTES)) {
+                    try (ForStDBWriteBatchWrapper writeBatch =
+                            new ForStDBWriteBatchWrapper(db, writeOptions, 
batchRequest.size())) {
                         for (ForStDBPutRequest<?, ?> request : batchRequest) {
+                            ColumnFamilyHandle cf = 
request.getColumnFamilyHandle();
                             if (request.valueIsNull()) {
-                                // put(key, null) == delete(key)
-                                writeBatch.delete(
-                                        request.getColumnFamilyHandle(),
-                                        request.buildSerializedKey());
+                                if (request instanceof ForStDBBunchPutRequest) 
{
+                                    ForStDBBunchPutRequest<?> bunchPutRequest =
+                                            (ForStDBBunchPutRequest<?>) 
request;
+                                    byte[] primaryKey = 
bunchPutRequest.buildSerializedKey(null);

Review Comment:
   For map state, there are two types of key: primary key and user key.
   Logically it is organized like this:
   ```<primary key, <user key, user value>```
   
   We concatenate primary key and user key together as the key of rocksdb,  and 
physically organized like this:
   ```< <primary key - user key> , user value> ```
   
   For `Bunch remove`, we will remove all user keys under one primary key, so 
we use the prefix to match all keys.
   For `Single remove`, we use the bytes of <`primary key`-`user key`> to match 
one specific key.



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