[
https://issues.apache.org/jira/browse/FLINK-2837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15029735#comment-15029735
]
ASF GitHub Bot commented on FLINK-2837:
---------------------------------------
Github user mjsax commented on a diff in the pull request:
https://github.com/apache/flink/pull/1398#discussion_r46033705
--- Diff:
flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/SpoutSplitExample.java
---
@@ -70,7 +70,7 @@ public static void main(final String[] args) throws
Exception {
oddStream.transform("oddBolt",
TypeExtractor.getForObject(new Tuple2<String,
Integer>("", 0)),
new BoltWrapper<SplitStreamType<Integer>,
Tuple2<String, Integer>>(
- new VerifyAndEnrichBolt(false)))
+ new VerifyAndEnrichBolt(false),
"stream", "component"))
--- End diff --
Default values for both? Or use `ODD_STREAM` for consistency.
> 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)