FYI: Opened https://github.com/apache/flink/pull/17985 which will introduce
the config option,
so we can continue working on the CAST fixes and improvements. It will be
very easy to flip
the default behaviour (currently on the PR: legacy = ENABLED) when this
discussion concludes,
and update the documentation accordingly.

On Mon, Nov 29, 2021 at 10:37 AM Marios Trivyzas <mat...@gmail.com> wrote:

> I definitely understand the argument to continue supporting the existing
> (return null) as the default behaviour.
> I'd like to point out though that if we decide this (option no.2) it's
> kind of unnatural, to directly drop the flag in *Flink 2.0*
> for example, and force the users at that point to use either *CAST *(throw
> errors) or *TRY_CAST* (define a default return value).
>
> The decision for the default value of this flag, is a commitment, because
> In my opinion, changing this default value in the future
> to throw errors instead, is not an option, as this will definitely confuse
> the users, so the next step (in future versions) would be to
> completely drop the flag and have the users choosing between *CAST* and
> *TRY_CAST.*
>
> Therefore, and speaking from a developing cycle perspective, my personal
> preference is to go with option no.1 which is in line
> with the usual approach (at least to my experience :)) in the open source
> software.
>
> Best regards,
> Marios
>
>
> On Tue, Nov 23, 2021 at 12:59 PM Martijn Visser <mart...@ververica.com>
> wrote:
>
>> Hi all,
>>
>> My conclusion at this point is that there is consensus that the default
>> behaviour of CAST should raise errors when it fails and that it should be
>> configurable via a configuration flag.
>>
>> The open discussion is on when we want to fix the current (incorrect)
>> behaviour:
>>
>> 1. Doing it in the next Flink release (1.15) by setting the configuration
>> flag to fail by default
>> 2. Keep the current (incorrect) behaviour in Flink 1.15 by setting the
>> configuration flag to the current situation and only changing this
>> if/whenever a Flink 2.0 version is released.
>>
>> From my perspective, I can understand that going for option 2 is a
>> preferred option for those that are running large Flink setups with a
>> great
>> number of users. I am wondering if those platforms also have the ability
>> to
>> set default values and/or override user configuration. That could be a way
>> to solve this issue for these platform teams.
>>
>> I would prefer to go for option 1, because I think correct execution of
>> CAST is important, especially for new Flink users. These new users should
>> have a smooth user experience and shouldn't need to change configuration
>> flags to get correct behaviour. I do expect that users who have used Flink
>> before are more familiar with checking release notes and interpreting how
>> this potentially affects them. That's why we have release notes. I also
>> doubt that we will have a Flink 2.0 release any time soon, meaning that we
>> are only going to make the pain even bigger for more users if we change
>> this incorrect behaviour at a later time.
>>
>> Best regards,
>>
>> Martijn
>>
>> On Tue, 23 Nov 2021 at 02:10, Kurt Young <ykt...@gmail.com> wrote:
>>
>> > > This is what I don't really understand here: how adding a
>> configuration
>> > option causes issues here?
>> > This is why: for most Flink production use cases I see, it's not like a
>> > couple of people manage ~5 Flink
>> > jobs, so they can easily track all the big changes in every minor Flink
>> > version. Typically use case are like
>> > a group of people managing some streaming platform, which will provide
>> > Flink as an execution engine
>> > to their users. Alibaba has more than 40K online streaming SQL jobs, and
>> > ByteDance also has a similar
>> > number. Most of the time, whether upgrading Flink version will be
>> > controlled by the user of the platform,
>> > not the platform provider. The platform will most likely provide
>> multiple
>> > Flink version support.
>> >
>> > Even if you can count on the platform provider to read all the release
>> > notes carefully, their users won't. So
>> > we are kind of throw the responsibility to all the platform provider,
>> make
>> > them to take care of the semantic
>> > changes. They have to find some good way to control the impactions when
>> > their users upgrade Flink's version.
>> > And if they don't find a good solution around this, and their users
>> > encounter some online issues, they will be
>> > blamed. And you can guess who they would blame.
>> >
>> > Flink is a very popular engine now, every decision we make will affect
>> the
>> > users a lot. If you want them to make
>> > some changes, I would argue we should make them think it's worth it.
>> >
>> > Best,
>> > Kurt
>> >
>> >
>> > On Mon, Nov 22, 2021 at 11:29 PM Francesco Guardiani <
>> > france...@ververica.com> wrote:
>> >
>> > > > NULL in SQL essentially means "UNKNOWN", it's not as scary as a
>> null in
>> > > java which will easily cause a NPE or some random behavior with a c++
>> > > function call.
>> > >
>> > > This is true from the user point of view, except our runtime doesn't
>> > treat
>> > > null as some value where you can safely execute operations and get
>> "noop"
>> > > results. In our runtime null is Java's null, hence causing issues and
>> > > generating NPEs here and there when nulls are not expected.
>> > >
>> > > > It will really create a big mess after users upgrade their SQL jobs
>> > >
>> > > This is what I don't really understand here: how adding a
>> configuration
>> > > option causes issues here? We make it very clear in our release notes
>> > that
>> > > you need to switch that flag if you're relying on this behavior and
>> > that's
>> > > it: if you reprocess jobs every time you upgrade, you just flip the
>> > switch
>> > > before reprocessing and you won't have any issues. If you don't
>> because
>> > you
>> > > use the hybrid source, either you upgrade your query or you flip the
>> flag
>> > > and in both cases this shouldn't generate any issue.
>> > > Since it's a big change, I also expect to keep this flag for some
>> > releases,
>> > > at least up to Flink 2.
>> > >
>> > > On Sat, Nov 20, 2021 at 7:25 AM Kurt Young <ykt...@gmail.com> wrote:
>> > >
>> > > > Hi Francesco,
>> > > >
>> > > > Thanks for sharing your opinion about this and examples with other
>> > > > programming
>> > > > languages. I just want to mention, that NULL in SQL world is a bit
>> > > > different with the
>> > > > meaning in programming languages like java.
>> > > >
>> > > > NULL in SQL essentially means "UNKNOWN", it's not as scary as a
>> null in
>> > > > java which
>> > > > will easily cause a NPE or some random behavior with a c++ function
>> > call.
>> > > > UNKNOWN
>> > > > means it could be any value. In java, the condition "null == null"
>> > always
>> > > > return true. But
>> > > > in SQL, it returns NULL, which means UNKNOWN.
>> > > >
>> > > > Another example, if you run following statements:
>> > > > select 'true' where 3 in (1, 2, 3, null) // this will print true
>> > > > select 'true' where 3 not in (1, 2, null) // this won't print
>> anything
>> > > >
>> > > > In summary, SQL's NULL is a bit different from others, it has its
>> own
>> > > > meaning. So I won't
>> > > > compare the behavior of returning NULL with programming languages
>> and
>> > > then
>> > > > judge it
>> > > > as bad behavior. And it's not a very big deal if we return NULL when
>> > > trying
>> > > > to cast "abc"
>> > > > to an integer, which means we don't know the correct value.
>> > > >
>> > > > But still, I'm ok to change the behavior, but just not now. It will
>> > > really
>> > > > create a big mess after
>> > > > users upgrade their SQL jobs. I'm either fine to do it in some
>> really
>> > big
>> > > > version change like
>> > > > Flink 2.0, or we can do it after we have some universal error
>> records
>> > > > handling mechanism, so
>> > > > in that way, users could have a chance to handle such a situation.
>> > > >
>> > > > Best,
>> > > > Kurt
>> > > >
>> > > >
>> > > > On Fri, Nov 19, 2021 at 7:29 PM Francesco Guardiani <
>> > > > france...@ververica.com>
>> > > > wrote:
>> > > >
>> > > > > Hi all,
>> > > > >
>> > > > > tl;dr:
>> > > > >
>> > > > > I think Timo pretty much said it all. As described in the issue,
>> my
>> > > > > proposal is:
>> > > > >
>> > > > > * Let's switch the default behavior of CAST to fail
>> > > > > * Let's add TRY_CAST to have the old behavior
>> > > > > * Let's add a rule (disabled by default) that wraps every CAST in
>> a
>> > > TRY,
>> > > > in
>> > > > > order to keep the old behavior.
>> > > > > * Let's put a giant warning in the release notes explaining to
>> enable
>> > > the
>> > > > > rule, in case you're depending on the old behavior
>> > > > >
>> > > > > This way, we break no SQL scripts, as you can apply this flag to
>> > every
>> > > > > previously running script. We can also think to another strategy,
>> > more
>> > > > than
>> > > > > the planner rule, to keep the old behavior, always behind a flag
>> > > disabled
>> > > > > by default.
>> > > > >
>> > > > > Timing of this proposal is also crucial, since CAST is a basic
>> > > primitive
>> > > > of
>> > > > > our language and, after we have the upgrade story in place, this
>> is
>> > > going
>> > > > > to be a whole more harder to deal with.
>> > > > >
>> > > > > And I would say that in the next future, we should start thinking
>> to
>> > > > > support proper error handling strategies, that is:
>> > > > >
>> > > > > * How users are supposed to handle records that fails an
>> expression
>> > > > > computation, aggregation, etc?
>> > > > > * Can we provide some default strategies, like log and discard,
>> send
>> > > to a
>> > > > > dead letter queue?
>> > > > >
>> > > > > Now let me go a bit more deep in the reason we really need such
>> > change:
>> > > > >
>> > > > > For me the issue is not really about being compliant with the SQL
>> > > > standard
>> > > > > or not, or the fact that other databases behaves differently from
>> us,
>> > > but
>> > > > > the fact that the CAST function we have right now is effectively a
>> > > > footgun
>> > > > > <https://en.wiktionary.org/wiki/footgun> for our users.
>> > > > > The concept of casting one value to another inherently involves
>> some
>> > > > > concept of failure, this is something as a programmer I expect,
>> > exactly
>> > > > > like when dividing a value by 0 or when sending a message to an
>> > > external
>> > > > > system. And this is why every programming language has some
>> explicit
>> > > way
>> > > > to
>> > > > > signal to you such failures exist and, some of them, even force
>> you
>> > to
>> > > > deal
>> > > > > with such failures, e.g. Java has the Exceptions and the try catch
>> > > block,
>> > > > > Rust has the ? operator, Golang returns you an error together with
>> > the
>> > > > > result. Not failing when a failure is inherently defined by the
>> > > operation
>> > > > > itself, or even not being explicit about the fact that such
>> operation
>> > > can
>> > > > > fail, leads users to mistakenly think the operation they're doing
>> is
>> > > > always
>> > > > > safe and cannot lead to failures. And this is IMHO really the
>> problem
>> > > > with
>> > > > > the CAST primitive we have right now: it has a concept of failure
>> but
>> > > we
>> > > > > shade it, and we're not even explicit about the fact that we're
>> > shading
>> > > > it
>> > > > > [1], and we expect users to go in the documentation and read that
>> > CAST
>> > > > can
>> > > > > return an eventual NULL and then deal with it. I even question why
>> > for
>> > > > > example, we return NULL more than a default sane value when an
>> > > exception
>> > > > > happens, e.g. it would be way better to return epoch 0 more than
>> NULL
>> > > > when
>> > > > > failing a cast from STRING to TIMESTAMP. This is for example the
>> > > approach
>> > > > > taken by Golang: parsing string to timestamp function returns
>> both a
>> > > sane
>> > > > > value and an error.
>> > > > >
>> > > > > And the cherry on top is that, for our users, the consequences of
>> the
>> > > bad
>> > > > > usage of CAST are simply disastrous: best case, some operator
>> fails
>> > > with
>> > > > > NPE, worst case you get bad results or even some data is lost
>> down in
>> > > the
>> > > > > pipeline. We give no indication at all that the cast failed, and
>> even
>> > > if
>> > > > we
>> > > > > push a change to log "hey this cast failed on this record" it
>> would
>> > > still
>> > > > > be extremely complicated to track down how badly a single cast
>> > failure
>> > > > > affected the results of a projection, a grouping, an aggregation,
>> > etc.
>> > > > > Hence my definition of our CAST function as a footgun.
>> > > > >
>> > > > > The bottom line for me is that our CAST primitive goes directly
>> > against
>> > > > the
>> > > > > goal of Flink SQL to provide a simple to use API for developers
>> and
>> > > > > business people to develop computation pipelines, because it's not
>> > > > > explicit, it silently fail with NULLs, and we require users to
>> deal
>> > > with
>> > > > > it.
>> > > > >
>> > > > > The very same discussion applies with TO_TIMESTAMP, which among
>> the
>> > > > others
>> > > > > might even be more crucial because we directly use it in our
>> > > > documentation
>> > > > > to tell users how to compute rowtime.
>> > > > >
>> > > > > FG
>> > > > >
>> > > > > [1] Note: here the naming is a fundamental part of the issue, the
>> > > > function
>> > > > > we have today is named CAST and not TRY_CAST or CAST_OR_NULL or
>> any
>> > > other
>> > > > > name giving the indication that the operation might fail and
>> provide
>> > a
>> > > > > result different from the cast result.
>> > > > >
>> > > > >
>> > > > > On Fri, Nov 19, 2021 at 4:00 AM Kurt Young <ykt...@gmail.com>
>> wrote:
>> > > > >
>> > > > > > Hi Timo,
>> > > > > >
>> > > > > > Regarding CAST, I think no one denies the standard behavior
>> which
>> > > > should
>> > > > > > raise errors when
>> > > > > > failed. The only question is how do we solve it, given lots of
>> > users
>> > > > > > already relying on current
>> > > > > > more tolerant behavior. Some violation of standard but
>> acceptable
>> > > > > behavior
>> > > > > > doesn't deserve
>> > > > > > a breaking change in Flink minor version IMO, i'm more
>> comfortable
>> > to
>> > > > fix
>> > > > > > it in versions like
>> > > > > > Flink 2.0.
>> > > > > >
>> > > > > > Best,
>> > > > > > Kurt
>> > > > > >
>> > > > > >
>> > > > > > On Thu, Nov 18, 2021 at 11:44 PM Timo Walther <
>> twal...@apache.org>
>> > > > > wrote:
>> > > > > >
>> > > > > > > Hi everyone,
>> > > > > > >
>> > > > > > >
>> > > > > > > thanks for finally have this discussion on the mailing list.
>> As
>> > > both
>> > > > a
>> > > > > > > contributor and user, I have experienced a couple issues
>> around
>> > > > > > > nullability coming out of nowhere in a pipeline. This
>> discussion
>> > > > should
>> > > > > > > not only cover CAST but failure handling in general.
>> > > > > > >
>> > > > > > > Let me summarize my opinion:
>> > > > > > >
>> > > > > > > 1) CAST vs. TRY_CAST
>> > > > > > >
>> > > > > > > CAST is a SQL standard core operation with well-defined
>> semantics
>> > > > > across
>> > > > > > > all major SQL vendors. There should be no discussion whether
>> it
>> > > > returns
>> > > > > > > NULL or an error. The semantics are already defined
>> externally. I
>> > > > don't
>> > > > > > > agree with "Streaming computing is a resident program ...
>> users
>> > do
>> > > > not
>> > > > > > > want it to frequently fail", the same argument is also true
>> for
>> > > > nightly
>> > > > > > > batch jobs. A batch job can also get stuck through a SQL
>> > statement
>> > > > that
>> > > > > > > is not lenient enough defined by the user.
>> > > > > > >
>> > > > > > > An option that restores the old behavior and TRY_CAST for the
>> > > future
>> > > > > > > should solve this use case and make all parties happy.
>> > > > > > >
>> > > > > > > 2) TO_TIMESTAMP / TO_DATE
>> > > > > > >
>> > > > > > > We should distinguish between CASTING and CONVERSION /
>> PARSING.
>> > As
>> > > a
>> > > > > > > user, I would expect that parsing can fail and have to deal
>> with
>> > > this
>> > > > > > > accordingly. Therefore, I'm fine with returning NULL in TO_ or
>> > > > CONVERT_
>> > > > > > > functions. This is also consistent with other vendors. Take
>> PARSE
>> > > of
>> > > > > SQL
>> > > > > > > Server as an example [1]: "If a parameter with a null value is
>> > > passed
>> > > > > at
>> > > > > > > run time, then a null is returned, to avoid canceling the
>> whole
>> > > > > batch.".
>> > > > > > > Here we can be more flexible with the semantics because users
>> > need
>> > > to
>> > > > > > > read the docs anyway.
>> > > > > > >
>> > > > > > > 3) Null at other locations
>> > > > > > >
>> > > > > > > In general, we should stick to our data type constraints.
>> > > Everything
>> > > > > > > else will mess up the architecture of functions/connectors and
>> > > their
>> > > > > > > return types. Take the rowtime (event-time timestamp)
>> attribute
>> > as
>> > > an
>> > > > > > > example: PRs like the one for FLINK-24885 are just the peak of
>> > the
>> > > > > > > iceberg. If we would allow rowtime columns to be NULL we would
>> > need
>> > > > to
>> > > > > > > check all time-based operators and implement additional
>> handling
>> > > > logic
>> > > > > > > for this.
>> > > > > > >
>> > > > > > > It would be better to define unified error-handling for
>> operators
>> > > and
>> > > > > > > maybe drop rows if the per-element processing failed. We
>> should
>> > > have
>> > > > a
>> > > > > > > unified approach how to log/side output such records.
>> > > > > > >
>> > > > > > > Until this is in place, I would suggest we spend some time in
>> > rules
>> > > > > that
>> > > > > > > can be enabled with an option for modifying the plan and wrap
>> > > > > frequently
>> > > > > > > failing expressions with a generic TRY() function. In this
>> case,
>> > we
>> > > > > > > don't need to deal with NULL in all built-in functions, we can
>> > > throw
>> > > > > > > helpful errors during development, and can return NULL even
>> > though
>> > > > the
>> > > > > > > return type is NOT NULL. It would also make the NULL returning
>> > > > explicit
>> > > > > > > in the plan.
>> > > > > > >
>> > > > > > > Regards,
>> > > > > > > Timo
>> > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > > > [1]
>> > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://docs.microsoft.com/en-us/sql/t-sql/functions/parse-transact-sql?view=sql-server-ver15
>> > > > > > > [2] https://issues.apache.org/jira/browse/FLINK-24885
>> > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > > > On 18.11.21 11:34, Kurt Young wrote:
>> > > > > > > > Sorry I forgot to add user ML. I also would like to gather
>> some
>> > > > users
>> > > > > > > > feedback on this thing.
>> > > > > > > > Since I didn't get any feedback on this topic before from
>> > users.
>> > > > > > > >
>> > > > > > > > Best,
>> > > > > > > > Kurt
>> > > > > > > >
>> > > > > > > >
>> > > > > > > > On Thu, Nov 18, 2021 at 6:33 PM Kurt Young <
>> ykt...@gmail.com>
>> > > > wrote:
>> > > > > > > >
>> > > > > > > >> (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.
>> > > > > > > >>>>>>
>> > > > > > > >>>>>
>> > > > > > > >>>>
>> > > > > > > >>>
>> > > > > > > >>
>> > > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>
>
> --
> Marios
>


-- 
Marios

Reply via email to