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

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

Github user StephanEwen commented on the pull request:

    https://github.com/apache/flink/pull/1831#issuecomment-200951040
  
    Thanks for describing this.
    
    This has quite some big implications, as far as I can see it. The state in 
the connected stream is now a "broadcast state" not partitioned, so allowing to 
do that on the key/value state probably breaks some ongoing efforts, like 
scaling, etc.
    
    How about a more clean separation of these things:
      1. The connected streams (fully partitioned or not)
      2. Broadcast inputs, which are similar to the "side inputs" in cloud 
dataflow or the broadcast variables in DataSet.
    
    That gives us
      - clean semantics, behavior that users can work with
      - We do not overcomplicate the key/value abstraction
      - and we can also make sure we checkpoint the broadcast state once only 
(rather than on each parallel subtask). 



> Allow ConnectedStreams to Be Keyed on Only One Side
> ---------------------------------------------------
>
>                 Key: FLINK-3659
>                 URL: https://issues.apache.org/jira/browse/FLINK-3659
>             Project: Flink
>          Issue Type: Improvement
>          Components: Streaming
>    Affects Versions: 1.0.0
>            Reporter: Aljoscha Krettek
>            Assignee: Aljoscha Krettek
>
> Currently, we only allow partitioned state when both inputs are keyed (with 
> the same key type). I think a valid use case is to have only one side keyed 
> and have the other side broadcast to publish some updates that are relevant 
> for all keys.
> When relaxing the requirement to have only one side keyed we must still 
> ensure that the same key is used if both sides are keyed.
> [~gyfora] Did you try this with what you're working on?



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

Reply via email to