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

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

Github user tillrohrmann commented on the issue:

    https://github.com/apache/flink/pull/2629
  
    Ah, I think now I understand. Upon checkpointing you disable the output 
emission. The emission can only be reactivated when adding a `StreamRecord`, a 
`Watermark` or a `LatencyMarker` to the `AsyncCollectorBuffer`. And this cannot 
happen as long as the checkpointing isn't completed, because it holds the 
checkpoint lock.
    
    But isn't this a problem in the following case: What if we have some 
elements pending in the `queue` when doing the checkpoint. After the checkpoint 
has completed, there won't be any other `StreamRecords`, `Watermarks` (because 
we haven't activated them) and `LatencyMarkers` coming on the input. This would 
imply that the pending elements in the `AsyncCollectorBuffer` will never be 
emitted, won't they?


> 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