[
https://issues.apache.org/jira/browse/FLINK-2837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15027112#comment-15027112
]
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_r45890935
--- Diff:
flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltWrapperTwoInput.java
---
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.storm.wrappers;
+
+import backtype.storm.generated.StormTopology;
+import backtype.storm.topology.IRichBolt;
+import backtype.storm.tuple.Fields;
+import org.apache.flink.api.java.tuple.Tuple0;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple25;
+import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+import java.util.Collection;
+
+/**
+ * A {@link BoltWrapperTwoInput} wraps an {@link IRichBolt} in order to
execute the Storm bolt within a Flink Streaming
+ * program. In contrast to {@link BoltWrapper}, this wrapper takes two
input stream as input.
+ */
+public class BoltWrapperTwoInput<IN1, IN2, OUT> extends BoltWrapper<IN1,
OUT> implements TwoInputStreamOperator<IN1, IN2, OUT> {
+
+ /** The schema (ie, ordered field names) of the second input stream. */
+ private final Fields inputSchema2;
+
+ private final String componentId2;
+ private final String streamId2;
+
+ /**
+ * Instantiates a new {@link BoltWrapperTwoInput} that wraps the given
Storm {@link IRichBolt bolt} such that it can be
+ * used within a Flink streaming program. The given input schema enable
attribute-by-name access for input types
+ * {@link Tuple0} to {@link Tuple25}. The output type can be any type
if parameter {@code rawOutput} is {@code true}
+ * and the bolt's number of declared output tuples is 1. If {@code
rawOutput} is {@code false} the output type will
+ * be one of {@link Tuple0} to {@link Tuple25} depending on the bolt's
declared number of attributes.
+ * @param bolt
+ * The Storm {@link IRichBolt bolt} to be used.
+ * @param boltId
+ * @param componentId2
+ * @param streamId1
+ * @param inputSchema1
+* The schema (ie, ordered field names) of the input stream.
@throws IllegalArgumentException
+* If {@code rawOuput} is {@code true} and the number of
declared output attributes is not 1 or if
+* {@code rawOuput} is {@code false} and the number of declared
output attributes is not with range
+ * */
--- End diff --
formatting (space and stars) incomplete JavaDoc; missing `@throws`
> 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)