Zakelly commented on code in PR #25308:
URL: https://github.com/apache/flink/pull/25308#discussion_r1756743480


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/v2/InternalReducingState.java:
##########
@@ -58,6 +72,108 @@ public V get() {
 
     @Override
     public void add(V value) {
-        handleRequestSync(StateRequestType.REDUCING_ADD, value);
+        V oldValue = handleRequestSync(StateRequestType.REDUCING_GET, null);
+        try {
+            V newValue = oldValue == null ? value : 
reduceFunction.reduce(oldValue, value);
+            handleRequestSync(StateRequestType.REDUCING_ADD, newValue);
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public StateFuture<Void> asyncMergeNamespaces(N target, Collection<N> 
sources)
+            throws Exception {
+        if (sources == null || sources.isEmpty()) {
+            return StateFutureUtils.completedVoidFuture();
+        }
+        List<StateFuture<V>> futures = new ArrayList<>();
+        for (N source : sources) {
+            if (source != null) {
+                setCurrentNamespace(source);
+                futures.add(handleRequest(StateRequestType.REDUCING_GET, 
null));
+            }
+        }
+        return StateFutureUtils.combineAll(futures)
+                .thenCompose(
+                        values -> {
+                            List<StateFuture<V>> removeFutures = new 
ArrayList<>();
+                            V current = null;
+                            Iterator<N> sourceIter = sources.iterator();
+                            for (V value : values) {
+                                N source = sourceIter.next();
+                                if (value != null) {
+                                    setCurrentNamespace(source);
+                                    removeFutures.add(
+                                            
handleRequest(StateRequestType.REDUCING_REMOVE, null));
+                                    if (current != null) {
+                                        current = 
reduceFunction.reduce(current, value);
+                                    } else {
+                                        current = value;
+                                    }
+                                }
+                            }
+                            V finalCurrent = current;
+                            return StateFutureUtils.combineAll(removeFutures)
+                                    .thenApply(ignore -> finalCurrent);
+                        })
+                .thenAccept(
+                        currentValue -> {
+                            if (currentValue == null) {
+                                return;
+                            }
+                            setCurrentNamespace(target);
+                            handleRequest(StateRequestType.REDUCING_GET, null)
+                                    .thenAccept(
+                                            targetValue -> {
+                                                if (targetValue == null) {
+                                                    handleRequest(
+                                                            
StateRequestType.REDUCING_ADD,
+                                                            currentValue);
+                                                } else {
+                                                    handleRequest(
+                                                            
StateRequestType.REDUCING_ADD,
+                                                            
reduceFunction.reduce(
+                                                                    
currentValue, (V) targetValue));
+                                                }
+                                            });
+                        });

Review Comment:
   Optimization Suggestion: Can we split all requests into two phase, 
GET(including the target's and sources' state split) and UPDATE?



##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStReducingState.java:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.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.InternalReducingState;
+import org.apache.flink.runtime.state.v2.ReducingStateDescriptor;
+import org.apache.flink.util.Preconditions;
+
+import org.rocksdb.ColumnFamilyHandle;
+
+import java.io.IOException;
+import java.util.function.Supplier;
+
+/**
+ * The {@link InternalReducingState} implement for ForStDB.
+ *
+ * @param <K> The type of the key.
+ * @param <N> The type of the namespace.
+ * @param <V> The type of the value.
+ */
+public class ForStReducingState<K, N, V> extends InternalReducingState<K, N, V>
+        implements ForStInnerTable<K, N, 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 default namespace if not set. * */
+    private final N defaultNamespace;
+
+    /** The serializer for namespace. * */
+    private final ThreadLocal<TypeSerializer<N>> namespaceSerializer;
+
+    /** 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 ForStReducingState(
+            StateRequestHandler stateRequestHandler,
+            ColumnFamilyHandle columnFamily,
+            ReducingStateDescriptor<V> reducingStateDescriptor,
+            Supplier<SerializedCompositeKeyBuilder<K>> 
serializedKeyBuilderInitializer,
+            N defaultNamespace,
+            Supplier<TypeSerializer<N>> namespaceSerializerInitializer,
+            Supplier<DataOutputSerializer> valueSerializerViewInitializer,
+            Supplier<DataInputDeserializer> valueDeserializerViewInitializer) {
+        super(stateRequestHandler, reducingStateDescriptor);
+        this.columnFamilyHandle = columnFamily;
+        this.serializedKeyBuilder = 
ThreadLocal.withInitial(serializedKeyBuilderInitializer);
+        this.defaultNamespace = defaultNamespace;
+        this.namespaceSerializer = 
ThreadLocal.withInitial(namespaceSerializerInitializer);
+        this.valueSerializerView = 
ThreadLocal.withInitial(valueSerializerViewInitializer);
+        this.valueDeserializerView = 
ThreadLocal.withInitial(valueDeserializerViewInitializer);
+    }
+
+    @Override
+    public ColumnFamilyHandle getColumnFamilyHandle() {
+        return columnFamilyHandle;
+    }
+
+    @Override
+    public byte[] serializeKey(ContextKey<K, N> contextKey) throws IOException 
{
+        return contextKey.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(V value) throws IOException {
+        DataOutputSerializer outputView = valueSerializerView.get();
+        outputView.clear();
+        getValueSerializer().serialize(value, outputView);
+        return outputView.getCopyOfBuffer();
+    }
+
+    @Override
+    public V deserializeValue(byte[] valueBytes) throws IOException {
+        DataInputDeserializer inputView = valueDeserializerView.get();
+        inputView.setBuffer(valueBytes);
+        return getValueSerializer().deserialize(inputView);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public ForStDBGetRequest<K, N, V, V> buildDBGetRequest(StateRequest<?, ?, 
?, ?> stateRequest) {
+        Preconditions.checkArgument(
+                stateRequest.getRequestType() == 
StateRequestType.AGGREGATING_GET);

Review Comment:
   Wrong type?



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