Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-06-29 Thread Martijn Visser
Hi Xintong,

With regards to the deprecation of the Scala APIs, during the PR
review it was requested to not mark all APIs as deprecated but only
the entry point [1], to avoid a fully striked through experience in
the IDE. I think the same idea was applicable on the DataSet API. I
think it depends on how formal we want to treat this: if really
formal, then we should deprecate them in 1.18. I think in both cases,
it's quite well known that they are deprecated. I'm +0 for either way,
as long as we're all agreeing that they can be removed in 2.0.

With regards to Queryable State and Source/SinkFunction, +1 to mark
these as deprecated.

Best regards,

Martijn

[1] https://github.com/apache/flink/pull/21176#pullrequestreview-1159706808

On Thu, Jun 29, 2023 at 10:23 AM Xintong Song  wrote:
>
> Hi devs,
>
> Looking at the release 2.0 proposals [1], I noticed that many APIs that are
> proposed to be removed in 2.0 are not (fully) deprecated yet. We might want
> to properly mark them as `@Deprecated` in 1.18 if we agree they should be
> removed in 2.0. Moreover, according to FLIP-321 [2] (not voted yet but IMO
> is close to consensus IMO), a migration period is required after APIs are
> deprecated and before they can be removed.
>
> I might not be familiar with the status of all the APIs below. So I'd like
> to bring them up and see if there's any concern regarding deprecating them
> in 1.18. If there's concern for deprecating API, we can start a separate
> discussion thread for it. For those with no objections, I'd create JIRA
> tickets and try to properly deprecate them in 1.18.
>
> 1. DataSet API
> It's described as "legacy", "soft deprecated" in user documentation [3].
> However, it's not annotated with `@Deprecated` in codes. According to
> FLIP-131 [4], DataSet API should be deprecated when DataStream API and
> Table API / SQL meet certain requirements. AFAICS, all the requirements
> mentioned in the FLIP are already fulfilled. We should annotate it as
> `@Deprecated` now.
>
> 2. SourceFunction / SinkFunction
> They are described as deprecated in the roadmap[5], and I don't find
> anything regarding them in user documentation. But they are also not
> annotated with `@Deprecated` in codes. TBH, I'm not aware of any formal
> decision to deprecate these. AFAICS, the replacement for SourceFunction
> (Source) has already been promoted to `@Public`, while the replacement for
> SinkFunction (SinkV2) is still `@PublicEvolving`. I found a discussion[6]
> regarding promoting SinkV2 to `@Public`, but it's unclear to me what the
> conclusion is.
>
> 3. Queryable State
> It's described as approaching end-of-life in the roadmap [5], but is
> neither deprecated in codes nor in user documentation [7]. I also found a
> discussion [8] about rescuing it from deprecation, and it seems to me there
> are more negative opinions than positive ones.
>
> 4. All Scala APIs
> I think we agreed to drop Scala API support in FLIP-265 [9], and have tried
> to deprecate them in FLINK-29740 [10]. Also, both user documentation and
> roadmap[5] shows that scala API supports are deprecated. However, AFAICS,
> none of the APIs in `flink-streaming-scala` are annotated with
> `@Deprecated`, and only `ExecutionEnvironment` and `package` are marked
> `@Deprecated` in `flink-scala`.
>
> Looking forward to your feedback.
>
> Best,
>
> Xintong
>
>
> [1] https://cwiki.apache.org/confluence/display/FLINK/2.0+Release
>
> [2] https://lists.apache.org/thread/vmhzv8fcw2b33pqxp43486owrxbkd5x9
>
> [3]
> https://nightlies.apache.org/flink/flink-docs-master/docs/dev/dataset/overview/
>
> [4]
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
>
> [5] https://flink.apache.org/roadmap/
>
> [6] https://lists.apache.org/thread/q62nj89rrz0t5xtggy5n65on95f2rmmx
>
> [7]
> https://nightlies.apache.org/flink/flink-docs-master/docs/dev/datastream/fault-tolerance/queryable_state/
>
> [8] https://lists.apache.org/thread/9hmwcjb3q5c24pk3qshjvybfqk62v17m
>
> [9]
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-265+Deprecate+and+remove+Scala+API+support
>
> [10] https://issues.apache.org/jira/browse/FLINK-29740


Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-06-29 Thread Jing Ge
Hi Xintong,

2. SourceFunction / SinkFunction
This is one with challenges. There were many discussions in the past.

2.1 SourceFunction
The voting was passed[1], but there are still some ongoing issues[2]. The
SourceFunction is not marked as @deprecated yet[3], afaik.

2.2 SinkFunction
SinkV2 is a complex one and e.g. even the most important implementation
KafkaSink has not graduated to @Public yet[4]. FlieSink is @Experimental
and there were some concerns with data loss issues when I proposed to
graduate it to @PublicEvolving. It deserves a more serious graduation
process. Otherwise, users would not migrate to SinkV2 and all SinkFunction
and related implementations could not be removed. You can find more details
and other issues from Dong's and my replies at [5]

Best regards,
Jing

[1] https://lists.apache.org/thread/hrpsddgz65hjvhjozhg72s0wsmxz145p
[2] https://github.com/apache/flink/pull/20049
[3]
https://github.com/apache/flink/blob/c11fd82e0cc63904b43d2aca3a79d85dfeb57c2f/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java#L98
[4]
https://github.com/apache/flink-connector-kafka/blob/910fc5a3f06ab7d95f79c0fc4b5ce64e2aaa8db2/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java#L57
[5] https://lists.apache.org/thread/q62nj89rrz0t5xtggy5n65on95f2rmmx


On Thu, Jun 29, 2023 at 10:45 AM Martijn Visser 
wrote:

> Hi Xintong,
>
> With regards to the deprecation of the Scala APIs, during the PR
> review it was requested to not mark all APIs as deprecated but only
> the entry point [1], to avoid a fully striked through experience in
> the IDE. I think the same idea was applicable on the DataSet API. I
> think it depends on how formal we want to treat this: if really
> formal, then we should deprecate them in 1.18. I think in both cases,
> it's quite well known that they are deprecated. I'm +0 for either way,
> as long as we're all agreeing that they can be removed in 2.0.
>
> With regards to Queryable State and Source/SinkFunction, +1 to mark
> these as deprecated.
>
> Best regards,
>
> Martijn
>
> [1]
> https://github.com/apache/flink/pull/21176#pullrequestreview-1159706808
>
> On Thu, Jun 29, 2023 at 10:23 AM Xintong Song 
> wrote:
> >
> > Hi devs,
> >
> > Looking at the release 2.0 proposals [1], I noticed that many APIs that
> are
> > proposed to be removed in 2.0 are not (fully) deprecated yet. We might
> want
> > to properly mark them as `@Deprecated` in 1.18 if we agree they should be
> > removed in 2.0. Moreover, according to FLIP-321 [2] (not voted yet but
> IMO
> > is close to consensus IMO), a migration period is required after APIs are
> > deprecated and before they can be removed.
> >
> > I might not be familiar with the status of all the APIs below. So I'd
> like
> > to bring them up and see if there's any concern regarding deprecating
> them
> > in 1.18. If there's concern for deprecating API, we can start a separate
> > discussion thread for it. For those with no objections, I'd create JIRA
> > tickets and try to properly deprecate them in 1.18.
> >
> > 1. DataSet API
> > It's described as "legacy", "soft deprecated" in user documentation [3].
> > However, it's not annotated with `@Deprecated` in codes. According to
> > FLIP-131 [4], DataSet API should be deprecated when DataStream API and
> > Table API / SQL meet certain requirements. AFAICS, all the requirements
> > mentioned in the FLIP are already fulfilled. We should annotate it as
> > `@Deprecated` now.
> >
> > 2. SourceFunction / SinkFunction
> > They are described as deprecated in the roadmap[5], and I don't find
> > anything regarding them in user documentation. But they are also not
> > annotated with `@Deprecated` in codes. TBH, I'm not aware of any formal
> > decision to deprecate these. AFAICS, the replacement for SourceFunction
> > (Source) has already been promoted to `@Public`, while the replacement
> for
> > SinkFunction (SinkV2) is still `@PublicEvolving`. I found a discussion[6]
> > regarding promoting SinkV2 to `@Public`, but it's unclear to me what the
> > conclusion is.
> >
> > 3. Queryable State
> > It's described as approaching end-of-life in the roadmap [5], but is
> > neither deprecated in codes nor in user documentation [7]. I also found a
> > discussion [8] about rescuing it from deprecation, and it seems to me
> there
> > are more negative opinions than positive ones.
> >
> > 4. All Scala APIs
> > I think we agreed to drop Scala API support in FLIP-265 [9], and have
> tried
> > to deprecate them in FLINK-29740 [10]. Also, both user documentation and
> > roadmap[5] shows that scala API supports are deprecated. However, AFAICS,
> > none of the APIs in `flink-streaming-scala` are annotated with
> > `@Deprecated`, and only `ExecutionEnvironment` and `package` are marked
> > `@Deprecated` in `flink-scala`.
> >
> > Looking forward to your feedback.
> >
> > Best,
> >
> > Xintong
> >
> >
> > [1] https://cwiki.apache.org/confluence/display

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-06-29 Thread Alexander Fedulov
Hi Xintong,

Thanks for bringing up this topic. I can provide some details regarding
the SourceFunction deprecation efforts. Marking SourceFunction as
deprecated was not possible until now since we have stringent compiler
checks in flink-examples against using any deprecated APIs. We actually
merged the migration of all examples to the new FLIP-27-based
DataGeneratorSource [1] just two days ago [2]. Now the PR marking
it @Deprecated is finally unblocked [3] (I would be grateful if you could
merge it).

With regards to the Flink 2.0 scope, I compiled a list of items required to
be able to drop the SourceFunction API [4] a while ago and as you can see,
there is still quite some work to be done. Some items [5] might even
require additions to the new Source API. Overall, I am happy to take
ownership of completing this work package.

Best,
Alex


[1] https://cwiki.apache.org/confluence/x/9Av1D
[2] https://github.com/apache/flink/pull/21774
[3] https://github.com/apache/flink/pull/20049
[4] https://issues.apache.org/jira/browse/FLINK-28045
[5] https://issues.apache.org/jira/browse/FLINK-28054

On Thu, 29 Jun 2023 at 10:45, Martijn Visser 
wrote:

> Hi Xintong,
>
> With regards to the deprecation of the Scala APIs, during the PR
> review it was requested to not mark all APIs as deprecated but only
> the entry point [1], to avoid a fully striked through experience in
> the IDE. I think the same idea was applicable on the DataSet API. I
> think it depends on how formal we want to treat this: if really
> formal, then we should deprecate them in 1.18. I think in both cases,
> it's quite well known that they are deprecated. I'm +0 for either way,
> as long as we're all agreeing that they can be removed in 2.0.
>
> With regards to Queryable State and Source/SinkFunction, +1 to mark
> these as deprecated.
>
> Best regards,
>
> Martijn
>
> [1]
> https://github.com/apache/flink/pull/21176#pullrequestreview-1159706808
>
> On Thu, Jun 29, 2023 at 10:23 AM Xintong Song 
> wrote:
> >
> > Hi devs,
> >
> > Looking at the release 2.0 proposals [1], I noticed that many APIs that
> are
> > proposed to be removed in 2.0 are not (fully) deprecated yet. We might
> want
> > to properly mark them as `@Deprecated` in 1.18 if we agree they should be
> > removed in 2.0. Moreover, according to FLIP-321 [2] (not voted yet but
> IMO
> > is close to consensus IMO), a migration period is required after APIs are
> > deprecated and before they can be removed.
> >
> > I might not be familiar with the status of all the APIs below. So I'd
> like
> > to bring them up and see if there's any concern regarding deprecating
> them
> > in 1.18. If there's concern for deprecating API, we can start a separate
> > discussion thread for it. For those with no objections, I'd create JIRA
> > tickets and try to properly deprecate them in 1.18.
> >
> > 1. DataSet API
> > It's described as "legacy", "soft deprecated" in user documentation [3].
> > However, it's not annotated with `@Deprecated` in codes. According to
> > FLIP-131 [4], DataSet API should be deprecated when DataStream API and
> > Table API / SQL meet certain requirements. AFAICS, all the requirements
> > mentioned in the FLIP are already fulfilled. We should annotate it as
> > `@Deprecated` now.
> >
> > 2. SourceFunction / SinkFunction
> > They are described as deprecated in the roadmap[5], and I don't find
> > anything regarding them in user documentation. But they are also not
> > annotated with `@Deprecated` in codes. TBH, I'm not aware of any formal
> > decision to deprecate these. AFAICS, the replacement for SourceFunction
> > (Source) has already been promoted to `@Public`, while the replacement
> for
> > SinkFunction (SinkV2) is still `@PublicEvolving`. I found a discussion[6]
> > regarding promoting SinkV2 to `@Public`, but it's unclear to me what the
> > conclusion is.
> >
> > 3. Queryable State
> > It's described as approaching end-of-life in the roadmap [5], but is
> > neither deprecated in codes nor in user documentation [7]. I also found a
> > discussion [8] about rescuing it from deprecation, and it seems to me
> there
> > are more negative opinions than positive ones.
> >
> > 4. All Scala APIs
> > I think we agreed to drop Scala API support in FLIP-265 [9], and have
> tried
> > to deprecate them in FLINK-29740 [10]. Also, both user documentation and
> > roadmap[5] shows that scala API supports are deprecated. However, AFAICS,
> > none of the APIs in `flink-streaming-scala` are annotated with
> > `@Deprecated`, and only `ExecutionEnvironment` and `package` are marked
> > `@Deprecated` in `flink-scala`.
> >
> > Looking forward to your feedback.
> >
> > Best,
> >
> > Xintong
> >
> >
> > [1] https://cwiki.apache.org/confluence/display/FLINK/2.0+Release
> >
> > [2] https://lists.apache.org/thread/vmhzv8fcw2b33pqxp43486owrxbkd5x9
> >
> > [3]
> >
> https://nightlies.apache.org/flink/flink-docs-master/docs/dev/dataset/overview/
> >
> > [4]
> >
> https://cwiki.apache.org/confluence/pages/viewpa

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-06-29 Thread Xintong Song
Thanks for the inputs, Martijn, Jing and Alex.

@Martijn,
Regarding the Scala supports, I personally don't think "a fully striked
through experience in the IDE" is something we want to avoid, especially
given that we are planning to remove the deprecated APIs soon, unlike when
FLINK-29740 was resolved we didn't really know when they would be removed.
Moreover, the even entry point for DataStream Scala
(`StreamExecutionEnvironment`) is not annotated.


@Jing and @Alex,

IIUC, you mean SourceFunction can be annotated as `@Deprecated` in 1.18,
and there's already a PR doing so. However, after the deprecation, there
are still issues that need to be addressed before removing it in 2.0? This
sounds a bit weird. If the API cannot be dropped, which means without this
API some of functions cannot be supported, then how could it be deprecated?
How would we expect users to migrate away from it?


@Jing,

Sounds like it's impractical to deprecate SinkFunction in 1.18. Any
expectation / plan on when / how it can be deprecated / removed?


Best,

Xintong



On Thu, Jun 29, 2023 at 6:12 PM Alexander Fedulov <
alexander.fedu...@gmail.com> wrote:

> Hi Xintong,
>
> Thanks for bringing up this topic. I can provide some details regarding
> the SourceFunction deprecation efforts. Marking SourceFunction as
> deprecated was not possible until now since we have stringent compiler
> checks in flink-examples against using any deprecated APIs. We actually
> merged the migration of all examples to the new FLIP-27-based
> DataGeneratorSource [1] just two days ago [2]. Now the PR marking
> it @Deprecated is finally unblocked [3] (I would be grateful if you could
> merge it).
>
> With regards to the Flink 2.0 scope, I compiled a list of items required to
> be able to drop the SourceFunction API [4] a while ago and as you can see,
> there is still quite some work to be done. Some items [5] might even
> require additions to the new Source API. Overall, I am happy to take
> ownership of completing this work package.
>
> Best,
> Alex
>
>
> [1] https://cwiki.apache.org/confluence/x/9Av1D
> [2] https://github.com/apache/flink/pull/21774
> [3] https://github.com/apache/flink/pull/20049
> [4] https://issues.apache.org/jira/browse/FLINK-28045
> [5] https://issues.apache.org/jira/browse/FLINK-28054
>
> On Thu, 29 Jun 2023 at 10:45, Martijn Visser 
> wrote:
>
> > Hi Xintong,
> >
> > With regards to the deprecation of the Scala APIs, during the PR
> > review it was requested to not mark all APIs as deprecated but only
> > the entry point [1], to avoid a fully striked through experience in
> > the IDE. I think the same idea was applicable on the DataSet API. I
> > think it depends on how formal we want to treat this: if really
> > formal, then we should deprecate them in 1.18. I think in both cases,
> > it's quite well known that they are deprecated. I'm +0 for either way,
> > as long as we're all agreeing that they can be removed in 2.0.
> >
> > With regards to Queryable State and Source/SinkFunction, +1 to mark
> > these as deprecated.
> >
> > Best regards,
> >
> > Martijn
> >
> > [1]
> > https://github.com/apache/flink/pull/21176#pullrequestreview-1159706808
> >
> > On Thu, Jun 29, 2023 at 10:23 AM Xintong Song 
> > wrote:
> > >
> > > Hi devs,
> > >
> > > Looking at the release 2.0 proposals [1], I noticed that many APIs that
> > are
> > > proposed to be removed in 2.0 are not (fully) deprecated yet. We might
> > want
> > > to properly mark them as `@Deprecated` in 1.18 if we agree they should
> be
> > > removed in 2.0. Moreover, according to FLIP-321 [2] (not voted yet but
> > IMO
> > > is close to consensus IMO), a migration period is required after APIs
> are
> > > deprecated and before they can be removed.
> > >
> > > I might not be familiar with the status of all the APIs below. So I'd
> > like
> > > to bring them up and see if there's any concern regarding deprecating
> > them
> > > in 1.18. If there's concern for deprecating API, we can start a
> separate
> > > discussion thread for it. For those with no objections, I'd create JIRA
> > > tickets and try to properly deprecate them in 1.18.
> > >
> > > 1. DataSet API
> > > It's described as "legacy", "soft deprecated" in user documentation
> [3].
> > > However, it's not annotated with `@Deprecated` in codes. According to
> > > FLIP-131 [4], DataSet API should be deprecated when DataStream API and
> > > Table API / SQL meet certain requirements. AFAICS, all the requirements
> > > mentioned in the FLIP are already fulfilled. We should annotate it as
> > > `@Deprecated` now.
> > >
> > > 2. SourceFunction / SinkFunction
> > > They are described as deprecated in the roadmap[5], and I don't find
> > > anything regarding them in user documentation. But they are also not
> > > annotated with `@Deprecated` in codes. TBH, I'm not aware of any formal
> > > decision to deprecate these. AFAICS, the replacement for SourceFunction
> > > (Source) has already been promoted to `@Public`, while the replacemen

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-06-29 Thread Alexander Fedulov
@Xintong
The original discussion [1] and vote [2] converged on the idea that it is
better
to make it clear to the users that they should stop using SourceFunction
since it
is going away. The longer we do not have this indication, the more user
implementations will be based on it and the more pain will be induced when
we
finally drop it. Users now have an alternative API that they should use and
which
is fully functional, from that perspective nothing blocks marking it
@Deprecated.
As for the remaining work items - there are primarily three kinds:

