[ 
https://issues.apache.org/jira/browse/FLINK-19411?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Piotr Nowojski closed FLINK-19411.
----------------------------------
    Resolution: Fixed

Merged to master as 7bfd8ec1e3

> MultipleInputStreamTask fails with RuntimeException when its input contains 
> union
> ---------------------------------------------------------------------------------
>
>                 Key: FLINK-19411
>                 URL: https://issues.apache.org/jira/browse/FLINK-19411
>             Project: Flink
>          Issue Type: Bug
>          Components: API / DataStream
>    Affects Versions: 1.12.0
>            Reporter: Caizhi Weng
>            Assignee: Piotr Nowojski
>            Priority: Critical
>              Labels: pull-request-available
>             Fix For: 1.12.0
>
>
> MultipleInputStreamTask fails with the following exception stack when its 
> input contains union.
> {code}
> Caused by: java.lang.RuntimeException: No such input gate.
>       at 
> org.apache.flink.streaming.runtime.io.InputGateUtil.createInputGate(InputGateUtil.java:37)
>       at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>       at 
> java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:948)
>       at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:481)
>       at 
> java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:471)
>       at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:545)
>       at 
> java.util.stream.AbstractPipeline.evaluateToArrayNode(AbstractPipeline.java:260)
>       at 
> java.util.stream.ReferencePipeline.toArray(ReferencePipeline.java:438)
>       at 
> org.apache.flink.streaming.runtime.io.InputProcessorUtil.createCheckpointedMultipleInputGate(InputProcessorUtil.java:104)
>       at 
> org.apache.flink.streaming.runtime.tasks.MultipleInputStreamTask.createInputProcessor(MultipleInputStreamTask.java:120)
>       at 
> org.apache.flink.streaming.runtime.tasks.MultipleInputStreamTask.init(MultipleInputStreamTask.java:98)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:480)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:550)
>       at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:722)
>       at org.apache.flink.runtime.taskmanager.Task.run(Task.java:547)
>       at java.lang.Thread.run(Thread.java:748)
> {code}
> Add the following test case to {{MultipleInputITCase}} will reproduce this 
> bug.
> {code:java}
> @Test
> public void testUnion() throws Exception {
>       StreamExecutionEnvironment env = 
> StreamExecutionEnvironment.getExecutionEnvironment();
>       env.setParallelism(1);
>       TestListResultSink<Long> resultSink = new TestListResultSink<>();
>       DataStream<Long> source1 = env.fromElements(1L, 10L);
>       DataStream<Long> source2 = env.fromElements(2L, 11L);
>       DataStream<String> source3 = env.fromElements("42", "44");
>       UnionTransformation<Long> union = new UnionTransformation<>(
>               Arrays.asList(source1.getTransformation(), 
> source2.getTransformation()));
>       MultipleInputTransformation<Long> multipleInput = new 
> MultipleInputTransformation<>(
>               "My Operator",
>               new SumAllInputOperatorFactory(),
>               BasicTypeInfo.LONG_TYPE_INFO,
>               1);
>       env.addOperator(multipleInput
>               .addInput(union)
>               .addInput(source3.getTransformation()));
>       new MultipleConnectedStreams(env)
>               .transform(multipleInput)
>               .addSink(resultSink);
>       env.execute();
>       List<Long> result = resultSink.getResult();
>       Collections.sort(result);
>       long actualSum = result.get(result.size() - 1);
>       assertEquals(1 + 10 + 2 + 11 + 42 + 44, actualSum);
> }
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to