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

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

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

    https://github.com/apache/flink/pull/1398#discussion_r45728418
  
    --- Diff: 
flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/WrapperSetupHelper.java
 ---
    @@ -187,14 +190,15 @@ static synchronized TopologyContext 
createTopologyContext(
                                }
                        }
                        for (Entry<String, StateSpoutSpec> stateSpout : 
stateSpouts.entrySet()) {
    -                           Integer rc = taskId = 
processSingleOperator(stateSpout.getKey(), stateSpout
    +                           Integer rc = 
processSingleOperator(stateSpout.getKey(), stateSpout
                                                .getValue().get_common(), 
operatorName, context.getIndexOfThisSubtask(),
                                                dop, taskToComponents, 
componentToSortedTasks, componentToStreamToFields);
                                if (rc != null) {
                                        taskId = rc;
                                }
                        }
    -                   assert (taskId != null);
    +
    +                   Preconditions.checkNotNull("Task ID may not be null!", 
taskId);
                }
    --- End diff --
    
    They are disabled by default and are not reliable enough.


> FlinkTopologyBuilder cannot handle multiple input streams
> ---------------------------------------------------------
>
>                 Key: FLINK-2837
>                 URL: https://issues.apache.org/jira/browse/FLINK-2837
>             Project: Flink
>          Issue Type: Bug
>          Components: Storm Compatibility
>            Reporter: Matthias J. Sax
>            Assignee: Maximilian Michels
>
> FlinkTopologyBuilder cannot handle multiple input streams correctly. Instead 
> of union the incoming streams, it replicates the consuming bolt and each 
> (logical) instance processes one of the input streams.
> For example:
> {noformat}
> final FlinkTopologyBuilder builder = new FlinkTopologyBuilder();
> builder.setSpout(spoutId1, new FiniteRandomSpout(0, 10));
> builder.setSpout(spoutId2, new FiniteRandomSpout(1, 8));
> builder.setSpout(spoutId3, new FiniteRandomSpout(2, 13));
> builder.setBolt(boltId, new MergerBolt())
>       .shuffleGrouping(spoutId1)
>       .shuffleGrouping(spoutId2)
>       .shuffleGrouping(spoutId3);
> builder.setBolt("sink", new BoltPrintSink(new SimpleOutputFormatter()))
>       .shuffleGrouping(boltId);
> {noformat}
> will only print the data from a single source instead of all sources.



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

Reply via email to