[ 
https://issues.apache.org/jira/browse/FLINK-33010?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17761085#comment-17761085
 ] 

Sergey Nuyanzin commented on FLINK-33010:
-----------------------------------------

Thanks for your response, yes confirm I was able to reproduce it on master.

Also I was able to reproduce with a simpler query
{code:sql}
SELECT GREATEST(IFNULL(1, 2), IFNULL(2, 1));
{code}

In fact could be reproduced with {{GREATEST}}, {{LEAST}} for the result of 
other functions.

The reason is that during code generation it generates
{code:java}
...
/* 25 */        @Override
/* 26 */        public Object map(Object _in1) throws Exception {
/* 27 */          org.apache.flink.table.data.RowData in1 = 
(org.apache.flink.table.data.RowData) _in1;
/* 28 */          
/* 29 */          org.apache.flink.table.data.TimestampData externalResult$24;
/* 30 */          org.apache.flink.table.data.TimestampData result$25;
/* 31 */          boolean isNull$25;
/* 32 */          org.apache.flink.table.data.TimestampData externalResult$26;
/* 33 */          org.apache.flink.table.data.TimestampData result$27;
/* 34 */          boolean isNull$27;
/* 35 */          
/* 36 */          
/* 37 */          
/* 38 */          
/* 39 */          
/* 40 */          
/* 41 */           org.apache.flink.table.data.TimestampData tmpResult$28 = 
result$25;
/* 42 */           org.apache.flink.table.data.TimestampData result$28 = null;
/* 43 */           boolean nullTerm$28 = false;
/* 44 */           
/* 45 */           
...
{code}
The problem is that here we try to assign {{tmpResult$28}} to {{result$25}} 
however {{result$25}} is not yet initialized. This happens only when 
{{GREATEST}}/{{LEAST}} is applied to result of other functions.

> NPE when using GREATEST() in Flink SQL
> --------------------------------------
>
>                 Key: FLINK-33010
>                 URL: https://issues.apache.org/jira/browse/FLINK-33010
>             Project: Flink
>          Issue Type: Bug
>          Components: Table SQL / API, Table SQL / Planner
>    Affects Versions: 1.16.1, 1.16.2
>            Reporter: Hector Rios
>            Priority: Minor
>
> Hi,
> I see NPEs in flink 1.14 and flink 1.16 when running queries with GREATEST() 
> and timestamps. Below is an example to help in reproducing the issue.
> {code:java}
> CREATE TEMPORARY VIEW Positions AS
> SELECT
> SecurityId,
> ccy1,
> CAST(publishTimestamp AS TIMESTAMP(3)) as publishTimestamp
> FROM (VALUES
> (1, 'USD', '2022-01-01'),
> (2, 'GBP', '2022-02-02'),
> (3, 'GBX', '2022-03-03'),
> (4, 'GBX', '2022-04-4'))
> AS ccy(SecurityId, ccy1, publishTimestamp);
> CREATE TEMPORARY VIEW Benchmarks AS
> SELECT
> SecurityId,
> ccy1,
> CAST(publishTimestamp AS TIMESTAMP(3)) as publishTimestamp
> FROM (VALUES
> (3, 'USD', '2023-01-01'),
> (4, 'GBP', '2023-02-02'),
> (5, 'GBX', '2023-03-03'),
> (6, 'GBX', '2023-04-4'))
> AS ccy(SecurityId, ccy1, publishTimestamp);
> SELECT *,
> GREATEST(
> IFNULL(Positions.publishTimestamp,CAST('1970-1-1' AS TIMESTAMP(3))),
> IFNULL(Benchmarks.publishTimestamp,CAST('1970-1-1' AS TIMESTAMP(3)))
> )
> FROM Positions
> FULL JOIN Benchmarks ON Positions.SecurityId = Benchmarks.SecurityId {code}
>  
> Using "IF" is a workaround at the moment instead of using "GREATEST"
>   



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to