1. Where Flink internally uses SourceFunction, without exposing this fact
to the
   outside world:
   - FLINK-28050 [3]
   - FLINK-28229 [4]
   - FLINK-28048 [5]

2. Very specific edge cases that might not be covered by the Source API as
is:
   - FLINK-28054 [6]
   - FLINK-28051 [7]

3. Usability improvements - something that was easily doable with
SourceFunction,
   but requires deep knowledge of the new, significantly more complex,
Source API
   to achieve:
   - FLINK-28056 [8]

In my mind, none of those are blockers for proceeding with adding the
@Deprecated
annotation:
(1) is a simple case of encapsulation, internals should not concern the API
users
(2) is really only relevant for "exotic" use cases. Does not mean we should
not
consider those, but since it is irrelevant for 99.9% of the users, I do not
think
we should get stuck here.
(3) is purely a nice to have. Formally speaking, all of the tools are
there, it is
just that due to the complexity of the new Source API some "simple" things
become
non-trivial and ideally we want to do better here.

[1] https://lists.apache.org/thread/d6cwqw9b3105wcpdkwq7rr4s7x4ywqr9
[2] https://lists.apache.org/thread/kv9rj3w2rmkb8jtss5bqffhw57or7v8v
[3] https://issues.apache.org/jira/browse/FLINK-28050
[4] https://issues.apache.org/jira/browse/FLINK-28229
[5] https://issues.apache.org/jira/browse/FLINK-28048
[6] https://issues.apache.org/jira/browse/FLINK-28054
[7] https://issues.apache.org/jira/browse/FLINK-28051
[8] https://issues.apache.org/jira/browse/FLINK-28056

On Thu, 29 Jun 2023 at 13:13, Xintong Song  wrote:

> Thanks for the inputs, Martijn, Jing and Alex.
>
> @Martijn,
> Regarding the Scala supports, I personally don't think "a fully striked
> through experience in the IDE" is something we want to avoid, especially
> given that we are planning to remove the deprecated APIs soon, unlike when
> FLINK-29740 was resolved we didn't really know when they would be removed.
> Moreover, the even entry point for DataStream Scala
> (`StreamExecutionEnvironment`) is not annotated.
>
>
> @Jing and @Alex,
>
> IIUC, you mean SourceFunction can be annotated as `@Deprecated` in 1.18,
> and there's already a PR doing so. However, after the deprecation, there
> are still issues that need to be addressed before removing it in 2.0? This
> sounds a bit weird. If the API cannot be dropped, which means without this
> API some of functions cannot be supported, then how could it be deprecated?
> How would we expect users to migrate away from it?
>
>
> @Jing,
>
> Sounds like it's impractical to deprecate SinkFunction in 1.18. Any
> expectation / plan on when / how it can be deprecated / removed?
>
>
> Best,
>
> Xintong
>
>
>
> On Thu, Jun 29, 2023 at 6:12 PM Alexander Fedulov <
> alexander.fedu...@gmail.com> wrote:
>
> > Hi Xintong,
> >
> > Thanks for bringing up this topic. I can provide some details regarding
> > the SourceFunction deprecation efforts. Marking SourceFunction as
> > deprecated was not possible until now since we have stringent compiler
> > checks in flink-examples against using any deprecated APIs. We actually
> > merged the migration of all examples to the new FLIP-27-based
> > DataGeneratorSource [1] just two days ago [2]. Now the PR marking
> > it @Deprecated is finally unblocked [3] (I would be grateful if you could
> > merge it).
> >
> > With regards to the Flink 2.0 scope, I compiled a list of items required
> to
> > be able to drop the SourceFunction API [4] a while ago and as you can
> see,
> > there is still quite some work to be done. Some items [5] might even
> > require additions to the new Source API. Overall, I am happy to take
> > ownership of completing this work package.
> >
> > Best,
> > Alex
> >
> >
> > [1] https://cwiki.apache.org/confluence/x/9Av1D
> > [2] https://github.com/apache/flink/pull/21774
> > [3] https://github.com/apache/flink/pull/20049
> > [4] https://issues.apache.org/jira/browse/FLINK-28045
> > [5] https://issues.apache.org/jira/browse/FLINK-28054
> >
> > On Thu, 29 Jun 2023 at 10:45, Martijn Visser 
> > wrote:
> >
> > > Hi Xintong,
> > >
> > > With regards to the deprecation of the Scala APIs, during the PR
> > > review it was requested to not mark all APIs as deprecated but only
> > > the entry point [1], to avoid a fully striked through experience in
> > > the IDE. I think the same idea was applicable on the DataSet API. I
> > > think it depends on how formal we want to treat this: if 

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-06-29 Thread Xintong Song
Thanks for the explanation, Alex.

Not blocking the deprecation on 1 & 3 makes sense to me. However, I'm not
sure about 2.

It sounds to me that, without FLINK-28051 & FLINK-28054, some of the
connectors cannot migrate to the new Source API, or at least further
investigation is needed to understand the situation. If this is the case,
we probably should not deprecate the API until these issues are resolved.
Do you think it is feasible to resolve them by the feature freeze date of
1.18?

Best,

Xintong



On Thu, Jun 29, 2023 at 8:02 PM Alexander Fedulov <
alexander.fedu...@gmail.com> wrote:

> @Xintong
> The original discussion [1] and vote [2] converged on the idea that it is
> better
> to make it clear to the users that they should stop using SourceFunction
> since it
> is going away. The longer we do not have this indication, the more user
> implementations will be based on it and the more pain will be induced when
> we
> finally drop it. Users now have an alternative API that they should use and
> which
> is fully functional, from that perspective nothing blocks marking it
> @Deprecated.
> As for the remaining work items - there are primarily three kinds:
>
> 1. Where Flink internally uses SourceFunction, without exposing this fact
> to the
>outside world:
>- FLINK-28050 [3]
>- FLINK-28229 [4]
>- FLINK-28048 [5]
>
> 2. Very specific edge cases that might not be covered by the Source API as
> is:
>- FLINK-28054 [6]
>- FLINK-28051 [7]
>
> 3. Usability improvements - something that was easily doable with
> SourceFunction,
>but requires deep knowledge of the new, significantly more complex,
> Source API
>to achieve:
>- FLINK-28056 [8]
>
> In my mind, none of those are blockers for proceeding with adding the
> @Deprecated
> annotation:
> (1) is a simple case of encapsulation, internals should not concern the API
> users
> (2) is really only relevant for "exotic" use cases. Does not mean we should
> not
> consider those, but since it is irrelevant for 99.9% of the users, I do not
> think
> we should get stuck here.
> (3) is purely a nice to have. Formally speaking, all of the tools are
> there, it is
> just that due to the complexity of the new Source API some "simple" things
> become
> non-trivial and ideally we want to do better here.
>
> [1] https://lists.apache.org/thread/d6cwqw9b3105wcpdkwq7rr4s7x4ywqr9
> [2] https://lists.apache.org/thread/kv9rj3w2rmkb8jtss5bqffhw57or7v8v
> [3] https://issues.apache.org/jira/browse/FLINK-28050
> [4] https://issues.apache.org/jira/browse/FLINK-28229
> [5] https://issues.apache.org/jira/browse/FLINK-28048
> [6] https://issues.apache.org/jira/browse/FLINK-28054
> [7] https://issues.apache.org/jira/browse/FLINK-28051
> [8] https://issues.apache.org/jira/browse/FLINK-28056
>
> On Thu, 29 Jun 2023 at 13:13, Xintong Song  wrote:
>
> > Thanks for the inputs, Martijn, Jing and Alex.
> >
> > @Martijn,
> > Regarding the Scala supports, I personally don't think "a fully striked
> > through experience in the IDE" is something we want to avoid, especially
> > given that we are planning to remove the deprecated APIs soon, unlike
> when
> > FLINK-29740 was resolved we didn't really know when they would be
> removed.
> > Moreover, the even entry point for DataStream Scala
> > (`StreamExecutionEnvironment`) is not annotated.
> >
> >
> > @Jing and @Alex,
> >
> > IIUC, you mean SourceFunction can be annotated as `@Deprecated` in 1.18,
> > and there's already a PR doing so. However, after the deprecation, there
> > are still issues that need to be addressed before removing it in 2.0?
> This
> > sounds a bit weird. If the API cannot be dropped, which means without
> this
> > API some of functions cannot be supported, then how could it be
> deprecated?
> > How would we expect users to migrate away from it?
> >
> >
> > @Jing,
> >
> > Sounds like it's impractical to deprecate SinkFunction in 1.18. Any
> > expectation / plan on when / how it can be deprecated / removed?
> >
> >
> > Best,
> >
> > Xintong
> >
> >
> >
> > On Thu, Jun 29, 2023 at 6:12 PM Alexander Fedulov <
> > alexander.fedu...@gmail.com> wrote:
> >
> > > Hi Xintong,
> > >
> > > Thanks for bringing up this topic. I can provide some details regarding
> > > the SourceFunction deprecation efforts. Marking SourceFunction as
> > > deprecated was not possible until now since we have stringent compiler
> > > checks in flink-examples against using any deprecated APIs. We actually
> > > merged the migration of all examples to the new FLIP-27-based
> > > DataGeneratorSource [1] just two days ago [2]. Now the PR marking
> > > it @Deprecated is finally unblocked [3] (I would be grateful if you
> could
> > > merge it).
> > >
> > > With regards to the Flink 2.0 scope, I compiled a list of items
> required
> > to
> > > be able to drop the SourceFunction API [4] a while ago and as you can
> > see,
> > > there is still quite some work to be done. Some items [5] might even
> > > require additions to the new Source

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-06-29 Thread Alexander Fedulov
> However, I'm not sure about 2.

I am not aware of a bylaw that states the specific requirements in order to
mark something as @Deprecated. My understanding from the discussion and the
vote was that the community recognizes the necessity to make it explicit
that
the usage of the SourceFunction API is discouraged. This can actually
stimulate
authors of connectors that rely on this very specific and non-baseline
functionality to contribute extensions to the new Source API themselves in
order to
close the gap. ExternallyInducedSource, for instance, was driven by Pravega
to
begin with, since it was only needed for their purposes [1]. We are not
removing
anything - until 2.0 everything will continue to work and we can work on
resolving the limitations until then, I personally don't see a big issue
here.

>Do you think it is feasible to resolve them by the feature freeze date of
1.18?
No, these are rather complex additions that would probably require FLIP(s).

[1]
https://flink.apache.org/2022/01/20/pravega-flink-connector-101/#checkpoint-integration

On Thu, 29 Jun 2023 at 14:25, Xintong Song  wrote:

> Thanks for the explanation, Alex.
>
> Not blocking the deprecation on 1 & 3 makes sense to me. However, I'm not
> sure about 2.
>
> It sounds to me that, without FLINK-28051 & FLINK-28054, some of the
> connectors cannot migrate to the new Source API, or at least further
> investigation is needed to understand the situation. If this is the case,
> we probably should not deprecate the API until these issues are resolved.
> Do you think it is feasible to resolve them by the feature freeze date of
> 1.18?
>
> Best,
>
> Xintong
>
>
>
> On Thu, Jun 29, 2023 at 8:02 PM Alexander Fedulov <
> alexander.fedu...@gmail.com> wrote:
>
> > @Xintong
> > The original discussion [1] and vote [2] converged on the idea that it is
> > better
> > to make it clear to the users that they should stop using SourceFunction
> > since it
> > is going away. The longer we do not have this indication, the more user
> > implementations will be based on it and the more pain will be induced
> when
> > we
> > finally drop it. Users now have an alternative API that they should use
> and
> > which
> > is fully functional, from that perspective nothing blocks marking it
> > @Deprecated.
> > As for the remaining work items - there are primarily three kinds:
> >
> > 1. Where Flink internally uses SourceFunction, without exposing this fact
> > to the
> >outside world:
> >- FLINK-28050 [3]
> >- FLINK-28229 [4]
> >- FLINK-28048 [5]
> >
> > 2. Very specific edge cases that might not be covered by the Source API
> as
> > is:
> >- FLINK-28054 [6]
> >- FLINK-28051 [7]
> >
> > 3. Usability improvements - something that was easily doable with
> > SourceFunction,
> >but requires deep knowledge of the new, significantly more complex,
> > Source API
> >to achieve:
> >- FLINK-28056 [8]
> >
> > In my mind, none of those are blockers for proceeding with adding the
> > @Deprecated
> > annotation:
> > (1) is a simple case of encapsulation, internals should not concern the
> API
> > users
> > (2) is really only relevant for "exotic" use cases. Does not mean we
> should
> > not
> > consider those, but since it is irrelevant for 99.9% of the users, I do
> not
> > think
> > we should get stuck here.
> > (3) is purely a nice to have. Formally speaking, all of the tools are
> > there, it is
> > just that due to the complexity of the new Source API some "simple"
> things
> > become
> > non-trivial and ideally we want to do better here.
> >
> > [1] https://lists.apache.org/thread/d6cwqw9b3105wcpdkwq7rr4s7x4ywqr9
> > [2] https://lists.apache.org/thread/kv9rj3w2rmkb8jtss5bqffhw57or7v8v
> > [3] https://issues.apache.org/jira/browse/FLINK-28050
> > [4] https://issues.apache.org/jira/browse/FLINK-28229
> > [5] https://issues.apache.org/jira/browse/FLINK-28048
> > [6] https://issues.apache.org/jira/browse/FLINK-28054
> > [7] https://issues.apache.org/jira/browse/FLINK-28051
> > [8] https://issues.apache.org/jira/browse/FLINK-28056
> >
> > On Thu, 29 Jun 2023 at 13:13, Xintong Song 
> wrote:
> >
> > > Thanks for the inputs, Martijn, Jing and Alex.
> > >
> > > @Martijn,
> > > Regarding the Scala supports, I personally don't think "a fully striked
> > > through experience in the IDE" is something we want to avoid,
> especially
> > > given that we are planning to remove the deprecated APIs soon, unlike
> > when
> > > FLINK-29740 was resolved we didn't really know when they would be
> > removed.
> > > Moreover, the even entry point for DataStream Scala
> > > (`StreamExecutionEnvironment`) is not annotated.
> > >
> > >
> > > @Jing and @Alex,
> > >
> > > IIUC, you mean SourceFunction can be annotated as `@Deprecated` in
> 1.18,
> > > and there's already a PR doing so. However, after the deprecation,
> there
> > > are still issues that need to be addressed before removing it in 2.0?
> > This
> > > sounds a bit weird. If the API cannot be dropped, which 

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-06-29 Thread Xintong Song
I see. Thanks for the explanation. I may have not looked into this deeply
enough, and would trust the decision from you and the community members who
participated in the discussion & vote.

Best,

Xintong



On Thu, Jun 29, 2023 at 10:28 PM Alexander Fedulov <
alexander.fedu...@gmail.com> wrote:

