Details
-
Bug
-
Status: Closed
-
Major
-
Resolution: Duplicate
-
1.15.2, 1.15.3
-
None
-
None
-
// code placeholder final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(1); final StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env); final DataStream<Tuple2> tab = env.fromCollection(Arrays.asList( new Tuple2<>(1L, "a_b_c"), new Tuple2<>(-1L, "a_b_c"))); final Table tableA = tableEnv.fromDataStream(tab); tableEnv.executeSql("SELECT if(f0 = -1, '', split_index(f1, '_', 0)) as id FROM " + tableA) .print();
// code placeholder final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(1); final StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env); final DataStream<Tuple2> tab = env.fromCollection(Arrays.asList( new Tuple2<>(1L, "a_b_c" ), new Tuple2<>(-1L, "a_b_c" ))); final Table tableA = tableEnv.fromDataStream(tab); tableEnv.executeSql( "SELECT if (f0 = -1, '', split_index(f1, ' _', 0)) as id FROM " + tableA) .print();
Description
Caused by: java.lang.NullPointerException
at StreamExecCalc$19.processElement_split1(Unknown Source)
at StreamExecCalc$19.processElement(Unknown Source)
at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.pushToOperator(CopyingChainingOutput.java:82)
at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:57)
at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:29)
at org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:56)
at org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:29)
at org.apache.flink.table.runtime.operators.source.InputConversionOperator.processElement(InputConversionOperator.java:128)
at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.pushToOperator(CopyingChainingOutput.java:82)
at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:57)
at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:29)
at org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:56)
at org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:29)
at org.apache.flink.streaming.api.operators.StreamSourceContexts$ManualWatermarkContext.processAndCollect(StreamSourceContexts.java:418)
at org.apache.flink.streaming.api.operators.StreamSourceContexts$WatermarkContext.collect(StreamSourceContexts.java:513)
at org.apache.flink.streaming.api.operators.StreamSourceContexts$SwitchingOnClose.collect(StreamSourceContexts.java:103)
at org.apache.flink.streaming.api.functions.source.FromElementsFunction.run(FromElementsFunction.java:231)
at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:110)
at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:67)
at org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.run(SourceStreamTask.java:332)
Attachments
Attachments
Issue Links
- duplicates
-
FLINK-30559 May get wrong result for `if` expression if it's string data type
- Resolved
- is duplicated by
-
FLINK-31653 Using`if` statement for a string subtype of the row type may meet npe in code generated by codegen
- Closed