masteryhx commented on code in PR #24681:
URL: https://github.com/apache/flink/pull/24681#discussion_r1574412285


##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStValueState.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.apache.flink.api.common.state.v2.ValueState;
+import org.apache.flink.core.memory.DataInputDeserializer;
+import org.apache.flink.core.memory.DataOutputSerializer;
+import org.apache.flink.runtime.asyncprocessing.ContextKey;
+import org.apache.flink.runtime.asyncprocessing.StateRequestHandler;
+import org.apache.flink.runtime.state.SerializedCompositeKeyBuilder;
+import org.apache.flink.runtime.state.v2.InternalValueState;
+import org.apache.flink.runtime.state.v2.ValueStateDescriptor;
+
+import org.rocksdb.ColumnFamilyHandle;
+
+import java.io.IOException;
+
+/**
+ * The {@link InternalValueState} implement for ForStDB.
+ *
+ * @param <K> The type of the key.
+ * @param <V> The type of the value.
+ */
+public class ForStValueState<K, V> extends InternalValueState<K, V>
+        implements ValueState<V>, ForStInnerTable<ContextKey<K>, V> {
+
+    /** The column family which this internal value state belongs to. */
+    private final ColumnFamilyHandle columnFamilyHandle;
+
+    /** The serialized key builder which should be thread-safe. */
+    private final ThreadLocal<SerializedCompositeKeyBuilder<K>> 
serializedKeyBuilder;
+
+    /** The data outputStream used for value serializer, which should be 
thread-safe. */
+    private final ThreadLocal<DataOutputSerializer> valueSerializerView;
+
+    /** The data inputStream used for value deserializer, which should be 
thread-safe. */
+    private final ThreadLocal<DataInputDeserializer> valueDeserializerView;
+
+    public ForStValueState(
+            StateRequestHandler stateRequestHandler,
+            ColumnFamilyHandle columnFamily,
+            ValueStateDescriptor<V> valueStateDescriptor,
+            ThreadLocal<SerializedCompositeKeyBuilder<K>> serializedKeyBuilder,

Review Comment:
   `Threadlocal` parameters are a bit confusing for callers.
   And also make life cycles of these variables expose to callers.
   Could we avoid it and just maintain them internally ?



##########
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) {
+            result.completeExceptionally(e);
+            throw new IOException("Error while adding data to ForStDB", e);
+        }
+        return result;
+    }
+
+    /** The Put access request for ForStDB. */
+    static class Request<K, V> {
+        final K key;
+        final V value;
+        final ForStInnerTable<K, V> table;
+
+        Request(K key, V value, ForStInnerTable<K, V> table) {
+            this.key = key;
+            this.value = value;
+            this.table = table;
+        }
+
+        static <K, V> Request<K, V> of(K key, V value, ForStInnerTable<K, V> 
table) {

Review Comment:
   A minor suggestion: 
   Adding some descriptions here ? Since value is nullable and we will delete 
it for null value which should be explicit for callers.



##########
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) {
+            result.completeExceptionally(e);
+            throw new IOException("Error while adding data to ForStDB", e);
+        }
+        return result;
+    }
+
+    /** The Put access request for ForStDB. */
+    static class Request<K, V> {

Review Comment:
   Could it have structural relationship with other one in 
`ForStGeneralMultiGetOperation` ?
   Or Could we use `WriteRequest` and `GetRequest` to distinguish them ?



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