[ 
https://issues.apache.org/jira/browse/FLINK-4391?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15683302#comment-15683302
 ] 

ASF GitHub Bot commented on FLINK-4391:
---------------------------------------

Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2629#discussion_r88715633
  
    --- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java
 ---
    @@ -0,0 +1,224 @@
    +/*
    + * 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.api.operators.async;
    +
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.runtime.state.StateInitializationContext;
    +import org.apache.flink.runtime.state.StateSnapshotContext;
    +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.AbstractUdfStreamOperator;
    +import org.apache.flink.streaming.api.operators.ChainingStrategy;
    +import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
    +import org.apache.flink.streaming.api.operators.Output;
    +import org.apache.flink.streaming.api.operators.TimestampedCollector;
    +import org.apache.flink.streaming.api.watermark.Watermark;
    +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
    +import 
org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.runtime.tasks.StreamTask;
    +import org.apache.flink.util.Preconditions;
    +
    +import java.util.Collection;
    +
    +@Internal
    +public class AsyncWaitOperator<IN, OUT>
    +   extends AbstractUdfStreamOperator<OUT, AsyncFunction<IN, OUT>>
    +   implements OneInputStreamOperator<IN, OUT>
    +{
    +   private final int DEFAULT_BUFFER_SIZE = 1000;
    +
    +   private static final long serialVersionUID = 1L;
    +
    +   private final static String STATE_NAME = "_async_wait_operator_state_";
    +
    +   /**
    +    * {@link TypeSerializer} for inputs while making snapshots.
    +    */
    +   private transient StreamElementSerializer<IN> inStreamElementSerializer;
    +
    +   /**
    +    * input stream elements from the state
    +    */
    +   private transient ListState<StreamElement> recoveredStreamElements;
    +
    +   private transient TimestampedCollector<OUT> collector;
    +
    +   private transient AsyncCollectorBuffer<IN, OUT> buffer;
    +
    +   /**
    +    * Checkpoint lock from {@link StreamTask#lock}
    +    */
    +   private transient Object checkpointLock;
    +
    +   private int bufferSize = DEFAULT_BUFFER_SIZE;
    +   private AsyncDataStream.OutputMode mode;
    +
    +   /**
    +    * For test only. Normally this flag is true, indicating that the 
Emitter Thread
    +    * in the buffer will work.
    +    */
    +   private boolean emitFlag = true;
    +
    +   /**
    +    * Test serializer used in unit test
    +    */
    +   private StreamElementSerializer<IN> inStreamElementSerializerForTest;
    --- End diff --
    
    Can't this be solved differently? It feels wrong to add testing code to non 
testing classes.


> Provide support for asynchronous operations over streams
> --------------------------------------------------------
>
>                 Key: FLINK-4391
>                 URL: https://issues.apache.org/jira/browse/FLINK-4391
>             Project: Flink
>          Issue Type: New Feature
>          Components: DataStream API
>            Reporter: Jamie Grier
>            Assignee: david.wang
>
> Many Flink users need to do asynchronous processing driven by data from a 
> DataStream.  The classic example would be joining against an external 
> database in order to enrich a stream with extra information.
> It would be nice to add general support for this type of operation in the 
> Flink API.  Ideally this could simply take the form of a new operator that 
> manages async operations, keeps so many of them in flight, and then emits 
> results to downstream operators as the async operations complete.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to