> > However, I'm not sure about 2.
>
> I am not aware of a bylaw that states the specific requirements in order to
> mark something as @Deprecated. My understanding from the discussion and the
> vote was that the community recognizes the necessity to make it explicit
> that
> the usage of the SourceFunction API is discouraged. This can actually
> stimulate
> authors of connectors that rely on this very specific and non-baseline
> functionality to contribute extensions to the new Source API themselves in
> order to
> close the gap. ExternallyInducedSource, for instance, was driven by Pravega
> to
> begin with, since it was only needed for their purposes [1]. We are not
> removing
> anything - until 2.0 everything will continue to work and we can work on
> resolving the limitations until then, I personally don't see a big issue
> here.
>
> >Do you think it is feasible to resolve them by the feature freeze date of
> 1.18?
> No, these are rather complex additions that would probably require FLIP(s).
>
> [1]
>
> https://flink.apache.org/2022/01/20/pravega-flink-connector-101/#checkpoint-integration
>
> On Thu, 29 Jun 2023 at 14:25, Xintong Song  wrote:
>
> > Thanks for the explanation, Alex.
> >
> > Not blocking the deprecation on 1 & 3 makes sense to me. However, I'm not
> > sure about 2.
> >
> > It sounds to me that, without FLINK-28051 & FLINK-28054, some of the
> > connectors cannot migrate to the new Source API, or at least further
> > investigation is needed to understand the situation. If this is the case,
> > we probably should not deprecate the API until these issues are resolved.
> > Do you think it is feasible to resolve them by the feature freeze date of
> > 1.18?
> >
> > Best,
> >
> > Xintong
> >
> >
> >
> > On Thu, Jun 29, 2023 at 8:02 PM Alexander Fedulov <
> > alexander.fedu...@gmail.com> wrote:
> >
> > > @Xintong
> > > The original discussion [1] and vote [2] converged on the idea that it
> is
> > > better
> > > to make it clear to the users that they should stop using
> SourceFunction
> > > since it
> > > is going away. The longer we do not have this indication, the more user
> > > implementations will be based on it and the more pain will be induced
> > when
> > > we
> > > finally drop it. Users now have an alternative API that they should use
> > and
> > > which
> > > is fully functional, from that perspective nothing blocks marking it
> > > @Deprecated.
> > > As for the remaining work items - there are primarily three kinds:
> > >
> > > 1. Where Flink internally uses SourceFunction, without exposing this
> fact
> > > to the
> > >outside world:
> > >- FLINK-28050 [3]
> > >- FLINK-28229 [4]
> > >- FLINK-28048 [5]
> > >
> > > 2. Very specific edge cases that might not be covered by the Source API
> > as
> > > is:
> > >- FLINK-28054 [6]
> > >- FLINK-28051 [7]
> > >
> > > 3. Usability improvements - something that was easily doable with
> > > SourceFunction,
> > >but requires deep knowledge of the new, significantly more complex,
> > > Source API
> > >to achieve:
> > >- FLINK-28056 [8]
> > >
> > > In my mind, none of those are blockers for proceeding with adding the
> > > @Deprecated
> > > annotation:
> > > (1) is a simple case of encapsulation, internals should not concern the
> > API
> > > users
> > > (2) is really only relevant for "exotic" use cases. Does not mean we
> > should
> > > not
> > > consider those, but since it is irrelevant for 99.9% of the users, I do
> > not
> > > think
> > > we should get stuck here.
> > > (3) is purely a nice to have. Formally speaking, all of the tools are
> > > there, it is
> > > just that due to the complexity of the new Source API some "simple"
> > things
> > > become
> > > non-trivial and ideally we want to do better here.
> > >
> > > [1] https://lists.apache.org/thread/d6cwqw9b3105wcpdkwq7rr4s7x4ywqr9
> > > [2] https://lists.apache.org/thread/kv9rj3w2rmkb8jtss5bqffhw57or7v8v
> > > [3] https://issues.apache.org/jira/browse/FLINK-28050
> > > [4] https://issues.apache.org/jira/browse/FLINK-28229
> > > [5] https://issues.apache.org/jira/browse/FLINK-28048
> > > [6] https://issues.apache.org/jira/browse/FLINK-28054
> > > [7] https://issues.apache.org/jira/browse/FLINK-28051
> > > [8] https://issues.apache.org/jira/browse/FLINK-28056
> > >
> > > On Thu, 29 Jun 2023 at 13:13, Xintong Song 
> > wrote:
> > >
> > > > Thanks for the inputs, Martijn, Jing and Alex.
> > > >
> > > > @Martijn,
> > > > Regarding the Scala supports, I personally don't think "a fully
> striked
> > > > through experience in the IDE" is something we want to avoid,
> > especially
> > > > given that we are planning to remove the deprecated APIs soon, unlike
> > > 

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-07-03 Thread Jing Ge
Hi Xingtong,

Option 1, secure plan would be:

1. graduate kafka, File, JDBC connectors to @Public
2. graduate SinkV2 to @Public
3. remove SinkFunction.

Option 2, risky plan but at a fast pace:

1. graduate SinkV2 to @Public and expecting more maintenance effort since
there are many known and unsolved issues.
2. remove SinkFunction.
3. It depends on the connectors' contributors whether connectors can
upgrade to Flink 2.0, since we moved forward with SinkV2 API without taking
care of implementations in external connectors.

I am ok with both of them and personally prefer option 1.

Best regards,
Jing


On Fri, Jun 30, 2023 at 3:41 AM Xintong Song  wrote:

> I see. Thanks for the explanation. I may have not looked into this deeply
> enough, and would trust the decision from you and the community members who
> participated in the discussion & vote.
>
> Best,
>
> Xintong
>
>
>
> On Thu, Jun 29, 2023 at 10:28 PM Alexander Fedulov <
> alexander.fedu...@gmail.com> wrote:
>
> > > However, I'm not sure about 2.
> >
> > I am not aware of a bylaw that states the specific requirements in order
> to
> > mark something as @Deprecated. My understanding from the discussion and
> the
> > vote was that the community recognizes the necessity to make it explicit
> > that
> > the usage of the SourceFunction API is discouraged. This can actually
> > stimulate
> > authors of connectors that rely on this very specific and non-baseline
> > functionality to contribute extensions to the new Source API themselves
> in
> > order to
> > close the gap. ExternallyInducedSource, for instance, was driven by
> Pravega
> > to
> > begin with, since it was only needed for their purposes [1]. We are not
> > removing
> > anything - until 2.0 everything will continue to work and we can work on
> > resolving the limitations until then, I personally don't see a big issue
> > here.
> >
> > >Do you think it is feasible to resolve them by the feature freeze date
> of
> > 1.18?
> > No, these are rather complex additions that would probably require
> FLIP(s).
> >
> > [1]
> >
> >
> https://flink.apache.org/2022/01/20/pravega-flink-connector-101/#checkpoint-integration
> >
> > On Thu, 29 Jun 2023 at 14:25, Xintong Song 
> wrote:
> >
> > > Thanks for the explanation, Alex.
> > >
> > > Not blocking the deprecation on 1 & 3 makes sense to me. However, I'm
> not
> > > sure about 2.
> > >
> > > It sounds to me that, without FLINK-28051 & FLINK-28054, some of the
> > > connectors cannot migrate to the new Source API, or at least further
> > > investigation is needed to understand the situation. If this is the
> case,
> > > we probably should not deprecate the API until these issues are
> resolved.
> > > Do you think it is feasible to resolve them by the feature freeze date
> of
> > > 1.18?
> > >
> > > Best,
> > >
> > > Xintong
> > >
> > >
> > >
> > > On Thu, Jun 29, 2023 at 8:02 PM Alexander Fedulov <
> > > alexander.fedu...@gmail.com> wrote:
> > >
> > > > @Xintong
> > > > The original discussion [1] and vote [2] converged on the idea that
> it
> > is
> > > > better
> > > > to make it clear to the users that they should stop using
> > SourceFunction
> > > > since it
> > > > is going away. The longer we do not have this indication, the more
> user
> > > > implementations will be based on it and the more pain will be induced
> > > when
> > > > we
> > > > finally drop it. Users now have an alternative API that they should
> use
> > > and
> > > > which
> > > > is fully functional, from that perspective nothing blocks marking it
> > > > @Deprecated.
> > > > As for the remaining work items - there are primarily three kinds:
> > > >
> > > > 1. Where Flink internally uses SourceFunction, without exposing this
> > fact
> > > > to the
> > > >outside world:
> > > >- FLINK-28050 [3]
> > > >- FLINK-28229 [4]
> > > >- FLINK-28048 [5]
> > > >
> > > > 2. Very specific edge cases that might not be covered by the Source
> API
> > > as
> > > > is:
> > > >- FLINK-28054 [6]
> > > >- FLINK-28051 [7]
> > > >
> > > > 3. Usability improvements - something that was easily doable with
> > > > SourceFunction,
> > > >but requires deep knowledge of the new, significantly more
> complex,
> > > > Source API
> > > >to achieve:
> > > >- FLINK-28056 [8]
> > > >
> > > > In my mind, none of those are blockers for proceeding with adding the
> > > > @Deprecated
> > > > annotation:
> > > > (1) is a simple case of encapsulation, internals should not concern
> the
> > > API
> > > > users
> > > > (2) is really only relevant for "exotic" use cases. Does not mean we
> > > should
> > > > not
> > > > consider those, but since it is irrelevant for 99.9% of the users, I
> do
> > > not
> > > > think
> > > > we should get stuck here.
> > > > (3) is purely a nice to have. Formally speaking, all of the tools are
> > > > there, it is
> > > > just that due to the complexity of the new Source API some "simple"
> > > things
> > > > become
> > > > non-trivial and ideally we want to

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-07-04 Thread Xintong Song
Thanks for the input, Jing. I'd also be +1 for option 1.

Best,

Xintong



On Mon, Jul 3, 2023 at 7:20 PM Jing Ge  wrote:

> Hi Xingtong,
>
> Option 1, secure plan would be:
>
> 1. graduate kafka, File, JDBC connectors to @Public
> 2. graduate SinkV2 to @Public
> 3. remove SinkFunction.
>
> Option 2, risky plan but at a fast pace:
>
> 1. graduate SinkV2 to @Public and expecting more maintenance effort since
> there are many known and unsolved issues.
> 2. remove SinkFunction.
> 3. It depends on the connectors' contributors whether connectors can
> upgrade to Flink 2.0, since we moved forward with SinkV2 API without taking
> care of implementations in external connectors.
>
> I am ok with both of them and personally prefer option 1.
>
> Best regards,
> Jing
>
>
> On Fri, Jun 30, 2023 at 3:41 AM Xintong Song 
> wrote:
>
> > I see. Thanks for the explanation. I may have not looked into this deeply
> > enough, and would trust the decision from you and the community members
> who
> > participated in the discussion & vote.
> >
> > Best,
> >
> > Xintong
> >
> >
> >
> > On Thu, Jun 29, 2023 at 10:28 PM Alexander Fedulov <
> > alexander.fedu...@gmail.com> wrote:
> >
> > > > However, I'm not sure about 2.
> > >
> > > I am not aware of a bylaw that states the specific requirements in
> order
> > to
> > > mark something as @Deprecated. My understanding from the discussion and
> > the
> > > vote was that the community recognizes the necessity to make it
> explicit
> > > that
> > > the usage of the SourceFunction API is discouraged. This can actually
> > > stimulate
> > > authors of connectors that rely on this very specific and non-baseline
> > > functionality to contribute extensions to the new Source API themselves
> > in
> > > order to
> > > close the gap. ExternallyInducedSource, for instance, was driven by
> > Pravega
> > > to
> > > begin with, since it was only needed for their purposes [1]. We are not
> > > removing
> > > anything - until 2.0 everything will continue to work and we can work
> on
> > > resolving the limitations until then, I personally don't see a big
> issue
> > > here.
> > >
> > > >Do you think it is feasible to resolve them by the feature freeze date
> > of
> > > 1.18?
> > > No, these are rather complex additions that would probably require
> > FLIP(s).
> > >
> > > [1]
> > >
> > >
> >
> https://flink.apache.org/2022/01/20/pravega-flink-connector-101/#checkpoint-integration
> > >
> > > On Thu, 29 Jun 2023 at 14:25, Xintong Song 
> > wrote:
> > >
> > > > Thanks for the explanation, Alex.
> > > >
> > > > Not blocking the deprecation on 1 & 3 makes sense to me. However, I'm
> > not
> > > > sure about 2.
> > > >
> > > > It sounds to me that, without FLINK-28051 & FLINK-28054, some of the
> > > > connectors cannot migrate to the new Source API, or at least further
> > > > investigation is needed to understand the situation. If this is the
> > case,
> > > > we probably should not deprecate the API until these issues are
> > resolved.
> > > > Do you think it is feasible to resolve them by the feature freeze
> date
> > of
> > > > 1.18?
> > > >
> > > > Best,
> > > >
> > > > Xintong
> > > >
> > > >
> > > >
> > > > On Thu, Jun 29, 2023 at 8:02 PM Alexander Fedulov <
> > > > alexander.fedu...@gmail.com> wrote:
> > > >
> > > > > @Xintong
> > > > > The original discussion [1] and vote [2] converged on the idea that
> > it
> > > is
> > > > > better
> > > > > to make it clear to the users that they should stop using
> > > SourceFunction
> > > > > since it
> > > > > is going away. The longer we do not have this indication, the more
> > user
> > > > > implementations will be based on it and the more pain will be
> induced
> > > > when
> > > > > we
> > > > > finally drop it. Users now have an alternative API that they should
> > use
> > > > and
> > > > > which
> > > > > is fully functional, from that perspective nothing blocks marking
> it
> > > > > @Deprecated.
> > > > > As for the remaining work items - there are primarily three kinds:
> > > > >
> > > > > 1. Where Flink internally uses SourceFunction, without exposing
> this
> > > fact
> > > > > to the
> > > > >outside world:
> > > > >- FLINK-28050 [3]
> > > > >- FLINK-28229 [4]
> > > > >- FLINK-28048 [5]
> > > > >
> > > > > 2. Very specific edge cases that might not be covered by the Source
> > API
> > > > as
> > > > > is:
> > > > >- FLINK-28054 [6]
> > > > >- FLINK-28051 [7]
> > > > >
> > > > > 3. Usability improvements - something that was easily doable with
> > > > > SourceFunction,
> > > > >but requires deep knowledge of the new, significantly more
> > complex,
> > > > > Source API
> > > > >to achieve:
> > > > >- FLINK-28056 [8]
> > > > >
> > > > > In my mind, none of those are blockers for proceeding with adding
> the
> > > > > @Deprecated
> > > > > annotation:
> > > > > (1) is a simple case of encapsulation, internals should not concern
> > the
> > > > API
> > > > > users
> > > > > (2) is really only relevant for "exo

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-07-04 Thread Xintong Song
Thanks all for the discussion.

It seems to me there's a consensus on marking the following as deprecated
in 1.18:
- DataSet API
- SourceFunction
- Queryable State
- All Scala APIs

More time is needed for deprecating SinkFunction.

I'll leave this discussion open for a few more days. And if there's no
objections, I'll create JIRA tickets accordingly.

Best,

Xintong



On Wed, Jul 5, 2023 at 1:34 PM Xintong Song  wrote:

> Thanks for the input, Jing. I'd also be +1 for option 1.
>
> Best,
>
> Xintong
>
>
>
> On Mon, Jul 3, 2023 at 7:20 PM Jing Ge  wrote:
>
>> Hi Xingtong,
>>
>> Option 1, secure plan would be:
>>
>> 1. graduate kafka, File, JDBC connectors to @Public
>> 2. graduate SinkV2 to @Public
>> 3. remove SinkFunction.
>>
>> Option 2, risky plan but at a fast pace:
>>
>> 1. graduate SinkV2 to @Public and expecting more maintenance effort since
>> there are many known and unsolved issues.
>> 2. remove SinkFunction.
>> 3. It depends on the connectors' contributors whether connectors can
>> upgrade to Flink 2.0, since we moved forward with SinkV2 API without
>> taking
>> care of implementations in external connectors.
>>
>> I am ok with both of them and personally prefer option 1.
>>
>> Best regards,
>> Jing
>>
>>
>> On Fri, Jun 30, 2023 at 3:41 AM Xintong Song 
>> wrote:
>>
>> > I see. Thanks for the explanation. I may have not looked into this
>> deeply
>> > enough, and would trust the decision from you and the community members
>> who
>> > participated in the discussion & vote.
>> >
>> > Best,
>> >
>> > Xintong
>> >
>> >
>> >
>> > On Thu, Jun 29, 2023 at 10:28 PM Alexander Fedulov <
>> > alexander.fedu...@gmail.com> wrote:
>> >
>> > > > However, I'm not sure about 2.
>> > >
>> > > I am not aware of a bylaw that states the specific requirements in
>> order
>> > to
>> > > mark something as @Deprecated. My understanding from the discussion
>> and
>> > the
>> > > vote was that the community recognizes the necessity to make it
>> explicit
>> > > that
>> > > the usage of the SourceFunction API is discouraged. This can actually
>> > > stimulate
>> > > authors of connectors that rely on this very specific and non-baseline
>> > > functionality to contribute extensions to the new Source API
>> themselves
>> > in
>> > > order to
>> > > close the gap. ExternallyInducedSource, for instance, was driven by
>> > Pravega
>> > > to
>> > > begin with, since it was only needed for their purposes [1]. We are
>> not
>> > > removing
>> > > anything - until 2.0 everything will continue to work and we can work
>> on
>> > > resolving the limitations until then, I personally don't see a big
>> issue
>> > > here.
>> > >
>> > > >Do you think it is feasible to resolve them by the feature freeze
>> date
>> > of
>> > > 1.18?
>> > > No, these are rather complex additions that would probably require
>> > FLIP(s).
>> > >
>> > > [1]
>> > >
>> > >
>> >
>> https://flink.apache.org/2022/01/20/pravega-flink-connector-101/#checkpoint-integration
>> > >
>> > > On Thu, 29 Jun 2023 at 14:25, Xintong Song 
>> > wrote:
>> > >
>> > > > Thanks for the explanation, Alex.
>> > > >
>> > > > Not blocking the deprecation on 1 & 3 makes sense to me. However,
>> I'm
>> > not
>> > > > sure about 2.
>> > > >
>> > > > It sounds to me that, without FLINK-28051 & FLINK-28054, some of the
>> > > > connectors cannot migrate to the new Source API, or at least further
>> > > > investigation is needed to understand the situation. If this is the
>> > case,
>> > > > we probably should not deprecate the API until these issues are
>> > resolved.
>> > > > Do you think it is feasible to resolve them by the feature freeze
>> date
>> > of
>> > > > 1.18?
>> > > >
>> > > > Best,
>> > > >
>> > > > Xintong
>> > > >
>> > > >
>> > > >
>> > > > On Thu, Jun 29, 2023 at 8:02 PM Alexander Fedulov <
>> > > > alexander.fedu...@gmail.com> wrote:
>> > > >
>> > > > > @Xintong
>> > > > > The original discussion [1] and vote [2] converged on the idea
>> that
>> > it
>> > > is
>> > > > > better
>> > > > > to make it clear to the users that they should stop using
>> > > SourceFunction
>> > > > > since it
>> > > > > is going away. The longer we do not have this indication, the more
>> > user
>> > > > > implementations will be based on it and the more pain will be
>> induced
>> > > > when
>> > > > > we
>> > > > > finally drop it. Users now have an alternative API that they
>> should
>> > use
>> > > > and
>> > > > > which
>> > > > > is fully functional, from that perspective nothing blocks marking
>> it
>> > > > > @Deprecated.
>> > > > > As for the remaining work items - there are primarily three kinds:
>> > > > >
>> > > > > 1. Where Flink internally uses SourceFunction, without exposing
>> this
>> > > fact
>> > > > > to the
>> > > > >outside world:
>> > > > >- FLINK-28050 [3]
>> > > > >- FLINK-28229 [4]
>> > > > >- FLINK-28048 [5]
>> > > > >
>> > > > > 2. Very specific edge cases that might not be covered by the
>> Source
>> > API
>> > > > as
>> > > > > is:
>> > > > >- FL

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-07-05 Thread Chesnay Schepler

