[
https://issues.apache.org/jira/browse/FLINK-31007?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17687874#comment-17687874
]
xzw0223 commented on FLINK-31007:
---------------------------------
[~lincoln.86xy] Hello, I would like to share with you the results of my
investigation.
The specific reason is that the return result of true_value is char (0) type,
and the false type is string type.
The specific reason is that the return result of true_value is char (0) type,
and the false type is string type, which leads to type inconsistency, because
there are some conflicts between the conversion type code generated in the
middle and the previous judgment code, resulting in a NullPointerException
problem when judging.
If the return type of true_value is char(0), and the result we return is
false_value, the following situations will occur:
eg1.
if(1=2, '' , 'abc' ) return ===> result =''
eg2.
if(1=2, 'a' , 'cde' ) return ===> result ='c'
He will return according to the field type of true_value.
nit : For this situation, I think it is wrong, it will affect our normal
results.
In my test, I found a problem. If the true_value and false_value types are the
same, there will be no NullPointerException problem,the cause of the problem
was explained at the very beginning.
Below is the result of the code in my test.
Compare Figure 1
!image-2023-02-13-18-24-08-380.png|width=658,height=263!
!image-2023-02-13-18-25-32-357.png|width=675,height=256!
Compare Figure 2
!image-2023-02-13-18-27-27-856.png|width=580,height=209!
My idea is that I think it is necessary to unify the return types of true_value
and false_value, because different type conversions may lead to incorrect data,
such as converting long to int may cause loss of precision
[~lincoln.86xy] I want to hear your opinions and opinions, thank you.
> The code generated by the IF function throws NullPointerException
> -----------------------------------------------------------------
>
> Key: FLINK-31007
> URL: https://issues.apache.org/jira/browse/FLINK-31007
> Project: Flink
> Issue Type: Bug
> Components: Table SQL / Planner, Table SQL / Runtime
> Affects Versions: 1.15.2, 1.15.3
> Environment: {code:java}
> // 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}
> Reporter: tivanli
> Assignee: xzw0223
> Priority: Major
> Attachments: StreamExecCalc$19.java,
> image-2023-02-10-17-20-51-619.png, image-2023-02-13-18-24-08-380.png,
> image-2023-02-13-18-25-32-357.png, image-2023-02-13-18-27-27-856.png
>
>
> 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)
--
This message was sent by Atlassian Jira
(v8.20.10#820010)