Hi all,

Agree with what has been said already. I've marked
https://issues.apache.org/jira/browse/FLINK-33523 as a blocker ticket
for 1.18.1

Best regards,

Martijn

On Thu, Dec 7, 2023 at 4:25 PM Márton Balassi <balassi.mar...@gmail.com> wrote:
>
> Thanks, for raising this Peter. +1 for reverting the change.
>
> Given the response from Timo and Aitozi, I believe it would be best if we
> could ship reverting the change in 1.18.1.
>
> On Thu, Dec 7, 2023 at 2:47 PM Aitozi <gjying1...@gmail.com> wrote:
>
> > Hi Peter, Timo
> >     Sorry for this breaking change, I didn't notice that was a breaking
> > change.
> > I'm +1 to revert the FLINK-33523
> >
> > Regards,
> > aitozi
> >
> > Timo Walther <twal...@apache.org> 于2023年12月7日周四 20:41写道:
> >
> > > Hi Peter,
> > >
> > > thanks for reaching out to the Flink community. This is indeed a serious
> > > issue. As the author of the Flink type system, DataType and many related
> > > utilities I strongly vote for reverting FLINK-33523:
> > >
> > > - It changes the Flink type system without a FLIP.
> > > - It breaks backwards compatibility with UDFs and connectors.
> > >
> > > Regards,
> > > Timo
> > >
> > > On 07.12.23 07:38, Péter Váry wrote:
> > > > Hi Team,
> > > >
> > > > We are working on upgrading the Iceberg-Flink connector from 1.17 to
> > > 1.18,
> > > > and found that some of our tests are failing. Prabhu Joseph created a
> > > jira
> > > > [1] to discuss this issue, along with short example code.
> > > >
> > > > In a nutshell:
> > > > - Create a table with an 'ARRAY<INT NOT NULL>' column
> > > > - Run a select which returns this column
> > > > - The return type changes:
> > > >      - From 'Object[]' - in 1.17
> > > >      - To 'int[]' - in 1.18
> > > >
> > > > The change is introduced by this jira [2].
> > > >
> > > > While I understand the reasoning behind this change, this will break
> > some
> > > > users existing workflow as evidenced by Xingcan Cui finding this
> > > > independently [3].
> > > >
> > > > What is the opinion of the community about this change?
> > > > - Do we want to revert the change?
> > > > - Do we ask the owners of the change to make this behavior
> > configurable?
> > > > - Do we accept this behavior change in a minor release?
> > > >
> > > > Thanks,
> > > > Peter
> > > >
> > > > [1] - https://issues.apache.org/jira/browse/FLINK-33523 - DataType
> > > > ARRAY<INT NOT NULL> fails to cast into Object[]
> > > > [2] - https://issues.apache.org/jira/browse/FLINK-31835 - DataTypeHint
> > > > don't support Row<i Array<int>>
> > > > [3] - https://issues.apache.org/jira/browse/FLINK-33547 - SQL
> > primitive
> > > > array type after upgrading to Flink 1.18.0
> > > >
> > >
> > >
> >

Reply via email to