jectpro7 commented on code in PR #25125:
URL: https://github.com/apache/flink/pull/25125#discussion_r1777919385


##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStAggregatingState.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.AggregatingState;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataInputDeserializer;
+import org.apache.flink.core.memory.DataOutputSerializer;
+import org.apache.flink.core.state.InternalStateFuture;
+import org.apache.flink.runtime.asyncprocessing.RecordContext;
+import org.apache.flink.runtime.asyncprocessing.StateRequest;
+import org.apache.flink.runtime.asyncprocessing.StateRequestHandler;
+import org.apache.flink.runtime.asyncprocessing.StateRequestType;
+import org.apache.flink.runtime.state.SerializedCompositeKeyBuilder;
+import org.apache.flink.runtime.state.v2.AbstractAggregatingState;
+import org.apache.flink.runtime.state.v2.AggregatingStateDescriptor;
+import org.apache.flink.util.Preconditions;
+
+import org.rocksdb.ColumnFamilyHandle;
+
+import java.io.IOException;
+import java.util.function.Supplier;
+
+/**
+ * The implementation of {@link AggregatingState} for ForStDB.
+ *
+ * @param <K> type of key
+ * @param <IN> type of input
+ * @param <ACC> type of aggregate state
+ * @param <OUT> type of output
+ */
+public class ForStAggregatingState<K, N, IN, ACC, OUT>
+        extends AbstractAggregatingState<K, N, IN, ACC, OUT> implements 
ForStInnerTable<K, N, ACC> {
+
+    /** 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;
+
+    /** The serializer for namespace. * */
+    private final ThreadLocal<TypeSerializer<N>> namespaceSerializer;
+
+    /** The default namespace if not set. * */
+    private final N defaultNamespace;
+    /* Creates a new InternalKeyedState with the given 
asyncExecutionController and stateDescriptor.
+     *
+     * @param stateRequestHandler The async request handler for handling all 
requests.
+     * @param stateDescriptor     The properties of the state.
+     */
+    public ForStAggregatingState(
+            AggregatingStateDescriptor<IN, ACC, OUT> stateDescriptor,
+            StateRequestHandler stateRequestHandler,
+            ColumnFamilyHandle columnFamily,
+            Supplier<SerializedCompositeKeyBuilder<K>> 
serializedKeyBuilderInitializer,
+            N defaultNamespace,
+            Supplier<TypeSerializer<N>> namespaceSerializerInitializer,
+            Supplier<DataOutputSerializer> valueSerializerViewInitializer,
+            Supplier<DataInputDeserializer> valueDeserializerViewInitializer) {
+        super(stateRequestHandler, stateDescriptor);
+        this.columnFamilyHandle = columnFamily;
+        this.serializedKeyBuilder = 
ThreadLocal.withInitial(serializedKeyBuilderInitializer);
+        this.namespaceSerializer = 
ThreadLocal.withInitial(namespaceSerializerInitializer);
+        this.defaultNamespace = defaultNamespace;
+        this.valueDeserializerView = 
ThreadLocal.withInitial(valueDeserializerViewInitializer);
+        this.valueSerializerView = 
ThreadLocal.withInitial(valueSerializerViewInitializer);
+    }
+
+    @Override
+    public ColumnFamilyHandle getColumnFamilyHandle() {
+        return columnFamilyHandle;
+    }
+
+    @Override
+    public byte[] serializeKey(ContextKey<K, N> key) throws IOException {
+        return key.getOrCreateSerializedKey(
+                ctxKey -> {
+                    SerializedCompositeKeyBuilder<K> builder = 
serializedKeyBuilder.get();
+                    builder.setKeyAndKeyGroup(ctxKey.getRawKey(), 
ctxKey.getKeyGroup());
+                    N namespace = ctxKey.getNamespace();
+                    return builder.buildCompositeKeyNamespace(
+                            namespace == null ? defaultNamespace : namespace,
+                            namespaceSerializer.get());
+                });
+    }
+
+    @Override
+    public byte[] serializeValue(ACC value) throws IOException {
+        DataOutputSerializer outputView = valueSerializerView.get();
+        outputView.clear();
+        getValueSerializer().serialize(value, outputView);
+        return outputView.getCopyOfBuffer();
+    }
+
+    @Override
+    public ACC deserializeValue(byte[] value) throws IOException {
+        DataInputDeserializer inputView = valueDeserializerView.get();
+        inputView.setBuffer(value);
+        return getValueSerializer().deserialize(inputView);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public ForStDBGetRequest<K, N, ACC, ?> buildDBGetRequest(
+            StateRequest<?, ?, ?, ?> stateRequest) {
+        Preconditions.checkArgument(
+                stateRequest.getRequestType() == 
StateRequestType.AGGREGATING_GET);
+        ContextKey<K, N> contextKey =
+                new ContextKey<>(
+                        (RecordContext<K>) stateRequest.getRecordContext(),
+                        (N) stateRequest.getNamespace());
+        return new ForStDBSingleGetRequest<>(
+                contextKey, this, (InternalStateFuture<ACC>) 
stateRequest.getFuture());
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public ForStDBPutRequest<?, ?, ?> buildDBPutRequest(StateRequest<?, ?, ?, 
?> stateRequest) {
+        Preconditions.checkArgument(

Review Comment:
   yes, my bad.  orz
   



##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStStateRequestClassifier.java:
##########
@@ -99,6 +99,20 @@ private void 
convertStateRequestsToForStDBRequests(StateRequest<?, ?, ?, ?> stat
                     
dbPutRequests.add(forStMapState.buildDBBunchPutRequest(stateRequest));
                     return;
                 }
+            case AGGREGATING_GET:

Review Comment:
   done



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

Reply via email to