(added user ML to this thread) HI all,
I would like to raise a different opinion about this change. I agree with Ingo that we should not just break some existing behavior, and even if we introduce an option to control the behavior, i would propose to set the default value to current behavior. I want to mention one angle to assess whether we should change it or not, which is "what could users benefit from the changes". To me, it looks like: * new users: happy about the behavior * existing users: suffer from the change, it either cause them to modify the SQL or got a call in late night reporting his online job got crashed and couldn't be able to restart. I would like to quote another breaking change we did when we adjust the time-related function in FLIP-162 [1]. In that case, both new users and existing users are suffered from *incorrectly* implemented time function behavior, and we saw a lots of feedbacks and complains from various channels. After we fixed that, we never saw related problems again. Back to this topic, do we ever seen a user complain about current CAST behavior? Form my side, no. To summarize: +1 to introduce TRY_CAST to better prepare for the future. -1 to modify the default behavior. +0 to introduce a config option, but with the default value to existing behavior. it's +0 because it seems not necessary if i'm -1 to change the default behavior and also don't see an urgent to modify. [1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-162%3A+Consistent+Flink+SQL+time+function+behavior Best, Kurt On Thu, Nov 18, 2021 at 4:26 PM Ingo Bürk <i...@ververica.com> wrote: > Hi, > > first of all, thanks for the summary of both sides, and for bringing up the > discussion on this. > I think it is obvious that this is not something we can just "break", so > the config option seems mandatory to me. > > Overall I agree with Martijn and Till that throwing errors is the more > expected behavior. I mostly think this is valuable default behavior because > it allows developers to find mistakes early and diagnose them much easier > compare to having to "work backwards" and figure out that it is the CAST > that failed. It also means that pipelines using TRY_CAST are > self-documenting because using that can signal "we might receive broken > data here". > > > Best > Ingo > > On Thu, Nov 18, 2021 at 9:11 AM Till Rohrmann <trohrm...@apache.org> > wrote: > > > Hi everyone, > > > > personally I would also prefer the system telling me that something is > > wrong instead of silently ignoring records. If there is a TRY_CAST > function > > that has the old behaviour, people can still get the old behaviour. For > > backwards compatibility reasons it is a good idea to introduce a switch > to > > get back the old behaviour. By default we could set it to the new > > behaviour, though. Of course, we should explicitly document this new > > behaviour so that people are aware of it before running their jobs for > days > > and then encountering an invalid input. > > > > Cheers, > > Till > > > > On Thu, Nov 18, 2021 at 9:02 AM Martijn Visser <mart...@ververica.com> > > wrote: > > > > > Hi Caizhi, > > > > > > Thanks for bringing this up for discussion. I think the important part > is > > > what do developers expect as the default behaviour of a CAST function > > when > > > casting fails. If I look at Postgres [1] or MSSQL [2], the default > > > behaviour of a CAST failing would be to return an error, which would be > > the > > > new behaviour. Returning a value when a CAST fails can lead to users > not > > > understanding immediately where that value comes from. So, I would be > in > > > favor of the new behaviour by default, but including a configuration > flag > > > to maintain the old behaviour to avoid that you need to rewrite all > these > > > jobs. > > > > > > Best regards, > > > > > > Martijn > > > > > > [1] https://www.postgresql.org/docs/current/sql-createcast.html > > > [2] > > > > > > > > > https://docs.microsoft.com/en-us/sql/t-sql/functions/try-cast-transact-sql?view=sql-server-ver15 > > > > > > On Thu, 18 Nov 2021 at 03:17, Caizhi Weng <tsreape...@gmail.com> > wrote: > > > > > > > Hi devs! > > > > > > > > We're discussing the behavior of casting functions (including cast, > > > > to_timestamp, to_date, etc.) for invalid input in > > > > https://issues.apache.org/jira/browse/FLINK-24924. As this topic is > > > > crucial > > > > to compatibility and usability we'd like to continue discussing this > > > > publicly in the mailing list. > > > > > > > > The main topic is to discuss that shall casting functions return null > > > (keep > > > > its current behavior) or throw exceptions (introduce a new behavior). > > I'm > > > > trying to conclude the ideas on both sides. Correct me if I miss > > > something. > > > > > > > > *From the devs who support throwing exceptions (new behavior):* > > > > > > > > The main concern is that if we silently return null then unexpected > > > results > > > > or exceptions (mainly NullPointerException) may be produced. However, > > it > > > > will be hard for users to reason for this because there is no > detailed > > > > message. If we throw exceptions in the first place, then it's much > > easier > > > > to catch the exception with nice detailed messages explaining what is > > > going > > > > wrong. Especially for this case of DATE/TIME/TIMESTAMP it's very > > helpful > > > to > > > > have a detailed error and see where and why the parsing broke. > > > > > > > > For compatibility concerns, we can provide a TRY_CAST function which > is > > > > exactly the same as the current CAST function by returning nulls for > > > > invalid input. > > > > > > > > *From the devs who support return null (current behavior):* > > > > > > > > The main concern is compatibility and usability. > > > > > > > > On usability: The upstream system may occasionally produce invalid > data > > > and > > > > if we throw exceptions when seeing this it will fail the job again > and > > > > again even after restart (because the invalid data is always > > > > there). Streaming computing is a resident program and users do not > want > > > it > > > > to frequently fail and cannot automatically recover. Most users are > > > willing > > > > to just skip that record and continue processing. Imagine an online > job > > > > running for a couple of weeks and suddenly fails due to some > unexpected > > > > dirty data. What choices do users have to quickly resume the job? > > > > > > > > On compatibility: There are currently thousands of users and tens of > > > > thousands of jobs relying on the current behavior to filter out > invalid > > > > input. If we change the behavior it will be a disaster for users as > > they > > > > have to rewrite and check their SQL very carefully. > > > > > > > > > > > > What do you think? We're looking forward to your feedback. > > > > > > > > > >