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


##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/asyncprocessing/AbstractAsyncStateStreamOperator.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.streaming.runtime.operators.asyncprocessing;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.runtime.asyncprocessing.AsyncExecutionController;
+import org.apache.flink.runtime.asyncprocessing.RecordContext;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.Input;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+/**
+ * This operator is an abstract class that give the {@link 
AbstractStreamOperator} the ability to
+ * perform {@link AsyncStateProcessing}. The aim is to make any subclass of 
{@link
+ * AbstractStreamOperator} could manipulate async state with only a change of 
base class.
+ */
+@Internal
+@SuppressWarnings("rawtypes")
+public abstract class AbstractAsyncStateStreamOperator<OUT> extends 
AbstractStreamOperator<OUT>
+        implements AsyncStateProcessing {
+
+    private AsyncExecutionController asyncExecutionController;
+
+    private RecordContext lastProcessContext;
+
+    /** Initialize necessary state components for {@link 
AbstractStreamOperator}. */
+    @Override
+    public void setup(
+            StreamTask<?, ?> containingTask,
+            StreamConfig config,
+            Output<StreamRecord<OUT>> output) {
+        super.setup(containingTask, config, output);
+        // TODO: properly read config and setup
+        final MailboxExecutor mailboxExecutor =
+                containingTask.getEnvironment().getMainMailboxExecutor();
+        this.asyncExecutionController = new 
AsyncExecutionController(mailboxExecutor, null);
+    }
+
+    @Override
+    public final boolean isAsyncStateProcessingEnabled() {
+        // TODO: Read from config
+        return true;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public final <T> void setAsyncKeyedContextElement(
+            StreamRecord<T> record, KeySelector<T, ?> keySelector) throws 
Exception {
+        lastProcessContext =
+                asyncExecutionController.buildContext(
+                        record.getValue(), 
keySelector.getKey(record.getValue()));
+        // The processElement will be treated as a callback for dummy request. 
So reference
+        // counting should be maintained.
+        // When state request submitted, ref count +1, as described in 
FLIP-425:
+        // To cover the statements without a callback, in addition to the 
reference count marked
+        // in Fig.5, each state request itself is also protected by a paired 
reference count.
+        lastProcessContext.retain();
+        asyncExecutionController.setCurrentContext(lastProcessContext);
+    }
+
+    @Override
+    public final void postProcessElement() {
+        // The processElement will be treated as a callback for dummy request. 
So reference
+        // counting should be maintained.
+        // When a state request completes, ref count -1, as described in 
FLIP-425:
+        // To cover the statements without a callback, in addition to the 
reference count marked
+        // in Fig.5, each state request itself is also protected by a paired 
reference count.
+        lastProcessContext.release();
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public final <T> ThrowingConsumer<StreamRecord<T>, Exception> 
getRecordProcessor(int inputId) {
+        // Ideally, only TwoStreamInputOperator/OneInputStreamOperator(Input) 
will invoke here.
+        // Only those operators have the definition of processElement(1/2).
+        if (this instanceof TwoInputStreamOperator) {
+            switch (inputId) {
+                case 1:
+                    return record -> {
+                        setAsyncKeyedContextElement(record, (KeySelector<T, 
?>) stateKeySelector1);
+                        ((TwoInputStreamOperator) 
this).processElement1(record);

Review Comment:
   Afraid not. I intended to keep `setKeyContextElement1` as only for the 
synchronous execution.



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