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


##########
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;
+
+    /** The type of this request. */
+    private final RequestType type;
+
+    /** The payload(input) of this request. */
+    @Nullable private final IN payload;
+
+    /** The future to collect the result of the request. */
+    private InternalStateFuture<OUT> stateFuture;
+
+    /** The record context of this request. */
+    private RecordContext<?, K> context;
+
+    StateRequest(@Nullable State state, RequestType type, @Nullable IN 
payload) {
+        this.state = state;
+        this.type = type;
+        this.payload = payload;
+    }
+
+    RequestType getRequestType() {
+        return type;
+    }
+
+    @Nullable
+    IN getPayload() {
+        return payload;
+    }
+
+    @Nullable
+    State getState() {
+        return state;
+    }
+
+    InternalStateFuture<OUT> getFuture() {
+        return stateFuture;
+    }
+
+    void setFuture(InternalStateFuture<OUT> future) {
+        stateFuture = future;
+    }
+
+    RecordContext<?, K> getRecordContext() {
+        return context;
+    }
+
+    void setRecordContext(RecordContext<?, K> context) {
+        this.context = context;
+    }

Review Comment:
   @masteryhx After trying, I think we are still not able to avoid lazy setting 
some contexts or create some temporary instances. How about we make 
`AEC#handleRequest` receive the information from outside and build the 
`StateRequest` internally?



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