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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/StateRequest.java:
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.asyncprocessing;
+
+import org.apache.flink.api.common.state.v2.State;
+import org.apache.flink.core.state.InternalStateFuture;
+
+import javax.annotation.Nullable;
+
+/**
+ * A request encapsulates the necessary data to perform a state request.
+ *
+ * @param <K> Type of partitioned key.
+ * @param <IN> Type of input of this request.
+ * @param <OUT> Type of value that request will return.
+ */
+public class StateRequest<K, IN, OUT> {
+
+    /** The type of processing request. */
+    public enum RequestType {
+        /** Process one record without state access. */
+        SYNC,
+        /** Get from one {@link State}. */
+        GET,
+        /** Put to one {@link State}. */
+        PUT,
+        /** Merge value to an exist key in {@link State}. Mainly used for 
listState. */
+        MERGE,
+        /** Delete from one {@link State}. */
+        DELETE
+    }
+
+    /** The underlying state to be accessed, can be empty for {@link 
RequestType#SYNC}. */
+    @Nullable private final State state;

Review Comment:
   The reason we keep it `org.apache.flink.api.common.state.v2.State` here is 
we hope the direct implementation of KV (aka `internal key-value` layer) to be 
an unified light-weight one,  which is only a simple bridge between 
user-interface to `AEC` (see `AsyncExecutionControllerTest#TestValueState` as 
an example). I think we could make it internal kv interface or class parameter 
in future but for now I suggest we keep it simple.



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