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

Timo Walther commented on FLINK-24924:
--------------------------------------

I agree that exceptions are not optimal for streaming pipelines. But even for 
batch jobs, this is a reoccurring problem. Take a batch job for example that 
runs every night and suddenly produces an exception due to some corrupt data in 
the current batch. It is basically the same problem.

Flink is a framework for general purpose data processing. It is not our 
responsibility to catch all potential errors that can occur with the faulty 
data of the user. This is the responsibility of the user. We offer a TRY_CAST 
for making things easier, and we can even provide a TRY_OR_NULL(any_expression) 
to make the development easier. Catching potential errors could be 
infrastructure of a hosted SQL services that ingests some helper functions 
automatically.

We should not let NULLs flow through the pipeline. This magic is highly 
confusing during development and downstream operators at various locations fail 
regularly because of a NULL that should not be present. Take this recent 
example that is strictly violating the SQL semantics and difficult to explain 
to users:

{code}
COALESCE(NULL, 0) => 0

COALESCE(CAST('aa' AS INT), 0) => NULL
{code}

I'm against such a strict mode. Because the more options we provide, the bigger 
the testing matrix. A strict-mode would mean that functions stick to the input 
and result data types that they have declared. And this should be the default 
anyway. A non-strict mode would modify all built-in function signatures.

> TO_TIMESTAMP and TO_DATE should fail
> ------------------------------------
>
>                 Key: FLINK-24924
>                 URL: https://issues.apache.org/jira/browse/FLINK-24924
>             Project: Flink
>          Issue Type: Sub-task
>            Reporter: Francesco Guardiani
>            Priority: Major
>
> In a similar fashion to what described 
> https://issues.apache.org/jira/browse/FLINK-24385, TO_TIMESTAMP and TO_DATE 
> should fail instead of returning {{null}}.
> In particular for these two functions, a failure in parsing could lead to 
> very unexpected behavior, for example it could lead to records with null 
> rowtime.
> We should change these functions to fail by default when parsing generates an 
> error. We can let users handle errors by letting them use TRY_CAST for the 
> same functionality:
> {code:sql}
> -- This fails when input is invalid
> TO_TIMESTAMP(input)
> -- Behaves the same as above
> CAST(input AS TIMESTAMP)
> -- This returns null when input is invalid
> TRY_CAST(input AS TIMESTAMP)
> {code}



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to