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


##########
flink-core-api/src/main/java/org/apache/flink/api/common/state/v2/AppendingState.java:
##########
@@ -54,12 +55,40 @@ public interface AppendingState<IN, OUT> extends State {
 
     /**
      * Updates the operator state accessible by {@link #asyncGet()} by adding 
the given value to the
-     * list of values. The next time {@link #asyncGet()} is called (for the 
same state partition)
-     * the returned state will represent the updated list.
+     * list of values asynchronously. The next time {@link #asyncGet()} is 
called (for the same
+     * state partition) the returned state will represent the updated list.
      *
      * <p>null value is not allowed to be passed in.
      *
      * @param value The new value for the state.
      */
     StateFuture<Void> asyncAdd(IN value);
+
+    /**
+     * Returns the current value for the state. When the state is not 
partitioned the returned value
+     * is the same for all inputs in a given operator instance. If state 
partitioning is applied,
+     * the value returned depends on the current operator input, as the 
operator maintains an
+     * independent state for each partition.
+     *
+     * <p><b>NOTE TO IMPLEMENTERS:</b> if the state is empty, then this method 
should return {@code
+     * null}.
+     *
+     * @return The operator state value corresponding to the current input or 
{@code null} if the
+     *     state is empty.
+     * @throws Exception Thrown if the system cannot access the state.
+     */
+    SYNCOUT get() throws Exception;
+
+    /**
+     * Updates the operator state accessible by {@link #get()} by adding the 
given value to the list
+     * of values. The next time {@link #get()} is called (for the same state 
partition) the returned
+     * state will represent the updated list.
+     *
+     * <p>If null is passed in, the behaviour is undefined (implementation 
related). TODO: An
+     * unified behaviour across all sub-classes.

Review Comment:
   Shall we remove this `TODO`? I dont think this is valid.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/v2/SyncIteratorWrapper.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.runtime.state.v2;
+
+import org.apache.flink.api.common.state.v2.StateIterator;
+import org.apache.flink.runtime.asyncprocessing.AbstractStateIterator;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+
+public class SyncIteratorWrapper<T> implements Iterator<T> {
+    private final ArrayList<T> cacheEntries;
+    private int cacheIndex;
+
+    public SyncIteratorWrapper(StateIterator<T> stateIterator) {
+        this.cacheEntries = new ArrayList<>();
+        this.cacheIndex = 0;
+        if (stateIterator instanceof AbstractStateIterator) {
+            ((AbstractStateIterator<T>) 
stateIterator).onNextSync(this.cacheEntries::add);
+        }
+    }
+
+    @Override
+    public boolean hasNext() {
+        return (cacheIndex < cacheEntries.size());
+    }
+
+    @Override
+    public T next() {
+        if (cacheIndex == cacheEntries.size()) {
+            return null;

Review Comment:
   It's better to throw `NoSuchElementException` if there is no more elements.



##########
flink-state-backends/flink-statebackend-forst/src/test/java/org/apache/flink/state/forst/ForStDBOperationTestBase.java:
##########
@@ -129,6 +137,16 @@ static class TestStateFuture<T> implements 
InternalStateFuture<T> {
 
         public CompletableFuture<T> future = new CompletableFuture<>();
 
+        @Override
+        public boolean isDone() {
+            throw new UnsupportedOperationException();

Review Comment:
   How about return `future.isDone`? Same for the `get()`



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