You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/05/03 21:33:00 UTC
[jira] [Commented] (FLINK-9141) Calling getSideOutput() and split()
on one DataStream causes NPE
[ https://issues.apache.org/jira/browse/FLINK-9141?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16463092#comment-16463092 ]
ASF GitHub Bot commented on FLINK-9141:
---------------------------------------
Github user fhueske commented on a diff in the pull request:
https://github.com/apache/flink/pull/5836#discussion_r185942147
--- Diff: flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/SplitSideOutputTest.java ---
@@ -0,0 +1,75 @@
+/*
+ * 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.streaming.api.datastream;
+
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.OutputTag;
+
+import org.junit.Test;
+
+import java.util.Collections;
+
+/**
+ * Tests that verify correct behavior when applying split/getSideOutput operations on one {@link DataStream}.
+ */
+public class SplitSideOutputTest {
+
+ private static final OutputTag<String> outputTag = new OutputTag<String>("outputTag") {};
+
+ @Test
+ public void testSideOutputAfterSelectIsForbidden() {
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+ SingleOutputStreamOperator<String> processInput = env.fromElements("foo")
+ .process(new DummyProcessFunction());
+
+ processInput.split(Collections::singleton);
+
+ try {
+ processInput.getSideOutput(outputTag);
--- End diff --
add `Assert.fail();` after `processInput.getSideOutput(outputTag);` to ensure that the test fails if no exception is thrown.
> Calling getSideOutput() and split() on one DataStream causes NPE
> ----------------------------------------------------------------
>
> Key: FLINK-9141
> URL: https://issues.apache.org/jira/browse/FLINK-9141
> Project: Flink
> Issue Type: Bug
> Components: DataStream API
> Affects Versions: 1.4.0
> Reporter: Chesnay Schepler
> Assignee: Chesnay Schepler
> Priority: Critical
>
> Calling both {{getSideOutput()}} and {{split()}} on one DataStream causes a {{NullPointerException}} to be thrown at runtime.
> As a work-around one can add a no-op map function before the split() call.
> Exception:
> {code}
> Caused by: java.lang.NullPointerException
> at org.apache.flink.streaming.api.collector.selector.DirectedOutput.<init>(DirectedOutput.java:79)
> at org.apache.flink.streaming.runtime.tasks.OperatorChain.createOutputCollector(OperatorChain.java:326)
> at org.apache.flink.streaming.runtime.tasks.OperatorChain.<init>(OperatorChain.java:128)
> at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:274)
> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:702)
> at java.lang.Thread.run(Thread.java:745)
> {code}
> Reproducer:
> {code}
> private static final OutputTag<String> tag = new OutputTag<String>("tag") {};
> public static void main(String[] args) throws Exception {
> StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
> DataStream<String> dataStream1 = env.fromElements("foo");
> SingleOutputStreamOperator<String> processedStream = dataStream1
> .process(new ProcessFunction<String, String>() {
> @Override
> public void processElement(String value, Context ctx, Collector<String> out) {
> }
> });
> processedStream.getSideOutput(tag)
> .print();
> processedStream
> .split(Collections::singletonList)
> .select("bar")
> .print();
> env.execute();
> }
> {code}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)