I opened https://issues.apache.org/jira/browse/FLINK-24396 to track this effort.

Not sure if this will happen in 1.15 already. We will needed automated compatibility tests and a well-defined list of stable API.

We can also do this incrementally and start with the interfaces for connectors.

Regards,
Timo


On 28.09.21 11:47, Leonard Xu wrote:
Thanks @peninx for the feedback, this will definitely help the flink community.

Recently, we also developed a series of connectors in Flink CDC project[1]. 
They are based on flink version 1.13.1, but many users still use flink version 
1.12.* in production. They have encountered similar problems, and it is 
difficult to upgrade the flink cluster version within their company.

Therefore, things like directly changing CatalogTable to ResolvedCatalogTable 
should not happen, we should have marked it as @Deprecated and keep at least 
one version for compatibility.

In one word, it's valuable feedback that we will pay more attention to API 
compatibility.

Best,
Leonard

[1] https://github.com/ververica/flink-cdc-connectors

在 2021年9月28日,17:16,Jeff Zhang <zjf...@gmail.com> 写道:

I believe I mentioned this before in the community, we (Zeppelin) use flink
api as well and would like to support multiple versions of flink in one
zeppelin version. For now we have to use reflection to achieve that.

https://github.com/apache/zeppelin/tree/master/flink


OpenInx <open...@gmail.com> 于2021年9月28日周二 下午5:10写道:

Thanks for the information, Martijin & Timo !

Since implementing a connector is not straightforward, we were expecting
that not many users implement custom connectors.

Currently, the apache iceberg & hudi are heavily depending on the
PublicEvolving API for their flink connectors.  I think apache hudi even
uses more public API than iceberg to implement their relatively complicated
flink sink DAG, I think Danny Chen [1] may want to provide more input.  API
compatibility has become one of the core reasons that downstream projects
maintainers vote to support a release or not because bandwidth from the
downstream projects are limited and we maintainers need to balance between
the community requirements and cost.  A great compatible flink release will
greatly save the maintenance cost (especially we flink release often ) and
we are also glad to make it a longer life cycle.

We therefore consider this part as a kind of "second level API" for which
we can evolve quicker.

That sounds great ! I'm glad to see that we are making the API more
friendly !

[1]. https://github.com/danny0405



On Tue, Sep 28, 2021 at 3:52 PM Timo Walther <twal...@apache.org> wrote:

Hi Zheng,

I'm very sorry for the inconvenience that we have caused with our API
changes. We are trying our best to avoid API breaking changes. Thanks
for giving us feedback.

There has been a reason why Table API was marked as @PublicEvolving
instead of @Public. Over the last two years, we have basically rewritten
the entire API [1] to digest the Blink merge and making the Table API
stable and ready for the future. We tried our best to give users 1-2
releases time to upgrade their implementations whenever we deprecated
API but we were aware that this might cause frustration, but hopefully
for the greater good. We have reworked type system, Catalog API, schema,
source/sinks, functions and much more. Flink 1.14 will hopefully be the
last release with major API changes. We could also mark most Table API
interfaces as `@Public` in 1.15.

For your mentioned incompatibility, I agree that the change from
CatalogTable to ResolvedCatalogTable was not very nice. Since
implementing a connector is not straight forward, we were expecting that
not many users implement custom connectors. We therefore consider this
part as kind of "second level API" for which we can evolve quicker. A
`context.getCatalogTable().getSchema()` should still work for 1.12 and
1.13, at least that was the goal.

Thanks again for the feedback. It was a good reminder and we will pay
more attention to this.

Regards,
Timo

[1]


https://cwiki.apache.org/confluence/display/FLINK/FLIP-32%3A+Restructure+flink-table+for+future+contributions


On 28.09.21 08:40, Martijn Visser wrote:
Hi Zheng,

Thanks for reaching out and sharing your frustration. No feelings are
hurt
and feedback is always welcome, because that's the only way we can
improve
for the future. API compatibility is a really important thing for us
while
also improving and building new capabilities. Let me investigate a bit
what
happened on our end, share that and then try to get some learnings out
of
it for the future. I'll get back to you in a couple of days.

Best regards,

Martijn Visser | Product Manager

mart...@ververica.com

<https://www.ververica.com/>


Follow us @VervericaData

--

Join Flink Forward <https://flink-forward.org/> - The Apache Flink
Conference

Stream Processing | Event Driven | Real Time


On Tue, 28 Sept 2021 at 07:39, OpenInx <open...@gmail.com> wrote:

Sorry about my unfriendly tone of the last e-mail, I got frustrated
about
the experience of maintaining the project which is closely with Flink.
My
intention was trying to remind everyone to be careful about API
compatibility and didn't really watch out for the tone I used.

Hope that doesn't hurt anyone's feelings.

On Tue, Sep 28, 2021 at 12:33 PM OpenInx <open...@gmail.com> wrote:

Hi Dev

We are trying to upgrade the flink version from 1.12.0 to 1.13.2 in
apache
iceberg project ( https://github.com/apache/iceberg/pull/3116),  but
it's
not a great experience.  We expect to support both flink1.12 and
flink1.13
in an iceberg-flink module without using the new API of flink1.13 for
saving maintenance cost,  but we find the iceberg-flink-runtime.jar
built
by flink 1.13 cannot works fine in flink 1.12 clusters because of the
basic
API compatibility was break when iterating flink 1.12 to flink1.13.2:

(The following are copied from the iceberg issue:
https://github.com/apache/iceberg/issues/3187#issuecomment-928755046
)

Thanks for the report, @Reo-LEI ! I think this issue was introduced
from
this apache flink PR (



https://github.com/apache/flink/pull/15316/files#diff-bd276ed951054125b39428ee61de103d9c7832246398f01514a574bb8e51757cR74
)
and FLINK-21913 (https://issues.apache.org/jira/browse/FLINK-21913),
it
just changed the returned data type from CatalogTable to
ResolvedCatalogTable without any compatibility guarantee. In this
case,
the
iceberg-flink-runtime jar which is compiled from apache flink 1.13
will
include the ResovledCatalogTable class inside it. Finally when we
package
this jar and submit the flink job to flink 1.12, the above
compatibility
issue happen.

As we all know, the DynamicTableFactory (



https://github.com/apache/flink/blob/99c2a415e9eeefafacf70762b6f54070f7911ceb/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/DynamicTableFactory.java
)
is a basic API which almost all flink connectors are built on top of
it.
The breaking compatibility makes the downstream projects really hard
to
deliver better compatibility to users, unless we iceberg maintain
different
modules for each maintained flink version (That's not the thing that
we
want to do).

The last flink upgrading work is also not a good experience (See the
discussion (https://github.com/apache/iceberg/pull/1956) and
comment (
https://github.com/apache/iceberg/pull/1956#discussion_r546534299)
),
because the flink 1.12 also breaks several API that was annotated
PublicEvolving in flink 1.11.0, that becomes one of the most
important
reasons leading to the conclusion that stops support flink 1.11.0 in
our
apache iceberg branch ( Supporting new features [such as flip-27
unified
iceberg source/sink] that depends the API introduced in flink 1.12 is
another reason). To better support the compatibility of downstream
systems
and delivering better experience to flink users, I will strongly
suggest
the Apache Flink community to pay more attention to ensuring API
compatibility.


Zheng Hu (openinx)

Thanks.








--
Best Regards

Jeff Zhang


Reply via email to