pnowojski commented on a change in pull request #9383: [FLINK-13248] [runtime]
Adding processing of downstream messages in AsyncWaitOperator's wait loops
URL: https://github.com/apache/flink/pull/9383#discussion_r317626081
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorFactory.java
##########
@@ -17,22 +17,68 @@
package org.apache.flink.streaming.api.operators.async;
-import org.apache.flink.streaming.api.operators.SimpleUdfStreamOperatorFactory;
+import org.apache.flink.streaming.api.datastream.AsyncDataStream;
+import org.apache.flink.streaming.api.functions.async.AsyncFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
import org.apache.flink.streaming.api.operators.YieldingOperatorFactory;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
import
org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxExecutor;
/**
* The factory of {@link AsyncWaitOperator}.
*
* @param <OUT> The output type of the operator
*/
-public class AsyncWaitOperatorFactory<OUT> extends
SimpleUdfStreamOperatorFactory<OUT> implements YieldingOperatorFactory {
- public AsyncWaitOperatorFactory(AsyncWaitOperator<?, OUT> operator) {
- super(operator);
+public class AsyncWaitOperatorFactory<IN, OUT> implements
OneInputStreamOperatorFactory<IN, OUT>, YieldingOperatorFactory {
+ private final AsyncFunction<IN, OUT> asyncFunction;
+ private final long timeout;
+ private final int capacity;
+ private final AsyncDataStream.OutputMode outputMode;
+ private MailboxExecutor mailboxExecutor;
+ private ChainingStrategy strategy = ChainingStrategy.HEAD;
+
+ public AsyncWaitOperatorFactory(AsyncFunction<IN, OUT> asyncFunction,
+ long timeout,
+ int capacity,
+ AsyncDataStream.OutputMode outputMode) {
+ this.asyncFunction = asyncFunction;
+ this.timeout = timeout;
+ this.capacity = capacity;
+ this.outputMode = outputMode;
}
@Override public void setMailboxExecutor(MailboxExecutor
mailboxExecutor) {
- AsyncWaitOperator<?, OUT> operator = (AsyncWaitOperator<?,
OUT>) getOperator();
- operator.setMailboxExecutor(mailboxExecutor);
+ this.mailboxExecutor = mailboxExecutor;
+ }
+
+ @Override public StreamOperator createStreamOperator(StreamTask
containingTask, StreamConfig config,
+ Output output) {
+ AsyncWaitOperator asyncWaitOperator = new
AsyncWaitOperator(asyncFunction,
+ timeout,
+ capacity,
+ outputMode,
+ mailboxExecutor);
Review comment:
`checkNotNull(mailboxExecutor)`?
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services