[ https://issues.apache.org/jira/browse/FLINK-2837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15024325#comment-15024325 ]
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_r45725506 --- Diff: flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltFileSink.java --- @@ -18,20 +18,23 @@ package org.apache.flink.storm.util; import backtype.storm.task.TopologyContext; +import org.apache.flink.core.fs.FSDataOutputStream; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; import java.io.BufferedWriter; -import java.io.FileWriter; import java.io.IOException; +import java.io.OutputStreamWriter; import java.util.Map; /** - * Implements a sink that write the received data to the given file (as a result of {@code Object.toString()} for each + * Implements a sink that writes the received data to the given file (as a result of {@code Object.toString()} for each * attribute). */ public final class BoltFileSink extends AbstractBoltSink { private static final long serialVersionUID = 2014027288631273666L; - private final String path; + private final Path path; private BufferedWriter writer; --- End diff -- Please do not use `Path` -- **all** spouts/bolts should be written the "Storm" way and not include dependencies to Flink. > 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)