There's a whole bunch of metric APIs that would need to be deprecated.
That is of course if the metric FLIPs are being accepted.

Which makes me wonder if we aren't doing things the wrong way around; 
shouldn't the decision to deprecate an API be part of the FLIP discussion?


On 05/07/2023 07:39, Xintong Song wrote:

Thanks all for the discussion.

It seems to me there's a consensus on marking the following as deprecated
in 1.18:
- DataSet API
- SourceFunction
- Queryable State
- All Scala APIs

More time is needed for deprecating SinkFunction.

I'll leave this discussion open for a few more days. And if there's no
objections, I'll create JIRA tickets accordingly.

Best,

Xintong



On Wed, Jul 5, 2023 at 1:34 PM Xintong Song  wrote:


Thanks for the input, Jing. I'd also be +1 for option 1.

Best,

Xintong



On Mon, Jul 3, 2023 at 7:20 PM Jing Ge  wrote:


Hi Xingtong,

Option 1, secure plan would be:

1. graduate kafka, File, JDBC connectors to @Public
2. graduate SinkV2 to @Public
3. remove SinkFunction.

Option 2, risky plan but at a fast pace:

1. graduate SinkV2 to @Public and expecting more maintenance effort since
there are many known and unsolved issues.
2. remove SinkFunction.
3. It depends on the connectors' contributors whether connectors can
upgrade to Flink 2.0, since we moved forward with SinkV2 API without
taking
care of implementations in external connectors.

I am ok with both of them and personally prefer option 1.

Best regards,
Jing


On Fri, Jun 30, 2023 at 3:41 AM Xintong Song 
wrote:


I see. Thanks for the explanation. I may have not looked into this

deeply

enough, and would trust the decision from you and the community members

who

participated in the discussion & vote.

Best,

Xintong



On Thu, Jun 29, 2023 at 10:28 PM Alexander Fedulov <
alexander.fedu...@gmail.com> wrote:


However, I'm not sure about 2.

I am not aware of a bylaw that states the specific requirements in

order

to

mark something as @Deprecated. My understanding from the discussion

and

the

vote was that the community recognizes the necessity to make it

explicit

that
the usage of the SourceFunction API is discouraged. This can actually
stimulate
authors of connectors that rely on this very specific and non-baseline
functionality to contribute extensions to the new Source API

themselves

in

order to
close the gap. ExternallyInducedSource, for instance, was driven by

Pravega

to
begin with, since it was only needed for their purposes [1]. We are

not

removing
anything - until 2.0 everything will continue to work and we can work

on

resolving the limitations until then, I personally don't see a big

issue

here.


Do you think it is feasible to resolve them by the feature freeze

date

of

1.18?
No, these are rather complex additions that would probably require

FLIP(s).

[1]



https://flink.apache.org/2022/01/20/pravega-flink-connector-101/#checkpoint-integration

On Thu, 29 Jun 2023 at 14:25, Xintong Song 

wrote:

Thanks for the explanation, Alex.

Not blocking the deprecation on 1 & 3 makes sense to me. However,

I'm

not

sure about 2.

It sounds to me that, without FLINK-28051 & FLINK-28054, some of the
connectors cannot migrate to the new Source API, or at least further
investigation is needed to understand the situation. If this is the

case,

we probably should not deprecate the API until these issues are

resolved.

Do you think it is feasible to resolve them by the feature freeze

date

of

1.18?

Best,

Xintong



On Thu, Jun 29, 2023 at 8:02 PM Alexander Fedulov <
alexander.fedu...@gmail.com> wrote:


@Xintong
The original discussion [1] and vote [2] converged on the idea

that

it

is

better
to make it clear to the users that they should stop using

SourceFunction

since it
is going away. The longer we do not have this indication, the more

user

implementations will be based on it and the more pain will be

induced

when

we
finally drop it. Users now have an alternative API that they

should

use

and

which
is fully functional, from that perspective nothing blocks marking

it

@Deprecated.
As for the remaining work items - there are primarily three kinds:

1. Where Flink internally uses SourceFunction, without exposing

this

fact

to the
outside world:
- FLINK-28050 [3]
- FLINK-28229 [4]
- FLINK-28048 [5]

2. Very specific edge cases that might not be covered by the

Source

API

as

is:
- FLINK-28054 [6]
- FLINK-28051 [7]

3. Usability improvements - something that was easily doable with
SourceFunction,
but requires deep knowledge of the new, significantly more

complex,

Source API
to achieve:
- FLINK-28056 [8]

In my mind, none of those are blockers for proceeding with adding

the

@Deprecated
annotation:
(1) is a simple case of encapsulation, internals should not

concern

the

API

users
(2) is really only relevant for "exotic" use cases. Does not mean

we

should

not
consider those, but since it is irrelevant for 99.9% of the

users, I

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-07-05 Thread Alexander Fedulov
@Jing
Just to clarify, when you say:

> 3. remove SinkFunction.
Which steps do you imply for the 1.18 release and for the 2.0 release?

@Xintong
A side note - with the new Source API we lose the ability to control
checkpointing from the source since there is no lock anymore. This
functionality
is currently used in a variety of tests for the Sinks - the tests that rely
on tight
synchronization between specific elements passed from the source  to the
sink before
allowing a checkpoint to complete (see FiniteTestSource [1]). Since FLIP-27
Sources rely
on decoupling via the mailbox, without exposing the lock, it is not
immediately clear
if it is possible to achieve the same functionality without major
extensions in the
runtime for such testing purposes. My hope initially was that only the
legacy Sinks
relied on this - this would have made it possible to drop
SourceFunction+SinkFunction
together, but, in fact, it also already became part of the new SinkV2
testing IT suits
[2]. Moreover, I know of at least one major connector that also relies on
it for
verifying committed sink metadata for a specific set of records (Iceberg)
[3]. In my
estimation this currently presents a major blocker for the SourceFunction
removal.

[1]
https://github.com/apache/flink/blob/master/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/FiniteTestSource.java
[2]
https://github.com/apache/flink/blob/master/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/StreamingExecutionFileSinkITCase.java#L132
[3]
https://github.com/apache/iceberg/blob/master/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java#L75C1-L85C2

Best,
Alex

On Wed, 5 Jul 2023 at 10:47, Chesnay Schepler  wrote:

> There's a whole bunch of metric APIs that would need to be deprecated.
> That is of course if the metric FLIPs are being accepted.
>
> Which makes me wonder if we aren't doing things the wrong way around;
> shouldn't the decision to deprecate an API be part of the FLIP discussion?
>
> On 05/07/2023 07:39, Xintong Song wrote:
> > Thanks all for the discussion.
> >
> > It seems to me there's a consensus on marking the following as deprecated
> > in 1.18:
> > - DataSet API
> > - SourceFunction
> > - Queryable State
> > - All Scala APIs
> >
> > More time is needed for deprecating SinkFunction.
> >
> > I'll leave this discussion open for a few more days. And if there's no
> > objections, I'll create JIRA tickets accordingly.
> >
> > Best,
> >
> > Xintong
> >
> >
> >
> > On Wed, Jul 5, 2023 at 1:34 PM Xintong Song 
> wrote:
> >
> >> Thanks for the input, Jing. I'd also be +1 for option 1.
> >>
> >> Best,
> >>
> >> Xintong
> >>
> >>
> >>
> >> On Mon, Jul 3, 2023 at 7:20 PM Jing Ge 
> wrote:
> >>
> >>> Hi Xingtong,
> >>>
> >>> Option 1, secure plan would be:
> >>>
> >>> 1. graduate kafka, File, JDBC connectors to @Public
> >>> 2. graduate SinkV2 to @Public
> >>> 3. remove SinkFunction.
> >>>
> >>> Option 2, risky plan but at a fast pace:
> >>>
> >>> 1. graduate SinkV2 to @Public and expecting more maintenance effort
> since
> >>> there are many known and unsolved issues.
> >>> 2. remove SinkFunction.
> >>> 3. It depends on the connectors' contributors whether connectors can
> >>> upgrade to Flink 2.0, since we moved forward with SinkV2 API without
> >>> taking
> >>> care of implementations in external connectors.
> >>>
> >>> I am ok with both of them and personally prefer option 1.
> >>>
> >>> Best regards,
> >>> Jing
> >>>
> >>>
> >>> On Fri, Jun 30, 2023 at 3:41 AM Xintong Song 
> >>> wrote:
> >>>
>  I see. Thanks for the explanation. I may have not looked into this
> >>> deeply
>  enough, and would trust the decision from you and the community
> members
> >>> who
>  participated in the discussion & vote.
> 
>  Best,
> 
>  Xintong
> 
> 
> 
>  On Thu, Jun 29, 2023 at 10:28 PM Alexander Fedulov <
>  alexander.fedu...@gmail.com> wrote:
> 
> >> However, I'm not sure about 2.
> > I am not aware of a bylaw that states the specific requirements in
> >>> order
>  to
> > mark something as @Deprecated. My understanding from the discussion
> >>> and
>  the
> > vote was that the community recognizes the necessity to make it
> >>> explicit
> > that
> > the usage of the SourceFunction API is discouraged. This can actually
> > stimulate
> > authors of connectors that rely on this very specific and
> non-baseline
> > functionality to contribute extensions to the new Source API
> >>> themselves
>  in
> > order to
> > close the gap. ExternallyInducedSource, for instance, was driven by
>  Pravega
> > to
> > begin with, since it was only needed for their purposes [1]. We are
> >>> not
> > removing
> > anything - until 2.0 everything will continue to work and we can work
> >>> on
> > resolving the limitations until then, I personally don't see a big

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-07-05 Thread Xintong Song
@Chesnay,

shouldn't the decision to deprecate an API be part of the FLIP discussion?
>

Exactly. I agree that deprecation of an old API should be part of the FLIP
where the new API is introduced. And I appreciate that many APIs that are
listed to be removed in release 2.0 are already deprecated when its
replacement is ready. E.g., SinkV1 is deprecated when SinkV2 is ready
(FLIP-191), TableSource / TableSink is deprecated when the new table source
& sink is ready (FLIP-95).

