Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-9141

Calling getSideOutput() and split() on one DataStream causes NPE

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Critical
    • Resolution: Fixed
    • 1.4.0
    • 1.5.0
    • API / DataStream
    • None

    Description

      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:

      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)
      

      Reproducer:

      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();
      }
      

      Attachments

        Issue Links

          Activity

            People

              chesnay Chesnay Schepler
              chesnay Chesnay Schepler
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: