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


##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/asyncprocessing/AbstractAsyncStateStreamOperator.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.common.typeutils.TypeSerializer;
+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.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;
+
+import java.lang.reflect.ParameterizedType;
+
+/**
+ * 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>

Review Comment:
   The async processing starts variant at `AbstractStreamOperator` and 
`AbstractStreamOperatorV2`. My brief idea is, if some operator need integrate 
with the async processing, they should change to extends 
`AbstractAsyncStateStreamOperator` instead of `AbstractStreamOperator`. 
Moreover, the `AbstractAsyncStateStreamOperator` is fully compatible with 
`AbstractStreamOperator`, if we make 
`AbstractAsyncStateStreamOperator#isAsyncStateProcessingEnabled` configurable 
to false, everything is not changed compared with `AbstractStreamOperator`. 
That's what this PR trying to introduce.
   
   Back to the `AbstractUdfStreamOperator`, it is a subclass of 
`AbstractStreamOperator`. We have two options, the first of which is a new 
introduced identical class `AbstractAsyncStateUdfStreamOperator` extending 
`AbstractAsyncStateStreamOperator`, the second is to make 
`AbstractUdfStreamOperator` extend `AbstractAsyncStateStreamOperator`. I'd 
prefer the former, we can discuss this in following PR.



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