In this thread, I only brought up APIs which possibly should have been
deprecated already, i.e., APIs which already (or won't) have replacements.
Ideally, they should be deprecated when the corresponding FLIP is
completed, but are somehow not deprecated yet.
- DataSet, which should be subsumed by DataStream & Table according to
FLIP-131
- SourceFunction / SinkFunction, which seem to be replaced by the new
Source (FLIP-27) and SinkV2 (FLIP-191)
- Queryable State, which seem to be replaced by the State Processor API
(FLIP-43)
- All scala APIs, the decision is made in FLIP-265, and this is more a
implementation issue that we are expected to but haven't really deprecate
all user-facing scala APIs

@Alex,
I may not have understood all the details, but based on what you described
I'd hesitate to block the deprecation / removal of SourceFunction on this.
- Typically, we should not block production code changes on testing
requirements.
- IIUC, the testing scenario you described is like blocking the source for
proceeding (emit data, finish, etc.) until a checkpoint is finished. I
believe there are other ways to achieve this. E.g., block the source on a
CompletableFuture, and completes the future when the completion of
checkpoint is detected, via REST API or scanning the checkpoint directory.

Best,

Xintong



On Wed, Jul 5, 2023 at 7:08 PM Alexander Fedulov <
alexander.fedu...@gmail.com> wrote:

> @Jing
> Just to clarify, when you say:
>
> > 3. remove SinkFunction.
> Which steps do you imply for the 1.18 release and for the 2.0 release?
>
> @Xintong
> A side note - with the new Source API we lose the ability to control
> checkpointing from the source since there is no lock anymore. This
> functionality
> is currently used in a variety of tests for the Sinks - the tests that rely
> on tight
> synchronization between specific elements passed from the source  to the
> sink before
> allowing a checkpoint to complete (see FiniteTestSource [1]). Since FLIP-27
> Sources rely
> on decoupling via the mailbox, without exposing the lock, it is not
> immediately clear
> if it is possible to achieve the same functionality without major
> extensions in the
> runtime for such testing purposes. My hope initially was that only the
> legacy Sinks
> relied on this - this would have made it possible to drop
> SourceFunction+SinkFunction
> together, but, in fact, it also already became part of the new SinkV2
> testing IT suits
> [2]. Moreover, I know of at least one major connector that also relies on
> it for
> verifying committed sink metadata for a specific set of records (Iceberg)
> [3]. In my
> estimation this currently presents a major blocker for the SourceFunction
> removal.
>
> [1]
>
> https://github.com/apache/flink/blob/master/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/FiniteTestSource.java
> [2]
>
> https://github.com/apache/flink/blob/master/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/StreamingExecutionFileSinkITCase.java#L132
> [3]
>
> https://github.com/apache/iceberg/blob/master/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java#L75C1-L85C2
>
> Best,
> Alex
>
> On Wed, 5 Jul 2023 at 10:47, Chesnay Schepler  wrote:
>
> > There's a whole bunch of metric APIs that would need to be deprecated.
> > That is of course if the metric FLIPs are being accepted.
> >
> > Which makes me wonder if we aren't doing things the wrong way around;
> > shouldn't the decision to deprecate an API be part of the FLIP
> discussion?
> >
> > On 05/07/2023 07:39, Xintong Song wrote:
> > > Thanks all for the discussion.
> > >
> > > It seems to me there's a consensus on marking the following as
> deprecated
> > > in 1.18:
> > > - DataSet API
> > > - SourceFunction
> > > - Queryable State
> > > - All Scala APIs
> > >
> > > More time is needed for deprecating SinkFunction.
> > >
> > > I'll leave this discussion open for a few more days. And if there's no
> > > objections, I'll create JIRA tickets accordingly.
> > >
> > > Best,
> > >
> > > Xintong
> > >
> > >
> > >
> > > On Wed, Jul 5, 2023 at 1:34 PM Xintong Song 
> > wrote:
> > >
> > >> Thanks for the input, Jing. I'd also be +1 for option 1.
> > >>
> > >> Best,
> > >>
> > >> Xintong
> > >>
> > >>
> > >>
> > >> On Mon, Jul 3, 2023 at 7:20 PM Jing Ge 
> > wrote:
> > >>
> > >>> Hi Xingtong,
> > >>>
> > >>> Option 1, secure plan would be:
> > >>>
> > >>> 1. graduate kafka, File, JDBC connectors to @Public
> >

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-07-05 Thread Jing Ge
Hi Alex,


> > 3. remove SinkFunction.
> Which steps do you imply for the 1.18 release and for the 2.0 release?
>

for 2.0 release. 1.18 will be released soon.

Best regards,
Jing


On Wed, Jul 5, 2023 at 1:08 PM Alexander Fedulov <
alexander.fedu...@gmail.com> wrote:

> @Jing
> Just to clarify, when you say:
>
> 3. remove SinkFunction.
> Which steps do you imply for the 1.18 release and for the 2.0 release?

@Xintong
> A side note - with the new Source API we lose the ability to control
> checkpointing from the source since there is no lock anymore. This
> functionality
> is currently used in a variety of tests for the Sinks - the tests that rely
> on tight
> synchronization between specific elements passed from the source  to the
> sink before
> allowing a checkpoint to complete (see FiniteTestSource [1]). Since FLIP-27
> Sources rely
> on decoupling via the mailbox, without exposing the lock, it is not
> immediately clear
> if it is possible to achieve the same functionality without major
> extensions in the
> runtime for such testing purposes. My hope initially was that only the
> legacy Sinks
> relied on this - this would have made it possible to drop
> SourceFunction+SinkFunction
> together, but, in fact, it also already became part of the new SinkV2
> testing IT suits
> [2]. Moreover, I know of at least one major connector that also relies on
> it for
> verifying committed sink metadata for a specific set of records (Iceberg)
> [3]. In my
> estimation this currently presents a major blocker for the SourceFunction
> removal.
>
> [1]
>
> https://github.com/apache/flink/blob/master/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/FiniteTestSource.java
> [2]
>
> https://github.com/apache/flink/blob/master/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/StreamingExecutionFileSinkITCase.java#L132
> [3]
>
> https://github.com/apache/iceberg/blob/master/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java#L75C1-L85C2
>
> Best,
> Alex
>
> On Wed, 5 Jul 2023 at 10:47, Chesnay Schepler  wrote:
>
> > There's a whole bunch of metric APIs that would need to be deprecated.
> > That is of course if the metric FLIPs are being accepted.
> >
> > Which makes me wonder if we aren't doing things the wrong way around;
> > shouldn't the decision to deprecate an API be part of the FLIP
> discussion?
> >
> > On 05/07/2023 07:39, Xintong Song wrote:
> > > Thanks all for the discussion.
> > >
> > > It seems to me there's a consensus on marking the following as
> deprecated
> > > in 1.18:
> > > - DataSet API
> > > - SourceFunction
> > > - Queryable State
> > > - All Scala APIs
> > >
> > > More time is needed for deprecating SinkFunction.
> > >
> > > I'll leave this discussion open for a few more days. And if there's no
> > > objections, I'll create JIRA tickets accordingly.
> > >
> > > Best,
> > >
> > > Xintong
> > >
> > >
> > >
> > > On Wed, Jul 5, 2023 at 1:34 PM Xintong Song 
> > wrote:
> > >
> > >> Thanks for the input, Jing. I'd also be +1 for option 1.
> > >>
> > >> Best,
> > >>
> > >> Xintong
> > >>
> > >>
> > >>
> > >> On Mon, Jul 3, 2023 at 7:20 PM Jing Ge 
> > wrote:
> > >>
> > >>> Hi Xingtong,
> > >>>
> > >>> Option 1, secure plan would be:
> > >>>
> > >>> 1. graduate kafka, File, JDBC connectors to @Public
> > >>> 2. graduate SinkV2 to @Public
> > >>> 3. remove SinkFunction.
> > >>>
> > >>> Option 2, risky plan but at a fast pace:
> > >>>
> > >>> 1. graduate SinkV2 to @Public and expecting more maintenance effort
> > since
> > >>> there are many known and unsolved issues.
> > >>> 2. remove SinkFunction.
> > >>> 3. It depends on the connectors' contributors whether connectors can
> > >>> upgrade to Flink 2.0, since we moved forward with SinkV2 API without
> > >>> taking
> > >>> care of implementations in external connectors.
> > >>>
> > >>> I am ok with both of them and personally prefer option 1.
> > >>>
> > >>> Best regards,
> > >>> Jing
> > >>>
> > >>>
> > >>> On Fri, Jun 30, 2023 at 3:41 AM Xintong Song 
> > >>> wrote:
> > >>>
> >  I see. Thanks for the explanation. I may have not looked into this
> > >>> deeply
> >  enough, and would trust the decision from you and the community
> > members
> > >>> who
> >  participated in the discussion & vote.
> > 
> >  Best,
> > 
> >  Xintong
> > 
> > 
> > 
> >  On Thu, Jun 29, 2023 at 10:28 PM Alexander Fedulov <
> >  alexander.fedu...@gmail.com> wrote:
> > 
> > >> However, I'm not sure about 2.
> > > I am not aware of a bylaw that states the specific requirements in
> > >>> order
> >  to
> > > mark something as @Deprecated. My understanding from the discussion
> > >>> and
> >  the
> > > vote was that the community recognizes the necessity to make it
> > >>> explicit
> > > that
> > > the usage of the SourceFunction API is discouraged. This can
> actually
> > > stimulate
> > >>

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-07-07 Thread Xintong Song
Thanks all for the discussion. I've created FLINK-32557 for this.

Best,

Xintong



On Thu, Jul 6, 2023 at 1:00 AM Jing Ge  wrote:

> Hi Alex,
>
>
> > > 3. remove SinkFunction.
> > Which steps do you imply for the 1.18 release and for the 2.0 release?
> >
>
> for 2.0 release. 1.18 will be released soon.
>
> Best regards,
> Jing
>
>
> On Wed, Jul 5, 2023 at 1:08 PM Alexander Fedulov <
> alexander.fedu...@gmail.com> wrote:
>
> > @Jing
> > Just to clarify, when you say:
> >
> > 3. remove SinkFunction.
> > Which steps do you imply for the 1.18 release and for the 2.0 release?
>
> @Xintong
> > A side note - with the new Source API we lose the ability to control
> > checkpointing from the source since there is no lock anymore. This
> > functionality
> > is currently used in a variety of tests for the Sinks - the tests that
> rely
> > on tight
> > synchronization between specific elements passed from the source  to the
> > sink before
> > allowing a checkpoint to complete (see FiniteTestSource [1]). Since
> FLIP-27
> > Sources rely
> > on decoupling via the mailbox, without exposing the lock, it is not
> > immediately clear
> > if it is possible to achieve the same functionality without major
> > extensions in the
> > runtime for such testing purposes. My hope initially was that only the
> > legacy Sinks
> > relied on this - this would have made it possible to drop
> > SourceFunction+SinkFunction
> > together, but, in fact, it also already became part of the new SinkV2
> > testing IT suits
> > [2]. Moreover, I know of at least one major connector that also relies on
> > it for
> > verifying committed sink metadata for a specific set of records (Iceberg)
> > [3]. In my
> > estimation this currently presents a major blocker for the SourceFunction
> > removal.
> >
> > [1]
> >
> >
> https://github.com/apache/flink/blob/master/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/FiniteTestSource.java
> > [2]
> >
> >
> https://github.com/apache/flink/blob/master/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/StreamingExecutionFileSinkITCase.java#L132
> > [3]
> >
> >
> https://github.com/apache/iceberg/blob/master/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java#L75C1-L85C2
> >
> > Best,
> > Alex
> >
> > On Wed, 5 Jul 2023 at 10:47, Chesnay Schepler 
> wrote:
> >
> > > There's a whole bunch of metric APIs that would need to be deprecated.
> > > That is of course if the metric FLIPs are being accepted.
> > >
> > > Which makes me wonder if we aren't doing things the wrong way around;
> > > shouldn't the decision to deprecate an API be part of the FLIP
> > discussion?
> > >
> > > On 05/07/2023 07:39, Xintong Song wrote:
> > > > Thanks all for the discussion.
> > > >
> > > > It seems to me there's a consensus on marking the following as
> > deprecated
> > > > in 1.18:
> > > > - DataSet API
> > > > - SourceFunction
> > > > - Queryable State
> > > > - All Scala APIs
> > > >
> > > > More time is needed for deprecating SinkFunction.
> > > >
> > > > I'll leave this discussion open for a few more days. And if there's
> no
> > > > objections, I'll create JIRA tickets accordingly.
> > > >
> > > > Best,
> > > >
> > > > Xintong
> > > >
> > > >
> > > >
> > > > On Wed, Jul 5, 2023 at 1:34 PM Xintong Song 
> > > wrote:
> > > >
> > > >> Thanks for the input, Jing. I'd also be +1 for option 1.
> > > >>
> > > >> Best,
> > > >>
> > > >> Xintong
> > > >>
> > > >>
> > > >>
> > > >> On Mon, Jul 3, 2023 at 7:20 PM Jing Ge 
> > > wrote:
> > > >>
> > > >>> Hi Xingtong,
> > > >>>
> > > >>> Option 1, secure plan would be:
> > > >>>
> > > >>> 1. graduate kafka, File, JDBC connectors to @Public
> > > >>> 2. graduate SinkV2 to @Public
> > > >>> 3. remove SinkFunction.
> > > >>>
> > > >>> Option 2, risky plan but at a fast pace:
> > > >>>
> > > >>> 1. graduate SinkV2 to @Public and expecting more maintenance effort
> > > since
> > > >>> there are many known and unsolved issues.
> > > >>> 2. remove SinkFunction.
> > > >>> 3. It depends on the connectors' contributors whether connectors
> can
> > > >>> upgrade to Flink 2.0, since we moved forward with SinkV2 API
> without
> > > >>> taking
> > > >>> care of implementations in external connectors.
> > > >>>
> > > >>> I am ok with both of them and personally prefer option 1.
> > > >>>
> > > >>> Best regards,
> > > >>> Jing
> > > >>>
> > > >>>
> > > >>> On Fri, Jun 30, 2023 at 3:41 AM Xintong Song <
> tonysong...@gmail.com>
> > > >>> wrote:
> > > >>>
> > >  I see. Thanks for the explanation. I may have not looked into this
> > > >>> deeply
> > >  enough, and would trust the decision from you and the community
> > > members
> > > >>> who
> > >  participated in the discussion & vote.
> > > 
> > >  Best,
> > > 
> > >  Xintong
> > > 
> > > 
> > > 
> > >  On Thu, Jun 29, 2023 at 10:28 PM Alexander Fedulov <
> > >  alexander.fedu...@gmail.com> wrote:
> 

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-07-07 Thread Alexander Fedulov
@Xintong
> - IIUC, the testing scenario you described is like blocking the source for
> proceeding (emit data, finish, etc.) until a checkpoint is finished.

It is more tricky than that - we need to prevent the Sink from receiving a
checkpoint barrier until the Source is done emitting a given set of
records. In
the current tests, which are also used for V2 Sinks, SourceFunction controls
when the Sink is "allowed" to commit by holding the checkpoint lock while
producing the records. The lock is not available in the new Source by design
and we need a solution that provides the same functionality (without
modifying
the Sinks). I am currently checking if a workaround is at all possible
without
adjusting anything in the Source interface.

> I may not have understood all the details, but based on what you described
> I'd hesitate to block the deprecation / removal of SourceFunction on this.

I don't think we should, just wanted to highlight that there are some
unknowns
with respect to estimating the amount of work required.

@Jing
I want to understand in which release would you target graduation of the
mentioned connectors to @Public/@PublicEvolving - basically the anticipated
timeline of the steps in both options with respect to releases.

Best,
Alex

On Fri, 7 Jul 2023 at 10:53, Xintong Song  wrote:

> Thanks all for the discussion. I've created FLINK-32557 for this.
>
> Best,
>
> Xintong
>
>
>
> On Thu, Jul 6, 2023 at 1:00 AM Jing Ge  wrote:
>
> > Hi Alex,
> >
> >
> > > > 3. remove SinkFunction.
> > > Which steps do you imply for the 1.18 release and for the 2.0 release?
> > >
> >
> > for 2.0 release. 1.18 will be released soon.
> >
> > Best regards,
> > Jing
> >
> >
> > On Wed, Jul 5, 2023 at 1:08 PM Alexander Fedulov <
> > alexander.fedu...@gmail.com> wrote:
> >
> > > @Jing
> > > Just to clarify, when you say:
> > >
> > > 3. remove SinkFunction.
> > > Which steps do you imply for the 1.18 release and for the 2.0 release?
> >
> > @Xintong
> > > A side note - with the new Source API we lose the ability to control
> > > checkpointing from the source since there is no lock anymore. This
> > > functionality
> > > is currently used in a variety of tests for the Sinks - the tests that
> > rely
> > > on tight
> > > synchronization between specific elements passed from the source  to
> the
> > > sink before
> > > allowing a checkpoint to complete (see FiniteTestSource [1]). Since
> > FLIP-27
> > > Sources rely
> > > on decoupling via the mailbox, without exposing the lock, it is not
> > > immediately clear
> > > if it is possible to achieve the same functionality without major
> > > extensions in the
> > > runtime for such testing purposes. My hope initially was that only the
> > > legacy Sinks
> > > relied on this - this would have made it possible to drop
> > > SourceFunction+SinkFunction
> > > together, but, in fact, it also already became part of the new SinkV2
> > > testing IT suits
> > > [2]. Moreover, I know of at least one major connector that also relies
> on
> > > it for
> > > verifying committed sink metadata for a specific set of records
> (Iceberg)
> > > [3]. In my
> > > estimation this currently presents a major blocker for the
> SourceFunction
> > > removal.
> > >
> > > [1]
> > >
> > >
> >
> https://github.com/apache/flink/blob/master/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/FiniteTestSource.java
> > > [2]
> > >
> > >
> >
> https://github.com/apache/flink/blob/master/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/StreamingExecutionFileSinkITCase.java#L132
> > > [3]
> > >
> > >
> >
> https://github.com/apache/iceberg/blob/master/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java#L75C1-L85C2
> > >
> > > Best,
> > > Alex
> > >
> > > On Wed, 5 Jul 2023 at 10:47, Chesnay Schepler 
> > wrote:
> > >
> > > > There's a whole bunch of metric APIs that would need to be
> deprecated.
> > > > That is of course if the metric FLIPs are being accepted.
> > > >
> > > > Which makes me wonder if we aren't doing things the wrong way around;
> > > > shouldn't the decision to deprecate an API be part of the FLIP
> > > discussion?
> > > >
> > > > On 05/07/2023 07:39, Xintong Song wrote:
> > > > > Thanks all for the discussion.
> > > > >
> > > > > It seems to me there's a consensus on marking the following as
> > > deprecated
> > > > > in 1.18:
> > > > > - DataSet API
> > > > > - SourceFunction
> > > > > - Queryable State
> > > > > - All Scala APIs
> > > > >
> > > > > More time is needed for deprecating SinkFunction.
> > > > >
> > > > > I'll leave this discussion open for a few more days. And if there's
> > no
> > > > > objections, I'll create JIRA tickets accordingly.
> > > > >
> > > > > Best,
> > > > >
> > > > > Xintong
> > > > >
> > > > >
> > > > >
> > > > > On Wed, Jul 5, 2023 at 1:34 PM Xintong Song  >
> > > > wrote:
> > > > >
> > > > >> Thanks for the input, Jing. I'd also be +1 for option 1.
> >

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-07-07 Thread Jing Ge
Hi Alex,

I would follow FLIP-197 and try to release them asap depending on dev
resources and how difficult those issues are. The fastest timeline is the
period defined in FLIP-197 in ideal conditions.

Best regards,
Jing

On Fri, Jul 7, 2023 at 12:20 PM Alexander Fedulov <
alexander.fedu...@gmail.com> wrote:

> @Xintong
> > - IIUC, the testing scenario you described is like blocking the source
> for
> > proceeding (emit data, finish, etc.) until a checkpoint is finished.
>
> It is more tricky than that - we need to prevent the Sink from receiving a
> checkpoint barrier until the Source is done emitting a given set of
> records. In
> the current tests, which are also used for V2 Sinks, SourceFunction
> controls
> when the Sink is "allowed" to commit by holding the checkpoint lock while
> producing the records. The lock is not available in the new Source by
> design
> and we need a solution that provides the same functionality (without
> modifying
> the Sinks). I am currently checking if a workaround is at all possible
> without
> adjusting anything in the Source interface.
>
> > I may not have understood all the details, but based on what you
> described
> > I'd hesitate to block the deprecation / removal of SourceFunction on
> this.
>
> I don't think we should, just wanted to highlight that there are some
> unknowns
> with respect to estimating the amount of work required.
>
> @Jing
> I want to understand in which release would you target graduation of the
> mentioned connectors to @Public/@PublicEvolving - basically the anticipated
> timeline of the steps in both options with respect to releases.
>
> Best,
> Alex
>
> On Fri, 7 Jul 2023 at 10:53, Xintong Song  wrote:
>
> > Thanks all for the discussion. I've created FLINK-32557 for this.
> >
> > Best,
> >
> > Xintong
> >
> >
> >
> > On Thu, Jul 6, 2023 at 1:00 AM Jing Ge 
> wrote:
> >
> > > Hi Alex,
> > >
> > >
> > > > > 3. remove SinkFunction.
> > > > Which steps do you imply for the 1.18 release and for the 2.0
> release?
> > > >
> > >
> > > for 2.0 release. 1.18 will be released soon.
> > >
> > > Best regards,
> > > Jing
> > >
> > >
> > > On Wed, Jul 5, 2023 at 1:08 PM Alexander Fedulov <
> > > alexander.fedu...@gmail.com> wrote:
> > >
> > > > @Jing
> > > > Just to clarify, when you say:
> > > >
> > > > 3. remove SinkFunction.
> > > > Which steps do you imply for the 1.18 release and for the 2.0
> release?
> > >
> > > @Xintong
> > > > A side note - with the new Source API we lose the ability to control
> > > > checkpointing from the source since there is no lock anymore. This
> > > > functionality
> > > > is currently used in a variety of tests for the Sinks - the tests
> that
> > > rely
> > > > on tight
> > > > synchronization between specific elements passed from the source  to
> > the
> > > > sink before
> > > > allowing a checkpoint to complete (see FiniteTestSource [1]). Since
> > > FLIP-27
> > > > Sources rely
> > > > on decoupling via the mailbox, without exposing the lock, it is not
> > > > immediately clear
> > > > if it is possible to achieve the same functionality without major
> > > > extensions in the
> > > > runtime for such testing purposes. My hope initially was that only
> the
> > > > legacy Sinks
> > > > relied on this - this would have made it possible to drop
> > > > SourceFunction+SinkFunction
> > > > together, but, in fact, it also already became part of the new SinkV2
> > > > testing IT suits
> > > > [2]. Moreover, I know of at least one major connector that also
> relies
> > on
> > > > it for
> > > > verifying committed sink metadata for a specific set of records
> > (Iceberg)
> > > > [3]. In my
> > > > estimation this currently presents a major blocker for the
> > SourceFunction
> > > > removal.
> > > >
> > > > [1]
> > > >
> > > >
> > >
> >
> https://github.com/apache/flink/blob/master/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/FiniteTestSource.java
> > > > [2]
> > > >
> > > >
> > >
> >
> https://github.com/apache/flink/blob/master/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/StreamingExecutionFileSinkITCase.java#L132
> > > > [3]
> > > >
> > > >
> > >
> >
> https://github.com/apache/iceberg/blob/master/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java#L75C1-L85C2
> > > >
> > > > Best,
> > > > Alex
> > > >
> > > > On Wed, 5 Jul 2023 at 10:47, Chesnay Schepler 
> > > wrote:
> > > >
> > > > > There's a whole bunch of metric APIs that would need to be
> > deprecated.
> > > > > That is of course if the metric FLIPs are being accepted.
> > > > >
> > > > > Which makes me wonder if we aren't doing things the wrong way
> around;
> > > > > shouldn't the decision to deprecate an API be part of the FLIP
> > > > discussion?
> > > > >
> > > > > On 05/07/2023 07:39, Xintong Song wrote:
> > > > > > Thanks all for the discussion.
> > > > > >
> > > > > > It seems to me there's a consensus on marking the following as
> > >

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-07-10 Thread Matthias Pohl
@Xintong did you come across FLINK-3957 [1] when looking for deprecated
issues? There are a few subtasks that would require deprecations as well as
far as I can see. For some I'm wondering whether we should add them to the
release 2.0 feature list [2] in some way and (as a consequence) missed them
in FLINK-32557 [3], e.g.:
- FLINK-4675 [4]
- FLINK-14068 [5] (listed in the release 2.0 feature list [2] but not
listed in FLINK-32557 [3])
- FLINK-5126 [6]
- FLINK-13926 [7]

For some other subtasks, I'm not 100% sure whether they still apply. Others
might resolve by removing the DataSet or Scala API. But for the 4 mentioned
above we might need to deprecate API in 1.18.

Matthias

[1] https://issues.apache.org/jira/browse/FLINK-3957
[2] https://cwiki.apache.org/confluence/display/FLINK/2.0+Release
[3] https://issues.apache.org/jira/browse/FLINK-32557

[4] https://issues.apache.org/jira/browse/FLINK-4675
[5] https://issues.apache.org/jira/browse/FLINK-14068
[6] https://issues.apache.org/jira/browse/FLINK-5126
[7] https://issues.apache.org/jira/browse/FLINK-13926

On Fri, Jul 7, 2023 at 12:59 PM Jing Ge  wrote:

> Hi Alex,
>
> I would follow FLIP-197 and try to release them asap depending on dev
> resources and how difficult those issues are. The fastest timeline is the
> period defined in FLIP-197 in ideal conditions.
>
> Best regards,
> Jing
>
> On Fri, Jul 7, 2023 at 12:20 PM Alexander Fedulov <
> alexander.fedu...@gmail.com> wrote:
>
> > @Xintong
> > > - IIUC, the testing scenario you described is like blocking the source
> > for
> > > proceeding (emit data, finish, etc.) until a checkpoint is finished.
> >
> > It is more tricky than that - we need to prevent the Sink from receiving
> a
> > checkpoint barrier until the Source is done emitting a given set of
> > records. In
> > the current tests, which are also used for V2 Sinks, SourceFunction
> > controls
> > when the Sink is "allowed" to commit by holding the checkpoint lock while
> > producing the records. The lock is not available in the new Source by
> > design
> > and we need a solution that provides the same functionality (without
> > modifying
> > the Sinks). I am currently checking if a workaround is at all possible
> > without
> > adjusting anything in the Source interface.
> >
> > > I may not have understood all the details, but based on what you
> > described
> > > I'd hesitate to block the deprecation / removal of SourceFunction on
> > this.
> >
> > I don't think we should, just wanted to highlight that there are some
> > unknowns
> > with respect to estimating the amount of work required.
> >
> > @Jing
> > I want to understand in which release would you target graduation of the
> > mentioned connectors to @Public/@PublicEvolving - basically the
> anticipated
> > timeline of the steps in both options with respect to releases.
> >
> > Best,
> > Alex
> >
> > On Fri, 7 Jul 2023 at 10:53, Xintong Song  wrote:
> >
> > > Thanks all for the discussion. I've created FLINK-32557 for this.
> > >
> > > Best,
> > >
> > > Xintong
> > >
> > >
> > >
> > > On Thu, Jul 6, 2023 at 1:00 AM Jing Ge 
> > wrote:
> > >
> > > > Hi Alex,
> > > >
> > > >
> > > > > > 3. remove SinkFunction.
> > > > > Which steps do you imply for the 1.18 release and for the 2.0
> > release?
> > > > >
> > > >
> > > > for 2.0 release. 1.18 will be released soon.
> > > >
> > > > Best regards,
> > > > Jing
> > > >
> > > >
> > > > On Wed, Jul 5, 2023 at 1:08 PM Alexander Fedulov <
> > > > alexander.fedu...@gmail.com> wrote:
> > > >
> > > > > @Jing
> > > > > Just to clarify, when you say:
> > > > >
> > > > > 3. remove SinkFunction.
> > > > > Which steps do you imply for the 1.18 release and for the 2.0
> > release?
> > > >
> > > > @Xintong
> > > > > A side note - with the new Source API we lose the ability to
> control
> > > > > checkpointing from the source since there is no lock anymore. This
> > > > > functionality
> > > > > is currently used in a variety of tests for the Sinks - the tests
> > that
> > > > rely
> > > > > on tight
> > > > > synchronization between specific elements passed from the source
> to
> > > the
> > > > > sink before
> > > > > allowing a checkpoint to complete (see FiniteTestSource [1]). Since
> > > > FLIP-27
> > > > > Sources rely
> > > > > on decoupling via the mailbox, without exposing the lock, it is not
> > > > > immediately clear
> > > > > if it is possible to achieve the same functionality without major
> > > > > extensions in the
> > > > > runtime for such testing purposes. My hope initially was that only
> > the
> > > > > legacy Sinks
> > > > > relied on this - this would have made it possible to drop
> > > > > SourceFunction+SinkFunction
> > > > > together, but, in fact, it also already became part of the new
> SinkV2
> > > > > testing IT suits
> > > > > [2]. Moreover, I know of at least one major connector that also
> > relies
> > > on
> > > > > it for
> > > > > verifying committed sink metadata for a specific set of records
> > > (Iceberg)
> > > > > [3]. In m

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-07-10 Thread Xintong Song
Thanks for bringing this up, Matthias. I've replied to you in the other
thread[1].

Best,

Xintong


[1] https://lists.apache.org/thread/l3dkdypyrovd3txzodn07lgdwtwvhgk4

On Mon, Jul 10, 2023 at 8:37 PM Matthias Pohl
 wrote:

> @Xintong did you come across FLINK-3957 [1] when looking for deprecated
> issues? There are a few subtasks that would require deprecations as well as
> far as I can see. For some I'm wondering whether we should add them to the
> release 2.0 feature list [2] in some way and (as a consequence) missed them
> in FLINK-32557 [3], e.g.:
> - FLINK-4675 [4]
> - FLINK-14068 [5] (listed in the release 2.0 feature list [2] but not
> listed in FLINK-32557 [3])
> - FLINK-5126 [6]
> - FLINK-13926 [7]
>
> For some other subtasks, I'm not 100% sure whether they still apply. Others
> might resolve by removing the DataSet or Scala API. But for the 4 mentioned
> above we might need to deprecate API in 1.18.
>
> Matthias
>
> [1] https://issues.apache.org/jira/browse/FLINK-3957
> [2] https://cwiki.apache.org/confluence/display/FLINK/2.0+Release
> [3] https://issues.apache.org/jira/browse/FLINK-32557
>
> [4] https://issues.apache.org/jira/browse/FLINK-4675
> [5] https://issues.apache.org/jira/browse/FLINK-14068
> [6] https://issues.apache.org/jira/browse/FLINK-5126
> [7] https://issues.apache.org/jira/browse/FLINK-13926
>
> On Fri, Jul 7, 2023 at 12:59 PM Jing Ge 
> wrote:
>
> > Hi Alex,
> >
> > I would follow FLIP-197 and try to release them asap depending on dev
> > resources and how difficult those issues are. The fastest timeline is the
> > period defined in FLIP-197 in ideal conditions.
> >
> > Best regards,
> > Jing
> >
> > On Fri, Jul 7, 2023 at 12:20 PM Alexander Fedulov <
> > alexander.fedu...@gmail.com> wrote:
> >
> > > @Xintong
> > > > - IIUC, the testing scenario you described is like blocking the
> source
> > > for
> > > > proceeding (emit data, finish, etc.) until a checkpoint is finished.
> > >
> > > It is more tricky than that - we need to prevent the Sink from
> receiving
> > a
> > > checkpoint barrier until the Source is done emitting a given set of
> > > records. In
> > > the current tests, which are also used for V2 Sinks, SourceFunction
> > > controls
> > > when the Sink is "allowed" to commit by holding the checkpoint lock
> while
> > > producing the records. The lock is not available in the new Source by
> > > design
> > > and we need a solution that provides the same functionality (without
> > > modifying
> > > the Sinks). I am currently checking if a workaround is at all possible
> > > without
> > > adjusting anything in the Source interface.
> > >
> > > > I may not have understood all the details, but based on what you
> > > described
> > > > I'd hesitate to block the deprecation / removal of SourceFunction on
> > > this.
> > >
> > > I don't think we should, just wanted to highlight that there are some
> > > unknowns
> > > with respect to estimating the amount of work required.
> > >
> > > @Jing
> > > I want to understand in which release would you target graduation of
> the
> > > mentioned connectors to @Public/@PublicEvolving - basically the
> > anticipated
> > > timeline of the steps in both options with respect to releases.
> > >
> > > Best,
> > > Alex
> > >
> > > On Fri, 7 Jul 2023 at 10:53, Xintong Song 
> wrote:
> > >
> > > > Thanks all for the discussion. I've created FLINK-32557 for this.
> > > >
> > > > Best,
> > > >
> > > > Xintong
> > > >
> > > >
> > > >
> > > > On Thu, Jul 6, 2023 at 1:00 AM Jing Ge 
> > > wrote:
> > > >
> > > > > Hi Alex,
> > > > >
> > > > >
> > > > > > > 3. remove SinkFunction.
> > > > > > Which steps do you imply for the 1.18 release and for the 2.0
> > > release?
> > > > > >
> > > > >
> > > > > for 2.0 release. 1.18 will be released soon.
> > > > >
> > > > > Best regards,
> > > > > Jing
> > > > >
> > > > >
> > > > > On Wed, Jul 5, 2023 at 1:08 PM Alexander Fedulov <
> > > > > alexander.fedu...@gmail.com> wrote:
> > > > >
> > > > > > @Jing
> > > > > > Just to clarify, when you say:
> > > > > >
> > > > > > 3. remove SinkFunction.
> > > > > > Which steps do you imply for the 1.18 release and for the 2.0
> > > release?
> > > > >
> > > > > @Xintong
> > > > > > A side note - with the new Source API we lose the ability to
> > control
> > > > > > checkpointing from the source since there is no lock anymore.
> This
> > > > > > functionality
> > > > > > is currently used in a variety of tests for the Sinks - the tests
> > > that
> > > > > rely
> > > > > > on tight
> > > > > > synchronization between specific elements passed from the source
> > to
> > > > the
> > > > > > sink before
> > > > > > allowing a checkpoint to complete (see FiniteTestSource [1]).
> Since
> > > > > FLIP-27
> > > > > > Sources rely
> > > > > > on decoupling via the mailbox, without exposing the lock, it is
> not
> > > > > > immediately clear
> > > > > > if it is possible to achieve the same functionality without major
> > > > > > extensions in the
> > > > > > runtime for such testin

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-07-18 Thread Jane Chan
Hi Xintong,

Thanks for driving this topic. Regarding the Table API deprecation, I can
provide some details to help with the process.

   1. This sheet
   

summarizes
   the user-facing classes and methods that need to be deprecated under the
   flink-table module, some of which are marked with a red background and
   belong to the APIs that need to be depreciated but are not explicitly
   marked in the code. This mainly includes legacy table source/sink, legacy
   table schema, legacy SQL function, and some internal APIs designed for
   Paimon but are now obsolete.
   2. In addition, during the process of organizing, it was found that some
   APIs under the flink-table-api-java and flink-table-common modules do not
   have an explicit API annotation (you can find detailed information in this
   sheet
   
).
   I suggest explicitly marking the level for these APIs.
   3. As there are still some internal and test code dependencies on these
   APIs,  can we first gradually migrate these dependencies to alternative
   APIs to make the deprecation process relatively easy?

I hope this helps.

Best,
Jane

On Tue, Jul 11, 2023 at 12:08 PM Xintong Song  wrote:

> Thanks for bringing this up, Matthias. I've replied to you in the other
> thread[1].
>
> Best,
>
> Xintong
>
>
> [1] https://lists.apache.org/thread/l3dkdypyrovd3txzodn07lgdwtwvhgk4
>
> On Mon, Jul 10, 2023 at 8:37 PM Matthias Pohl
>  wrote:
>
> > @Xintong did you come across FLINK-3957 [1] when looking for deprecated
> > issues? There are a few subtasks that would require deprecations as well
> as
> > far as I can see. For some I'm wondering whether we should add them to
> the
> > release 2.0 feature list [2] in some way and (as a consequence) missed
> them
> > in FLINK-32557 [3], e.g.:
> > - FLINK-4675 [4]
> > - FLINK-14068 [5] (listed in the release 2.0 feature list [2] but not
> > listed in FLINK-32557 [3])
> > - FLINK-5126 [6]
> > - FLINK-13926 [7]
> >
> > For some other subtasks, I'm not 100% sure whether they still apply.
> Others
> > might resolve by removing the DataSet or Scala API. But for the 4
> mentioned
> > above we might need to deprecate API in 1.18.
> >
> > Matthias
> >
> > [1] https://issues.apache.org/jira/browse/FLINK-3957
> > [2] https://cwiki.apache.org/confluence/display/FLINK/2.0+Release
> > [3] https://issues.apache.org/jira/browse/FLINK-32557
> >
> > [4] https://issues.apache.org/jira/browse/FLINK-4675
> > [5] https://issues.apache.org/jira/browse/FLINK-14068
> > [6] https://issues.apache.org/jira/browse/FLINK-5126
> > [7] https://issues.apache.org/jira/browse/FLINK-13926
> >
> > On Fri, Jul 7, 2023 at 12:59 PM Jing Ge 
> > wrote:
> >
> > > Hi Alex,
> > >
> > > I would follow FLIP-197 and try to release them asap depending on dev
> > > resources and how difficult those issues are. The fastest timeline is
> the
> > > period defined in FLIP-197 in ideal conditions.
> > >
> > > Best regards,
> > > Jing
> > >
> > > On Fri, Jul 7, 2023 at 12:20 PM Alexander Fedulov <
> > > alexander.fedu...@gmail.com> wrote:
> > >
> > > > @Xintong
> > > > > - IIUC, the testing scenario you described is like blocking the
> > source
> > > > for
> > > > > proceeding (emit data, finish, etc.) until a checkpoint is
> finished.
> > > >
> > > > It is more tricky than that - we need to prevent the Sink from
> > receiving
> > > a
> > > > checkpoint barrier until the Source is done emitting a given set of
> > > > records. In
> > > > the current tests, which are also used for V2 Sinks, SourceFunction
> > > > controls
> > > > when the Sink is "allowed" to commit by holding the checkpoint lock
> > while
> > > > producing the records. The lock is not available in the new Source by
> > > > design
> > > > and we need a solution that provides the same functionality (without
> > > > modifying
> > > > the Sinks). I am currently checking if a workaround is at all
> possible
> > > > without
> > > > adjusting anything in the Source interface.
> > > >
> > > > > I may not have understood all the details, but based on what you
> > > > described
> > > > > I'd hesitate to block the deprecation / removal of SourceFunction
> on
> > > > this.
> > > >
> > > > I don't think we should, just wanted to highlight that there are some
> > > > unknowns
> > > > with respect to estimating the amount of work required.
> > > >
> > > > @Jing
> > > > I want to understand in which release would you target graduation of
> > the
> > > > mentioned connectors to @Public/@PublicEvolving - basically the
> > > anticipated
> > > > timeline of the steps in both options with respect to releases.
> > > >
> > > > Best,
> > > > Alex
> > > >
> > > > On Fri, 7 Jul 2023 at 10:53, Xintong Song 
> > wrote:
> > > >
> > > > > Thanks all for the discussion. I've created FLINK-32557 for this.
> > > > >
> > > > > Be

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-07-18 Thread Xintong Song
Thanks for the beautiful sheets, Jane.

1. This sheet <
> https://docs.google.com/spreadsheets/d/1dZBNHLuAHYJt3pFU8ZtfUzrYyf2ZFQ6wybDXGS1bHno/edit?usp=sharing>
> summarizes the user-facing classes and methods that need to be deprecated
> under the flink-table module, some of which are marked with a red
> background and belong to the APIs that need to be depreciated but are not
> explicitly marked in the code. This mainly includes legacy table
> source/sink, legacy table schema, legacy SQL function, and some internal
> APIs designed for Paimon but are now obsolete.
>
IIUC, you are suggesting to mark the classes with a red background as
`@Deprecated` in 1.18?

   - +1 for deprecating `StreamRecordTimestamp` & `ExistingField` in 1.18.
   Based on your description, it seems these were not marked by mistake. Let's
   fix them.
   - For the ManagedTable related classes, is there any FLIP explicitly
   that decides to deprecate them? If not, I think it would be nice to have
   one, to formally decide the deprecation with a vote. I'd expect such a FLIP
   can be quite lightweight.
   - For `SourceFunctionProvider`, please beware there are recently some
   objections against deprecating `SourceFunction`, and the code changes
   marking it as `@Deprecated` might be reverted. See [1][2] for more details.
   So my question is, if `SourceFunction` will not be deprecated until all
   sub-tasks in FLINK-28045 are resolved, do you still think we should
   deprecate `SourceFunctionProvider` now?

2. In addition, during the process of organizing, it was found that some
> APIs under the flink-table-api-java and flink-table-common modules do not
> have an explicit API annotation (you can find detailed information in this
> sheet <
> https://docs.google.com/spreadsheets/d/1e8M0tUtKkZXEd8rCZtZ0C6Ty9QkNaPySsrCgz0vEID4/edit?usp=sharing>).
> I suggest explicitly marking the level for these APIs.

I'm in general +1 to add the missing API annotations. However, I don't have
the expertise to comment on the classes and suggestd API levels being
listed.

3. As there are still some internal and test code dependencies on these
> APIs,  can we first gradually migrate these dependencies to alternative
> APIs to make the deprecation process relatively easy?
>
That makes sense to me. I think the purpose of trying to mark the APIs as
deprecated in 1.18 is to send users the signal early that these APIs will
be removed. As for the internal and test code dependencies, I don't see any
problem in gradually migrating them.

Best,

Xintong


[1] https://lists.apache.org/thread/734zhkvs59w2o4d1rsnozr1bfqlr6rgm

[2] https://issues.apache.org/jira/browse/FLINK-28046



On Wed, Jul 19, 2023 at 11:41 AM Jane Chan  wrote:

> Hi Xintong,
>
> Thanks for driving this topic. Regarding the Table API deprecation, I can
> provide some details to help with the process.
>
>1. This sheet
><
> https://docs.google.com/spreadsheets/d/1dZBNHLuAHYJt3pFU8ZtfUzrYyf2ZFQ6wybDXGS1bHno/edit?usp=sharing
> >
> summarizes
>the user-facing classes and methods that need to be deprecated under the
>flink-table module, some of which are marked with a red background and
>belong to the APIs that need to be depreciated but are not explicitly
>marked in the code. This mainly includes legacy table source/sink,
> legacy
>table schema, legacy SQL function, and some internal APIs designed for
>Paimon but are now obsolete.
>2. In addition, during the process of organizing, it was found that some
>APIs under the flink-table-api-java and flink-table-common modules do
> not
>have an explicit API annotation (you can find detailed information in
> this
>sheet
><
> https://docs.google.com/spreadsheets/d/1e8M0tUtKkZXEd8rCZtZ0C6Ty9QkNaPySsrCgz0vEID4/edit?usp=sharing
> >).
>I suggest explicitly marking the level for these APIs.
>3. As there are still some internal and test code dependencies on these
>APIs,  can we first gradually migrate these dependencies to alternative
>APIs to make the deprecation process relatively easy?
>
> I hope this helps.
>
> Best,
> Jane
>
> On Tue, Jul 11, 2023 at 12:08 PM Xintong Song 
> wrote:
>
> > Thanks for bringing this up, Matthias. I've replied to you in the other
> > thread[1].
> >
> > Best,
> >
> > Xintong
> >
> >
> > [1] https://lists.apache.org/thread/l3dkdypyrovd3txzodn07lgdwtwvhgk4
> >
> > On Mon, Jul 10, 2023 at 8:37 PM Matthias Pohl
> >  wrote:
> >
> > > @Xintong did you come across FLINK-3957 [1] when looking for deprecated
> > > issues? There are a few subtasks that would require deprecations as
> well
> > as
> > > far as I can see. For some I'm wondering whether we should add them to
> > the
> > > release 2.0 feature list [2] in some way and (as a consequence) missed
> > them
> > > in FLINK-32557 [3], e.g.:
> > > - FLINK-4675 [4]
> > > - FLINK-14068 [5] (listed in the release 2.0 feature list [2] but not
> > > listed in FLINK-32557 [3])
> > > - FLINK-5126 [6]
> > > - FLINK-13926 [7]
> > >

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-07-19 Thread Jane Chan
Hi Xintong,

> IIUC, you are suggesting to mark the classes with a red background as
`@Deprecated` in 1.18?

Exactly. They are not marked at the code level but suppose to be deprecated.

> - For the ManagedTable related classes, is there any FLIP explicitly that
decides to deprecate them? If not, I think it would be nice to have one, to
formally decide the deprecation with a vote. I'd expect such a FLIP can be
quite lightweight.

Currently, there's no formal FLIP yet. I'd like to prepare one to initiate
the deprecation process.

> - For `SourceFunctionProvider`, please beware there are recently some
objections against deprecating `SourceFunction`, and the code changes
marking it as `@Deprecated` might be reverted. See [1][2] for more details.
So my question is, if `SourceFunction` will not be deprecated until all
sub-tasks in FLINK-28045 are resolved, do you still think we should
deprecate `SourceFunctionProvider` now?

Thanks for the reminder. If `SourceFunction` is no longer being deprecated,
then `SourceFunctionProvider` also needs to be retained. I have updated the
sheet and removed `SourceFunctionProvider`.

> I'm in general +1 to add the missing API annotations. However, I don't
have the expertise to comment on the classes and suggestd API levels being
listed.

I've updated the sheet and added comments on all the APIs that are
suggested to be marked as PublicEvolving, explaining the reasons. The rest
APIs are either util classes or implementations and hence are suggested to
be Internal. I can start a discussion about the suggested API level to find
developers who can help review them.

Best,
Jane

On Wed, Jul 19, 2023 at 12:22 PM Xintong Song  wrote:

> Thanks for the beautiful sheets, Jane.
>
> 1. This sheet <
> >
> https://docs.google.com/spreadsheets/d/1dZBNHLuAHYJt3pFU8ZtfUzrYyf2ZFQ6wybDXGS1bHno/edit?usp=sharing
> >
> > summarizes the user-facing classes and methods that need to be deprecated
> > under the flink-table module, some of which are marked with a red
> > background and belong to the APIs that need to be depreciated but are not
> > explicitly marked in the code. This mainly includes legacy table
> > source/sink, legacy table schema, legacy SQL function, and some internal
> > APIs designed for Paimon but are now obsolete.
> >
> IIUC, you are suggesting to mark the classes with a red background as
> `@Deprecated` in 1.18?
>
>- +1 for deprecating `StreamRecordTimestamp` & `ExistingField` in 1.18.
>Based on your description, it seems these were not marked by mistake.
> Let's
>fix them.
>- For the ManagedTable related classes, is there any FLIP explicitly
>that decides to deprecate them? If not, I think it would be nice to have
>one, to formally decide the deprecation with a vote. I'd expect such a
> FLIP
>can be quite lightweight.
>- For `SourceFunctionProvider`, please beware there are recently some
>objections against deprecating `SourceFunction`, and the code changes
>marking it as `@Deprecated` might be reverted. See [1][2] for more
> details.
>So my question is, if `SourceFunction` will not be deprecated until all
>sub-tasks in FLINK-28045 are resolved, do you still think we should
>deprecate `SourceFunctionProvider` now?
>
> 2. In addition, during the process of organizing, it was found that some
> > APIs under the flink-table-api-java and flink-table-common modules do not
> > have an explicit API annotation (you can find detailed information in
> this
> > sheet <
> >
> https://docs.google.com/spreadsheets/d/1e8M0tUtKkZXEd8rCZtZ0C6Ty9QkNaPySsrCgz0vEID4/edit?usp=sharing
> >).
> > I suggest explicitly marking the level for these APIs.
>
> I'm in general +1 to add the missing API annotations. However, I don't have
> the expertise to comment on the classes and suggestd API levels being
> listed.
>
> 3. As there are still some internal and test code dependencies on these
> > APIs,  can we first gradually migrate these dependencies to alternative
> > APIs to make the deprecation process relatively easy?
> >
> That makes sense to me. I think the purpose of trying to mark the APIs as
> deprecated in 1.18 is to send users the signal early that these APIs will
> be removed. As for the internal and test code dependencies, I don't see any
> problem in gradually migrating them.
>
> Best,
>
> Xintong
>
>
> [1] https://lists.apache.org/thread/734zhkvs59w2o4d1rsnozr1bfqlr6rgm
>
> [2] https://issues.apache.org/jira/browse/FLINK-28046
>
>
>
> On Wed, Jul 19, 2023 at 11:41 AM Jane Chan  wrote:
>
> > Hi Xintong,
> >
> > Thanks for driving this topic. Regarding the Table API deprecation, I can
> > provide some details to help with the process.
> >
> >1. This sheet
> ><
> >
> https://docs.google.com/spreadsheets/d/1dZBNHLuAHYJt3pFU8ZtfUzrYyf2ZFQ6wybDXGS1bHno/edit?usp=sharing
> > >
> > summarizes
> >the user-facing classes and methods that need to be deprecated under
> the
> >flink-table module, some of which are marked with a red backgroun

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-07-19 Thread Xintong Song
Thank you, Jane.

What you said (preparing a FLIP for ManagedTable related classes, not
deprecating SourceFunctionProvider, and starting a dedicated discussion for
the missing annotations) sounds good to me.

In addition, if there's no objections from the community on marking
`StreamRecordTimestamp` & `ExistingField` as deprecated, you may consider
creating a Jira ticket for it as a sub-task of FLINK-32557.

Best,

Xintong



On Wed, Jul 19, 2023 at 3:48 PM Jane Chan  wrote:

> Hi Xintong,
>
> > IIUC, you are suggesting to mark the classes with a red background as
> `@Deprecated` in 1.18?
>
> Exactly. They are not marked at the code level but suppose to be
> deprecated.
>
> > - For the ManagedTable related classes, is there any FLIP explicitly that
> decides to deprecate them? If not, I think it would be nice to have one, to
> formally decide the deprecation with a vote. I'd expect such a FLIP can be
> quite lightweight.
>
> Currently, there's no formal FLIP yet. I'd like to prepare one to initiate
> the deprecation process.
>
> > - For `SourceFunctionProvider`, please beware there are recently some
> objections against deprecating `SourceFunction`, and the code changes
> marking it as `@Deprecated` might be reverted. See [1][2] for more details.
> So my question is, if `SourceFunction` will not be deprecated until all
> sub-tasks in FLINK-28045 are resolved, do you still think we should
> deprecate `SourceFunctionProvider` now?
>
> Thanks for the reminder. If `SourceFunction` is no longer being deprecated,
> then `SourceFunctionProvider` also needs to be retained. I have updated the
> sheet and removed `SourceFunctionProvider`.
>
> > I'm in general +1 to add the missing API annotations. However, I don't
> have the expertise to comment on the classes and suggestd API levels being
> listed.
>
> I've updated the sheet and added comments on all the APIs that are
> suggested to be marked as PublicEvolving, explaining the reasons. The rest
> APIs are either util classes or implementations and hence are suggested to
> be Internal. I can start a discussion about the suggested API level to find
> developers who can help review them.
>
> Best,
> Jane
>
> On Wed, Jul 19, 2023 at 12:22 PM Xintong Song 
> wrote:
>
> > Thanks for the beautiful sheets, Jane.
> >
> > 1. This sheet <
> > >
> >
> https://docs.google.com/spreadsheets/d/1dZBNHLuAHYJt3pFU8ZtfUzrYyf2ZFQ6wybDXGS1bHno/edit?usp=sharing
> > >
> > > summarizes the user-facing classes and methods that need to be
> deprecated
> > > under the flink-table module, some of which are marked with a red
> > > background and belong to the APIs that need to be depreciated but are
> not
> > > explicitly marked in the code. This mainly includes legacy table
> > > source/sink, legacy table schema, legacy SQL function, and some
> internal
> > > APIs designed for Paimon but are now obsolete.
> > >
> > IIUC, you are suggesting to mark the classes with a red background as
> > `@Deprecated` in 1.18?
> >
> >- +1 for deprecating `StreamRecordTimestamp` & `ExistingField` in
> 1.18.
> >Based on your description, it seems these were not marked by mistake.
> > Let's
> >fix them.
> >- For the ManagedTable related classes, is there any FLIP explicitly
> >that decides to deprecate them? If not, I think it would be nice to
> have
> >one, to formally decide the deprecation with a vote. I'd expect such a
> > FLIP
> >can be quite lightweight.
> >- For `SourceFunctionProvider`, please beware there are recently some
> >objections against deprecating `SourceFunction`, and the code changes
> >marking it as `@Deprecated` might be reverted. See [1][2] for more
> > details.
> >So my question is, if `SourceFunction` will not be deprecated until
> all
> >sub-tasks in FLINK-28045 are resolved, do you still think we should
> >deprecate `SourceFunctionProvider` now?
> >
> > 2. In addition, during the process of organizing, it was found that some
> > > APIs under the flink-table-api-java and flink-table-common modules do
> not
> > > have an explicit API annotation (you can find detailed information in
> > this
> > > sheet <
> > >
> >
> https://docs.google.com/spreadsheets/d/1e8M0tUtKkZXEd8rCZtZ0C6Ty9QkNaPySsrCgz0vEID4/edit?usp=sharing
> > >).
> > > I suggest explicitly marking the level for these APIs.
> >
> > I'm in general +1 to add the missing API annotations. However, I don't
> have
> > the expertise to comment on the classes and suggestd API levels being
> > listed.
> >
> > 3. As there are still some internal and test code dependencies on these
> > > APIs,  can we first gradually migrate these dependencies to alternative
> > > APIs to make the deprecation process relatively easy?
> > >
> > That makes sense to me. I think the purpose of trying to mark the APIs as
> > deprecated in 1.18 is to send users the signal early that these APIs will
> > be removed. As for the internal and test code dependencies, I don't see
> any
> > problem in gradually migrating

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-07-20 Thread Jing Ge
Hi Jane, Hi Xintong,

The specific discussion of deprecating `StreamRecordTimestamp` &
`ExistingField` was kind of hiding in the discussion of the umbrella topic
that people might miss out. Does it make sense to start an
individual Discussion thread for it like we deprecate any other APIs?

For all APIs that should be marked as @PublicEvolving, dedicated graduation
threads are recommended.

Best regards,
Jing


On Wed, Jul 19, 2023 at 10:09 AM Xintong Song  wrote:

> Thank you, Jane.
>
> What you said (preparing a FLIP for ManagedTable related classes, not
> deprecating SourceFunctionProvider, and starting a dedicated discussion for
> the missing annotations) sounds good to me.
>
> In addition, if there's no objections from the community on marking
> `StreamRecordTimestamp` & `ExistingField` as deprecated, you may consider
> creating a Jira ticket for it as a sub-task of FLINK-32557.
>
> Best,
>
> Xintong
>
>
>
> On Wed, Jul 19, 2023 at 3:48 PM Jane Chan  wrote:
>
> > Hi Xintong,
> >
> > > IIUC, you are suggesting to mark the classes with a red background as
> > `@Deprecated` in 1.18?
> >
> > Exactly. They are not marked at the code level but suppose to be
> > deprecated.
> >
> > > - For the ManagedTable related classes, is there any FLIP explicitly
> that
> > decides to deprecate them? If not, I think it would be nice to have one,
> to
> > formally decide the deprecation with a vote. I'd expect such a FLIP can
> be
> > quite lightweight.
> >
> > Currently, there's no formal FLIP yet. I'd like to prepare one to
> initiate
> > the deprecation process.
> >
> > > - For `SourceFunctionProvider`, please beware there are recently some
> > objections against deprecating `SourceFunction`, and the code changes
> > marking it as `@Deprecated` might be reverted. See [1][2] for more
> details.
> > So my question is, if `SourceFunction` will not be deprecated until all
> > sub-tasks in FLINK-28045 are resolved, do you still think we should
> > deprecate `SourceFunctionProvider` now?
> >
> > Thanks for the reminder. If `SourceFunction` is no longer being
> deprecated,
> > then `SourceFunctionProvider` also needs to be retained. I have updated
> the
> > sheet and removed `SourceFunctionProvider`.
> >
> > > I'm in general +1 to add the missing API annotations. However, I don't
> > have the expertise to comment on the classes and suggestd API levels
> being
> > listed.
> >
> > I've updated the sheet and added comments on all the APIs that are
> > suggested to be marked as PublicEvolving, explaining the reasons. The
> rest
> > APIs are either util classes or implementations and hence are suggested
> to
> > be Internal. I can start a discussion about the suggested API level to
> find
> > developers who can help review them.
> >
> > Best,
> > Jane
> >
> > On Wed, Jul 19, 2023 at 12:22 PM Xintong Song 
> > wrote:
> >
> > > Thanks for the beautiful sheets, Jane.
> > >
> > > 1. This sheet <
> > > >
> > >
> >
> https://docs.google.com/spreadsheets/d/1dZBNHLuAHYJt3pFU8ZtfUzrYyf2ZFQ6wybDXGS1bHno/edit?usp=sharing
> > > >
> > > > summarizes the user-facing classes and methods that need to be
> > deprecated
> > > > under the flink-table module, some of which are marked with a red
> > > > background and belong to the APIs that need to be depreciated but are
> > not
> > > > explicitly marked in the code. This mainly includes legacy table
> > > > source/sink, legacy table schema, legacy SQL function, and some
> > internal
> > > > APIs designed for Paimon but are now obsolete.
> > > >
> > > IIUC, you are suggesting to mark the classes with a red background as
> > > `@Deprecated` in 1.18?
> > >
> > >- +1 for deprecating `StreamRecordTimestamp` & `ExistingField` in
> > 1.18.
> > >Based on your description, it seems these were not marked by
> mistake.
> > > Let's
> > >fix them.
> > >- For the ManagedTable related classes, is there any FLIP explicitly
> > >that decides to deprecate them? If not, I think it would be nice to
> > have
> > >one, to formally decide the deprecation with a vote. I'd expect
> such a
> > > FLIP
> > >can be quite lightweight.
> > >- For `SourceFunctionProvider`, please beware there are recently
> some
> > >objections against deprecating `SourceFunction`, and the code
> changes
> > >marking it as `@Deprecated` might be reverted. See [1][2] for more
> > > details.
> > >So my question is, if `SourceFunction` will not be deprecated until
> > all
> > >sub-tasks in FLINK-28045 are resolved, do you still think we should
> > >deprecate `SourceFunctionProvider` now?
> > >
> > > 2. In addition, during the process of organizing, it was found that
> some
> > > > APIs under the flink-table-api-java and flink-table-common modules do
> > not
> > > > have an explicit API annotation (you can find detailed information in
> > > this
> > > > sheet <
> > > >
> > >
> >
> https://docs.google.com/spreadsheets/d/1e8M0tUtKkZXEd8rCZtZ0C6Ty9QkNaPySsrCgz0vEID4/edit?usp=sharing
> > > >).
> > > > I

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-07-20 Thread Xintong Song
I personally don't think it's necessary. IIUC, we have already decided to
deprecate TableSource / TableSink, and not marking StreamRecordTimestamp
and ExistingField deprecated is just an oversight in carrying out that
decision.


Best,

Xintong



On Fri, Jul 21, 2023 at 7:48 AM Jing Ge  wrote:

> Hi Jane, Hi Xintong,
>
> The specific discussion of deprecating `StreamRecordTimestamp` &
> `ExistingField` was kind of hiding in the discussion of the umbrella topic
> that people might miss out. Does it make sense to start an
> individual Discussion thread for it like we deprecate any other APIs?
>
> For all APIs that should be marked as @PublicEvolving, dedicated graduation
> threads are recommended.
>
> Best regards,
> Jing
>
>
> On Wed, Jul 19, 2023 at 10:09 AM Xintong Song 
> wrote:
>
> > Thank you, Jane.
> >
> > What you said (preparing a FLIP for ManagedTable related classes, not
> > deprecating SourceFunctionProvider, and starting a dedicated discussion
> for
> > the missing annotations) sounds good to me.
> >
> > In addition, if there's no objections from the community on marking
> > `StreamRecordTimestamp` & `ExistingField` as deprecated, you may consider
> > creating a Jira ticket for it as a sub-task of FLINK-32557.
> >
> > Best,
> >
> > Xintong
> >
> >
> >
> > On Wed, Jul 19, 2023 at 3:48 PM Jane Chan  wrote:
> >
> > > Hi Xintong,
> > >
> > > > IIUC, you are suggesting to mark the classes with a red background as
> > > `@Deprecated` in 1.18?
> > >
> > > Exactly. They are not marked at the code level but suppose to be
> > > deprecated.
> > >
> > > > - For the ManagedTable related classes, is there any FLIP explicitly
> > that
> > > decides to deprecate them? If not, I think it would be nice to have
> one,
> > to
> > > formally decide the deprecation with a vote. I'd expect such a FLIP can
> > be
> > > quite lightweight.
> > >
> > > Currently, there's no formal FLIP yet. I'd like to prepare one to
> > initiate
> > > the deprecation process.
> > >
> > > > - For `SourceFunctionProvider`, please beware there are recently some
> > > objections against deprecating `SourceFunction`, and the code changes
> > > marking it as `@Deprecated` might be reverted. See [1][2] for more
> > details.
> > > So my question is, if `SourceFunction` will not be deprecated until all
> > > sub-tasks in FLINK-28045 are resolved, do you still think we should
> > > deprecate `SourceFunctionProvider` now?
> > >
> > > Thanks for the reminder. If `SourceFunction` is no longer being
> > deprecated,
> > > then `SourceFunctionProvider` also needs to be retained. I have updated
> > the
> > > sheet and removed `SourceFunctionProvider`.
> > >
> > > > I'm in general +1 to add the missing API annotations. However, I
> don't
> > > have the expertise to comment on the classes and suggestd API levels
> > being
> > > listed.
> > >
> > > I've updated the sheet and added comments on all the APIs that are
> > > suggested to be marked as PublicEvolving, explaining the reasons. The
> > rest
> > > APIs are either util classes or implementations and hence are suggested
> > to
> > > be Internal. I can start a discussion about the suggested API level to
> > find
> > > developers who can help review them.
> > >
> > > Best,
> > > Jane
> > >
> > > On Wed, Jul 19, 2023 at 12:22 PM Xintong Song 
> > > wrote:
> > >
> > > > Thanks for the beautiful sheets, Jane.
> > > >
> > > > 1. This sheet <
> > > > >
> > > >
> > >
> >
> https://docs.google.com/spreadsheets/d/1dZBNHLuAHYJt3pFU8ZtfUzrYyf2ZFQ6wybDXGS1bHno/edit?usp=sharing
> > > > >
> > > > > summarizes the user-facing classes and methods that need to be
> > > deprecated
> > > > > under the flink-table module, some of which are marked with a red
> > > > > background and belong to the APIs that need to be depreciated but
> are
> > > not
> > > > > explicitly marked in the code. This mainly includes legacy table
> > > > > source/sink, legacy table schema, legacy SQL function, and some
> > > internal
> > > > > APIs designed for Paimon but are now obsolete.
> > > > >
> > > > IIUC, you are suggesting to mark the classes with a red background as
> > > > `@Deprecated` in 1.18?
> > > >
> > > >- +1 for deprecating `StreamRecordTimestamp` & `ExistingField` in
> > > 1.18.
> > > >Based on your description, it seems these were not marked by
> > mistake.
> > > > Let's
> > > >fix them.
> > > >- For the ManagedTable related classes, is there any FLIP
> explicitly
> > > >that decides to deprecate them? If not, I think it would be nice
> to
> > > have
> > > >one, to formally decide the deprecation with a vote. I'd expect
> > such a
> > > > FLIP
> > > >can be quite lightweight.
> > > >- For `SourceFunctionProvider`, please beware there are recently
> > some
> > > >objections against deprecating `SourceFunction`, and the code
> > changes
> > > >marking it as `@Deprecated` might be reverted. See [1][2] for more
> > > > details.
> > > >So my question is, if `SourceFunction` will not be depr

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-07-20 Thread Jane Chan
Hi, Jing,

I share the same opinion with Xintong. The base class `TimestampExtractor`
for these two classes was deprecated in release-1.12[1][2], so deprecating
these two subclasses is relatively straightforward.

[1] https://issues.apache.org/jira/browse/FLINK-19453
[2]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-95%3A+New+TableSource+and+TableSink+interfaces


Best,
Jane

On Fri, Jul 21, 2023 at 10:19 AM Xintong Song  wrote:

> I personally don't think it's necessary. IIUC, we have already decided to
> deprecate TableSource / TableSink, and not marking StreamRecordTimestamp
> and ExistingField deprecated is just an oversight in carrying out that
> decision.
>
>
> Best,
>
> Xintong
>
>
>
> On Fri, Jul 21, 2023 at 7:48 AM Jing Ge 
> wrote:
>
> > Hi Jane, Hi Xintong,
> >
> > The specific discussion of deprecating `StreamRecordTimestamp` &
> > `ExistingField` was kind of hiding in the discussion of the umbrella
> topic
> > that people might miss out. Does it make sense to start an
> > individual Discussion thread for it like we deprecate any other APIs?
> >
> > For all APIs that should be marked as @PublicEvolving, dedicated
> graduation
> > threads are recommended.
> >
> > Best regards,
> > Jing
> >
> >
> > On Wed, Jul 19, 2023 at 10:09 AM Xintong Song 
> > wrote:
> >
> > > Thank you, Jane.
> > >
> > > What you said (preparing a FLIP for ManagedTable related classes, not
> > > deprecating SourceFunctionProvider, and starting a dedicated discussion
> > for
> > > the missing annotations) sounds good to me.
> > >
> > > In addition, if there's no objections from the community on marking
> > > `StreamRecordTimestamp` & `ExistingField` as deprecated, you may
> consider
> > > creating a Jira ticket for it as a sub-task of FLINK-32557.
> > >
> > > Best,
> > >
> > > Xintong
> > >
> > >
> > >
> > > On Wed, Jul 19, 2023 at 3:48 PM Jane Chan 
> wrote:
> > >
> > > > Hi Xintong,
> > > >
> > > > > IIUC, you are suggesting to mark the classes with a red background
> as
> > > > `@Deprecated` in 1.18?
> > > >
> > > > Exactly. They are not marked at the code level but suppose to be
> > > > deprecated.
> > > >
> > > > > - For the ManagedTable related classes, is there any FLIP
> explicitly
> > > that
> > > > decides to deprecate them? If not, I think it would be nice to have
> > one,
> > > to
> > > > formally decide the deprecation with a vote. I'd expect such a FLIP
> can
> > > be
> > > > quite lightweight.
> > > >
> > > > Currently, there's no formal FLIP yet. I'd like to prepare one to
> > > initiate
> > > > the deprecation process.
> > > >
> > > > > - For `SourceFunctionProvider`, please beware there are recently
> some
> > > > objections against deprecating `SourceFunction`, and the code changes
> > > > marking it as `@Deprecated` might be reverted. See [1][2] for more
> > > details.
> > > > So my question is, if `SourceFunction` will not be deprecated until
> all
> > > > sub-tasks in FLINK-28045 are resolved, do you still think we should
> > > > deprecate `SourceFunctionProvider` now?
> > > >
> > > > Thanks for the reminder. If `SourceFunction` is no longer being
> > > deprecated,
> > > > then `SourceFunctionProvider` also needs to be retained. I have
> updated
> > > the
> > > > sheet and removed `SourceFunctionProvider`.
> > > >
> > > > > I'm in general +1 to add the missing API annotations. However, I
> > don't
> > > > have the expertise to comment on the classes and suggestd API levels
> > > being
> > > > listed.
> > > >
> > > > I've updated the sheet and added comments on all the APIs that are
> > > > suggested to be marked as PublicEvolving, explaining the reasons. The
> > > rest
> > > > APIs are either util classes or implementations and hence are
> suggested
> > > to
> > > > be Internal. I can start a discussion about the suggested API level
> to
> > > find
> > > > developers who can help review them.
> > > >
> > > > Best,
> > > > Jane
> > > >
> > > > On Wed, Jul 19, 2023 at 12:22 PM Xintong Song  >
> > > > wrote:
> > > >
> > > > > Thanks for the beautiful sheets, Jane.
> > > > >
> > > > > 1. This sheet <
> > > > > >
> > > > >
> > > >
> > >
> >
> https://docs.google.com/spreadsheets/d/1dZBNHLuAHYJt3pFU8ZtfUzrYyf2ZFQ6wybDXGS1bHno/edit?usp=sharing
> > > > > >
> > > > > > summarizes the user-facing classes and methods that need to be
> > > > deprecated
> > > > > > under the flink-table module, some of which are marked with a red
> > > > > > background and belong to the APIs that need to be depreciated but
> > are
> > > > not
> > > > > > explicitly marked in the code. This mainly includes legacy table
> > > > > > source/sink, legacy table schema, legacy SQL function, and some
> > > > internal
> > > > > > APIs designed for Paimon but are now obsolete.
> > > > > >
> > > > > IIUC, you are suggesting to mark the classes with a red background
> as
> > > > > `@Deprecated` in 1.18?
> > > > >
> > > > >- +1 for deprecating `StreamRecordTimestamp` & `ExistingField`
> in
> > > > 1.18.
> > > > >Based on your de

Re: [DISUCSS] Deprecate multiple APIs in 1.18

2023-07-21 Thread Jing Ge
got it, thanks for the clarification.

Best regards,
Jing

On Fri, Jul 21, 2023 at 5:14 AM Jane Chan  wrote:

> Hi, Jing,
>
> I share the same opinion with Xintong. The base class `TimestampExtractor`
> for these two classes was deprecated in release-1.12[1][2], so deprecating
> these two subclasses is relatively straightforward.
>
> [1] https://issues.apache.org/jira/browse/FLINK-19453
> [2]
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-95%3A+New+TableSource+and+TableSink+interfaces
>
>
> Best,
> Jane
>
> On Fri, Jul 21, 2023 at 10:19 AM Xintong Song 
> wrote:
>
> > I personally don't think it's necessary. IIUC, we have already decided to
> > deprecate TableSource / TableSink, and not marking StreamRecordTimestamp
> > and ExistingField deprecated is just an oversight in carrying out that
> > decision.
> >
> >
> > Best,
> >
> > Xintong
> >
> >
> >
> > On Fri, Jul 21, 2023 at 7:48 AM Jing Ge 
> > wrote:
> >
> > > Hi Jane, Hi Xintong,
> > >
> > > The specific discussion of deprecating `StreamRecordTimestamp` &
> > > `ExistingField` was kind of hiding in the discussion of the umbrella
> > topic
> > > that people might miss out. Does it make sense to start an
> > > individual Discussion thread for it like we deprecate any other APIs?
> > >
> > > For all APIs that should be marked as @PublicEvolving, dedicated
> > graduation
> > > threads are recommended.
> > >
> > > Best regards,
> > > Jing
> > >
> > >
> > > On Wed, Jul 19, 2023 at 10:09 AM Xintong Song 
> > > wrote:
> > >
> > > > Thank you, Jane.
> > > >
> > > > What you said (preparing a FLIP for ManagedTable related classes, not
> > > > deprecating SourceFunctionProvider, and starting a dedicated
> discussion
> > > for
> > > > the missing annotations) sounds good to me.
> > > >
> > > > In addition, if there's no objections from the community on marking
> > > > `StreamRecordTimestamp` & `ExistingField` as deprecated, you may
> > consider
> > > > creating a Jira ticket for it as a sub-task of FLINK-32557.
> > > >
> > > > Best,
> > > >
> > > > Xintong
> > > >
> > > >
> > > >
> > > > On Wed, Jul 19, 2023 at 3:48 PM Jane Chan 
> > wrote:
> > > >
> > > > > Hi Xintong,
> > > > >
> > > > > > IIUC, you are suggesting to mark the classes with a red
> background
> > as
> > > > > `@Deprecated` in 1.18?
> > > > >
> > > > > Exactly. They are not marked at the code level but suppose to be
> > > > > deprecated.
> > > > >
> > > > > > - For the ManagedTable related classes, is there any FLIP
> > explicitly
> > > > that
> > > > > decides to deprecate them? If not, I think it would be nice to have
> > > one,
> > > > to
> > > > > formally decide the deprecation with a vote. I'd expect such a FLIP
> > can
> > > > be
> > > > > quite lightweight.
> > > > >
> > > > > Currently, there's no formal FLIP yet. I'd like to prepare one to
> > > > initiate
> > > > > the deprecation process.
> > > > >
> > > > > > - For `SourceFunctionProvider`, please beware there are recently
> > some
> > > > > objections against deprecating `SourceFunction`, and the code
> changes
> > > > > marking it as `@Deprecated` might be reverted. See [1][2] for more
> > > > details.
> > > > > So my question is, if `SourceFunction` will not be deprecated until
> > all
> > > > > sub-tasks in FLINK-28045 are resolved, do you still think we should
> > > > > deprecate `SourceFunctionProvider` now?
> > > > >
> > > > > Thanks for the reminder. If `SourceFunction` is no longer being
> > > > deprecated,
> > > > > then `SourceFunctionProvider` also needs to be retained. I have
> > updated
> > > > the
> > > > > sheet and removed `SourceFunctionProvider`.
> > > > >
> > > > > > I'm in general +1 to add the missing API annotations. However, I
> > > don't
> > > > > have the expertise to comment on the classes and suggestd API
> levels
> > > > being
> > > > > listed.
> > > > >
> > > > > I've updated the sheet and added comments on all the APIs that are
> > > > > suggested to be marked as PublicEvolving, explaining the reasons.
> The
> > > > rest
> > > > > APIs are either util classes or implementations and hence are
> > suggested
> > > > to
> > > > > be Internal. I can start a discussion about the suggested API level
> > to
> > > > find
> > > > > developers who can help review them.
> > > > >
> > > > > Best,
> > > > > Jane
> > > > >
> > > > > On Wed, Jul 19, 2023 at 12:22 PM Xintong Song <
> tonysong...@gmail.com
> > >
> > > > > wrote:
> > > > >
> > > > > > Thanks for the beautiful sheets, Jane.
> > > > > >
> > > > > > 1. This sheet <
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://docs.google.com/spreadsheets/d/1dZBNHLuAHYJt3pFU8ZtfUzrYyf2ZFQ6wybDXGS1bHno/edit?usp=sharing
> > > > > > >
> > > > > > > summarizes the user-facing classes and methods that need to be
> > > > > deprecated
> > > > > > > under the flink-table module, some of which are marked with a
> red
> > > > > > > background and belong to the APIs that need to be depreciated
> but
> > > are
> > > > > not
> > > > > > > explicitly marked in