Re: [DISCUSS] FLIP-408: [Umbrella] Introduce DataStream API V2

2024-02-19 Thread weijie guo
Hi All,

Thanks for all the feedback.

If there are no more comments, I would like to start the vote thread,
thanks again!

Best regards,

Weijie


Xintong Song  于2024年1月30日周二 11:04写道:

> Thanks for working on this, Weijie.
>
> The design flaws of the current DataStream API (i.e., V1) have been a pain
> for a long time. It's great to see efforts going on trying to resolve them.
>
> Significant changes to such an important and comprehensive set of public
> APIs deserves caution. From that perspective, the ideas of introducing a
> new set of APIs that gradually replace the current one, splitting the
> introducing of the new APIs into many separate FLIPs, and making
> intermediate APIs @Experiemental until all of them are completed make
> great sense to me.
>
> Besides, the ideas of generalized watermark, execution hints sound quite
> interesting. Looking forward to more detailed discussions in the
> corresponding sub-FLIPs.
>
> +1 for the roadmap.
>
> Best,
>
> Xintong
>
>
>
> On Tue, Jan 30, 2024 at 11:00 AM weijie guo 
> wrote:
>
> > Hi Wencong:
> >
> > > The Processing TimerService is currently
> > defined as one of the basic primitives, partly because it's understood
> that
> > you have to choose between processing time and event time.
> > The other part of the reason is that it needs to work based on the task's
> > mailbox thread model to avoid concurrency issues. Could you clarify the
> > second
> > part of the reason?
> >
> > Since the processing logic of the operators takes place in the mailbox
> > thread, the processing timer's callback function must also be executed in
> > the mailbox to ensure thread safety.
> > If we do not define the Processing TimerService as primitive, there is no
> > way for the user to dispatch custom logic to the mailbox thread.
> >
> >
> > Best regards,
> >
> > Weijie
> >
> >
> > Xuannan Su  于2024年1月29日周一 17:12写道:
> >
> > > Hi Weijie,
> > >
> > > Thanks for driving the work! There are indeed many pain points in the
> > > current DataStream API, which are challenging to resolve with its
> > > existing design. It is a great opportunity to propose a new DataStream
> > > API that tackles these issues. I like the way we've divided the FLIP
> > > into multiple sub-FLIPs; the roadmap is clear and comprehensible. +1
> > > for the umbrella FLIP. I am eager to see the sub-FLIPs!
> > >
> > > Best regards,
> > > Xuannan
> > >
> > >
> > >
> > >
> > > On Wed, Jan 24, 2024 at 8:55 PM Wencong Liu 
> > wrote:
> > > >
> > > > Hi Weijie,
> > > >
> > > >
> > > > Thank you for the effort you've put into the DataStream API ! By
> > > reorganizing and
> > > > redesigning the DataStream API, as well as addressing some of the
> > > unreasonable
> > > > designs within it, we can enhance the efficiency of job development
> for
> > > developers.
> > > > It also allows developers to design more flexible Flink jobs to meet
> > > business requirements.
> > > >
> > > >
> > > > I have conducted a comprehensive review of the DataStream API design
> in
> > > versions
> > > > 1.18 and 1.19. I found quite a few functional defects in the
> DataStream
> > > API, such as the
> > > > lack of corresponding APIs in batch processing scenarios. In the
> > > upcoming 1.20 version,
> > > > I will further improve the DataStream API in batch computing
> scenarios.
> > > >
> > > >
> > > > The issues existing in the old DataStream API (which can be referred
> to
> > > as V1) can be
> > > > addressed from a design perspective in the initial version of V2. I
> > hope
> > > to also have the
> > > >  opportunity to participate in the development of DataStream V2 and
> > make
> > > my contribution.
> > > >
> > > >
> > > > Regarding FLIP-408, I have a question: The Processing TimerService is
> > > currently
> > > > defined as one of the basic primitives, partly because it's
> understood
> > > that
> > > > you have to choose between processing time and event time.
> > > > The other part of the reason is that it needs to work based on the
> > task's
> > > > mailbox thread model to avoid concurrency issues. Could you clarify
> the
> > > second
> > > > part of the reason?
> > > >
> > > > Best,
> > > > Wencong Liu
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > > At 2023-12-26 14:42:20, "weijie guo" 
> > wrote:
> > > > >Hi devs,
> > > > >
> > > > >
> > > > >I'd like to start a discussion about FLIP-408: [Umbrella] Introduce
> > > > >DataStream API V2 [1].
> > > > >
> > > > >
> > > > >The DataStream API is one of the two main APIs that Flink provides
> for
> > > > >writing data processing programs. As an API that was introduced
> > > > >practically since day-1 of the project and has been evolved for
> nearly
> > > > >a decade, we are observing more and more problems of it.
> Improvements
> > > > >on these problems require significant breaking changes, which makes
> > > > >in-place refactor impractical. Therefore, we propose to introduce a
> > > > >new set of APIs, 

Re: [DISCUSS] FLIP-409: DataStream V2 Building Blocks: DataStream, Partitioning and ProcessFunction

2024-02-19 Thread weijie guo
Hi All,

Thanks for all the feedback.

If there are no more comments, I would like to start the vote thread,
thanks again!

Best regards,

Weijie


Xintong Song  于2024年2月20日周二 14:17写道:

> Thanks for the updates. LGTM.
>
> Best,
>
> Xintong
>
>
>
> On Mon, Feb 19, 2024 at 10:51 AM weijie guo 
> wrote:
>
> > Thanks for the reply, Xintong.
> >
> > Based on your comments, I made the following changes to this FLIP:
> >
> > 1. Renaming `TwoInputStreamProcessFunction` and
> > `BroadcastTwoInputStreamProcessFunction` to
> > `TwoInputNonBroadcastStreamProcessFunction` and
> > `TwoInputBroadcastStreamProcessFunction`, respectively.
> >
> > 2. Making `NonPartitionedContext` extend `RuntimeContext`.
> >
> > > Some of these changes also affect FLIP-410. I noticed that FLIP-410 is
> > also updated accordingly. It would be nice to also mention those changes
> in
> > the FLIP-410 discussion thread.
> >
> > Yes, I've now mentioned those updates in the FLIP-410 discussion thread.
> >
> >
> > Best regards,
> >
> > Weijie
> >
> >
> > Xintong Song  于2024年2月5日周一 10:58写道:
> >
> > > Thanks for updating the FLIP, Weijie.
> > >
> > > I think separating the TwoInputProcessFunction according to whether the
> > > input stream contains BroadcastStream makes sense.
> > >
> > > I have a few more comments.
> > > 1. I'd suggest the names `TwoInputNonBroadcastStreamProcessFunction`
> and
> > > `TwoInputBroadcastStreamProcessFunction` for the separated methods.
> > > 2. I'd suggest making `NonPartitionedContext` extend `RuntimeContext`.
> > > Otherwise, for all the functionalities that `RuntimeContext` provides,
> we
> > > need to duplicate them for `NonPartitionedContext`.
> > > 3. Some of these changes also affect FLIP-410. I noticed that FLIP-410
> is
> > > also updated accordingly. It would be nice to also mention those
> changes
> > in
> > > the FLIP-410 discussion thread.
> > >
> > > Best,
> > >
> > > Xintong
> > >
> > >
> > >
> > > On Sun, Feb 4, 2024 at 11:23 AM weijie guo 
> > > wrote:
> > >
> > > > Hi Xuannan and Xintong,
> > > >
> > > > Good point! After further consideration, I feel that we should make
> the
> > > > Broadcast + NonKeyed/Keyed process function different from the normal
> > > > TwoInputProcessFunction. Because the record from the broadcast input
> > > indeed
> > > > correspond to all partitions, while the record from the non-broadcast
> > > edge
> > > > have explicit partitions.
> > > >
> > > > When we consider the data of broadcast input, it is only valid to do
> > > > something on all the partitions at once, such as things like
> > > > `applyToKeyedState`. Similarly, other operations(e.g, endOfInput)
> that
> > do
> > > > not determine the current partition should also only be allowed to
> > > perform
> > > > on all partitions. This FLIP has been updated.
> > > >
> > > > Best regards,
> > > >
> > > > Weijie
> > > >
> > > >
> > > > Xintong Song  于2024年2月1日周四 11:31写道:
> > > >
> > > > > OK, I see your point.
> > > > >
> > > > > I think the demand for updating states and emitting outputs upon
> > > > receiving
> > > > > a broadcast record makes sense. However, the way
> > > > > `KeyedBroadcastProcessFunction` supports this may not be optimal.
> > E.g.,
> > > > if
> > > > > `Collector#collect` is called in `processBroadcastElement` but
> > outside
> > > of
> > > > > `Context#applyToKeyedState`, the result can be undefined.
> > > > >
> > > > > Currently in this FLIP, a `TwoInputStreamProcessFunction` is not
> > aware
> > > of
> > > > > which input is KeyedStream and which is BroadcastStream, which
> makes
> > > > > supporting things like `applyToKeyedState` difficult. I think we
> can
> > > > > provide a built-in function similar to
> > `KeyedBroadcastProcessFunction`
> > > on
> > > > > top of `TwoInputStreamProcessFunction` to address this demand.
> > > > >
> > > > > WDYT?
> > > > >
> > > > >
> > > > > Best,
> > > > >
> > > > > Xintong
> > > > >
> > > > >
> > > > >
> > > > > On Thu, Feb 1, 2024 at 10:41 AM Xuannan Su 
> > > > wrote:
> > > > >
> > > > > > Hi Weijie and Xingtong,
> > > > > >
> > > > > > Thanks for the reply! Please see my comments below.
> > > > > >
> > > > > > > Does this mean if we want to support (KeyedStream,
> > BroadcastStream)
> > > > ->
> > > > > > > (KeyedStream), we must make sure that no data can be output
> upon
> > > > > > processing
> > > > > > > records from the input BroadcastStream? That's probably a
> > > reasonable
> > > > > > > limitation.
> > > > > >
> > > > > > I don't think that the requirement for supporting (KeyedStream,
> > > > > > BroadcastStream) -> (KeyedStream) is that no data can be output
> > upon
> > > > > > processing the BroadcastStream. For instance, in the current
> > > > > > `KeyedBroadcastProcessFunction`, we use Context#applyToKeyedState
> > to
> > > > > > produce output results, which can be keyed in the same manner as
> > the
> > > > > > keyed input stream, upon processing data from the
> BroadcastStream.
> > > > > > Therefore, I believe it only requires that 

Re: [DISCUSS] FLIP-410: Config, Context and Processing Timer Service of DataStream API V2

2024-02-19 Thread weijie guo
Hi All,

Thanks for all the feedback.

If there are no more comments, I would like to start the vote thread,
thanks again!

Best regards,

Weijie


Xintong Song  于2024年2月20日周二 14:17写道:

> LGTM
>
> Best,
>
> Xintong
>
>
>
> On Mon, Feb 19, 2024 at 10:48 AM weijie guo 
> wrote:
>
> > Hi All,
> >
> > Based on the discussion thread of FLIP-409, I did a synchronized update
> to
> > this one. In simple terms, added `TwoInputBroadcastStreamProcessFunction`
> > related content.
> >
> >
> > Best regards,
> >
> > Weijie
> >
> >
> > weijie guo  于2024年1月31日周三 15:00写道:
> >
> > > Hi Xintong,
> > >
> > > Thanks for the quick reply.
> > >
> > > > Why introduce a new `MetricManager` rather than just return
> > `MetricGroup`
> > > from `RuntimeContext`?
> > >
> > > This is to facilitate possible future extensions. But I thought it
> > > through, MetricGroup itself also plays the role of a manager.
> > > So I think you are right, I will add a `getMetricGroup` method directly
> > in
> > > `RuntimeContext`.
> > >
> > > Best regards,
> > >
> > > Weijie
> > >
> > >
> > > Xintong Song  于2024年1月31日周三 14:02写道:
> > >
> > >> >
> > >> > > How can users define custom metrics within the `ProcessFunction`?
> > >> > Will there be a method like `getMetricGroup` available in the
> > >> > `RuntimeContext`?
> > >> >
> > >> > I think this is a reasonable request. For extensibility, I have
> added
> > >> the
> > >> > getMetricManager instead of getMetricGroup to RuntimeContext, we can
> > >> use it
> > >> > to get the MetricGroup.
> > >> >
> > >>
> > >> Why introduce a new `MetricManager` rather than just return
> > `MetricGroup`
> > >> from `RuntimeContext`?
> > >>
> > >> > Q2. The FLIP describes the interface for handling processing
> > >> >  timers (ProcessingTimeManager), but it does not mention
> > >> > how to delete or update an existing timer. V1 API provides
> TimeService
> > >> > that could delete a timer. Does this mean that
> > >> >  once a timer is registered, it cannot be changed?
> > >> >
> > >> > I think we do need to introduce a method to delete the timer, but
> I'm
> > >> kind
> > >> > of curious why we need to update the timer instead of registering a
> > new
> > >> > one. Anyway, I have updated the FLIP to support delete the timer.
> > >> >
> > >>
> > >> Registering a new timer does not mean the old timer should be removed.
> > >> There could be multiple timers.
> > >>
> > >> If we don't support deleting timers, developers can still decide to do
> > >> nothing upon the timer is triggered. In that case, they will need
> > >> additional logic to decide whether the timer should be skipped or not
> in
> > >> `onProcessingTimer`. Besides, there could also be additional
> performance
> > >> overhead in frequent calling and skipping the callback.
> > >>
> > >> Best,
> > >>
> > >> Xintong
> > >>
> > >>
> > >>
> > >> On Tue, Jan 30, 2024 at 3:26 PM weijie guo  >
> > >> wrote:
> > >>
> > >> > Hi Wencong,
> > >> >
> > >> > > Q1. In the "Configuration" section, it is mentioned that
> > >> > configurations can be set continuously using the withXXX methods.
> > >> > Are these configuration options the same as those provided by
> > DataStream
> > >> > V1,
> > >> > or might there be different options compared to V1?
> > >> >
> > >> > I haven't considered options that don't exist in V1 yet, but we may
> > have
> > >> > some new options as we continue to develop.
> > >> >
> > >> > > Q2. The FLIP describes the interface for handling processing
> > >> >  timers (ProcessingTimeManager), but it does not mention
> > >> > how to delete or update an existing timer. V1 API provides
> TimeService
> > >> > that could delete a timer. Does this mean that
> > >> >  once a timer is registered, it cannot be changed?
> > >> >
> > >> > I think we do need to introduce a method to delete the timer, but
> I'm
> > >> kind
> > >> > of curious why we need to update the timer instead of registering a
> > new
> > >> > one. Anyway, I have updated the FLIP to support delete the timer.
> > >> >
> > >> >
> > >> >
> > >> > Best regards,
> > >> >
> > >> > Weijie
> > >> >
> > >> >
> > >> > weijie guo  于2024年1月30日周二 14:35写道:
> > >> >
> > >> > > Hi Xuannan,
> > >> > >
> > >> > > > 1. +1 to only use XXXParititionStream if users only need to use
> > the
> > >> > > configurable PartitionStream.  If there are use cases for both,
> > >> > > perhaps we could use `ProcessConfigurableNonKeyedPartitionStream`
> or
> > >> > > `ConfigurableNonKeyedPartitionStream` for simplicity.
> > >> > >
> > >> > > As for why we need both, you can refer to my reply to Yunfeng's
> > first
> > >> > > question. As for the name, I can accept
> > >> > > ProcessConfigurableNonKeyedPartitionStream or keep the status quo.
> > >> But I
> > >> > > don't want to change it to ConfigurableNonKeyedPartitionStream,
> the
> > >> > reason
> > >> > > is the same, because the configuration is applied to the Process
> > >> rather
> > >> > > than the Stream.
> > >> > >
> > >> > > > Should we allow users to set custom 

Re: [VOTE] Release flink-connector-parent 1.1.0 release candidate #2

2024-02-19 Thread Leonard Xu
+1 (binding)

- verified signatures
- verified hashsums
- built from source code succeeded
- checked Github release tag 
- checked release notes
- reviewed all Jira tickets have been resolved
- reviewed the web PR

Best,
Leonard


> 2024年2月20日 上午11:14,Rui Fan <1996fan...@gmail.com> 写道:
> 
> Thanks for driving this, Etienne!
> 
> +1 (non-binding)
> 
> - Verified checksum and signature
> - Verified pom content
> - Build source on my Mac with jdk8
> - Verified no binaries in source
> - Checked staging repo on Maven central
> - Checked source code tag
> - Reviewed web PR
> 
> Best,
> Rui
> 
> On Tue, Feb 20, 2024 at 10:33 AM Qingsheng Ren  wrote:
> 
>> Thanks for driving this, Etienne!
>> 
>> +1 (binding)
>> 
>> - Checked release note
>> - Verified checksum and signature
>> - Verified pom content
>> - Verified no binaries in source
>> - Checked staging repo on Maven central
>> - Checked source code tag
>> - Reviewed web PR
>> - Built Kafka connector from source with parent pom in staging repo
>> 
>> Best,
>> Qingsheng
>> 
>> On Tue, Feb 20, 2024 at 1:34 AM Etienne Chauchot 
>> wrote:
>> 
>>> Hi everyone,
>>> Please review and vote on the release candidate #2 for the version
>>> 1.1.0, as follows:
>>> [ ] +1, Approve the release
>>> [ ] -1, Do not approve the release (please provide specific comments)
>>> 
>>> 
>>> The complete staging area is available for your review, which includes:
>>> * JIRA release notes [1],
>>> * the official Apache source release to be deployed to dist.apache.org
>>> [2], which are signed with the key with fingerprint
>>> D1A76BA19D6294DD0033F6843A019F0B8DD163EA [3],
>>> * all artifacts to be deployed to the Maven Central Repository [4],
>>> * source code tag v1.1.0-rc2 [5],
>>> * website pull request listing the new release [6].
>>> 
>>> * confluence wiki: connector parent upgrade to version 1.1.0 that will
>>> be validated after the artifact is released (there is no PR mechanism on
>>> the wiki) [7]
>>> 
>>> 
>>> The vote will be open for at least 72 hours. It is adopted by majority
>>> approval, with at least 3 PMC affirmative votes.
>>> 
>>> Thanks,
>>> Etienne
>>> 
>>> [1]
>>> 
>>> 
>> https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315522=12353442
>>> [2]
>>> 
>>> 
>> https://dist.apache.org/repos/dist/dev/flink/flink-connector-parent-1.1.0-rc2
>>> [3] https://dist.apache.org/repos/dist/release/flink/KEYS
>>> [4]
>> https://repository.apache.org/content/repositories/orgapacheflink-1707
>>> [5]
>>> 
>>> 
>> https://github.com/apache/flink-connector-shared-utils/releases/tag/v1.1.0-rc2
>>> 
>>> [6] https://github.com/apache/flink-web/pull/717
>>> 
>>> [7]
>>> 
>>> 
>> https://cwiki.apache.org/confluence/display/FLINK/Externalized+Connector+development
>>> 
>> 



Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread Runkang He
Congratulations Jiabao!

Best,
Runkang He

Jane Chan  于2024年2月20日周二 14:18写道:

> Congrats, Jiabao!
>
> Best,
> Jane
>
> On Tue, Feb 20, 2024 at 10:32 AM Paul Lam  wrote:
>
> > Congrats, Jiabao!
> >
> > Best,
> > Paul Lam
> >
> > > 2024年2月20日 10:29,Zakelly Lan  写道:
> > >
> > >> Congrats! Jiabao!
> >
> >
>


Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread Jane Chan
Congrats, Jiabao!

Best,
Jane

On Tue, Feb 20, 2024 at 10:32 AM Paul Lam  wrote:

> Congrats, Jiabao!
>
> Best,
> Paul Lam
>
> > 2024年2月20日 10:29,Zakelly Lan  写道:
> >
> >> Congrats! Jiabao!
>
>


Re: [DISCUSS] FLIP-410: Config, Context and Processing Timer Service of DataStream API V2

2024-02-19 Thread Xintong Song
LGTM

Best,

Xintong



On Mon, Feb 19, 2024 at 10:48 AM weijie guo 
wrote:

> Hi All,
>
> Based on the discussion thread of FLIP-409, I did a synchronized update to
> this one. In simple terms, added `TwoInputBroadcastStreamProcessFunction`
> related content.
>
>
> Best regards,
>
> Weijie
>
>
> weijie guo  于2024年1月31日周三 15:00写道:
>
> > Hi Xintong,
> >
> > Thanks for the quick reply.
> >
> > > Why introduce a new `MetricManager` rather than just return
> `MetricGroup`
> > from `RuntimeContext`?
> >
> > This is to facilitate possible future extensions. But I thought it
> > through, MetricGroup itself also plays the role of a manager.
> > So I think you are right, I will add a `getMetricGroup` method directly
> in
> > `RuntimeContext`.
> >
> > Best regards,
> >
> > Weijie
> >
> >
> > Xintong Song  于2024年1月31日周三 14:02写道:
> >
> >> >
> >> > > How can users define custom metrics within the `ProcessFunction`?
> >> > Will there be a method like `getMetricGroup` available in the
> >> > `RuntimeContext`?
> >> >
> >> > I think this is a reasonable request. For extensibility, I have added
> >> the
> >> > getMetricManager instead of getMetricGroup to RuntimeContext, we can
> >> use it
> >> > to get the MetricGroup.
> >> >
> >>
> >> Why introduce a new `MetricManager` rather than just return
> `MetricGroup`
> >> from `RuntimeContext`?
> >>
> >> > Q2. The FLIP describes the interface for handling processing
> >> >  timers (ProcessingTimeManager), but it does not mention
> >> > how to delete or update an existing timer. V1 API provides TimeService
> >> > that could delete a timer. Does this mean that
> >> >  once a timer is registered, it cannot be changed?
> >> >
> >> > I think we do need to introduce a method to delete the timer, but I'm
> >> kind
> >> > of curious why we need to update the timer instead of registering a
> new
> >> > one. Anyway, I have updated the FLIP to support delete the timer.
> >> >
> >>
> >> Registering a new timer does not mean the old timer should be removed.
> >> There could be multiple timers.
> >>
> >> If we don't support deleting timers, developers can still decide to do
> >> nothing upon the timer is triggered. In that case, they will need
> >> additional logic to decide whether the timer should be skipped or not in
> >> `onProcessingTimer`. Besides, there could also be additional performance
> >> overhead in frequent calling and skipping the callback.
> >>
> >> Best,
> >>
> >> Xintong
> >>
> >>
> >>
> >> On Tue, Jan 30, 2024 at 3:26 PM weijie guo 
> >> wrote:
> >>
> >> > Hi Wencong,
> >> >
> >> > > Q1. In the "Configuration" section, it is mentioned that
> >> > configurations can be set continuously using the withXXX methods.
> >> > Are these configuration options the same as those provided by
> DataStream
> >> > V1,
> >> > or might there be different options compared to V1?
> >> >
> >> > I haven't considered options that don't exist in V1 yet, but we may
> have
> >> > some new options as we continue to develop.
> >> >
> >> > > Q2. The FLIP describes the interface for handling processing
> >> >  timers (ProcessingTimeManager), but it does not mention
> >> > how to delete or update an existing timer. V1 API provides TimeService
> >> > that could delete a timer. Does this mean that
> >> >  once a timer is registered, it cannot be changed?
> >> >
> >> > I think we do need to introduce a method to delete the timer, but I'm
> >> kind
> >> > of curious why we need to update the timer instead of registering a
> new
> >> > one. Anyway, I have updated the FLIP to support delete the timer.
> >> >
> >> >
> >> >
> >> > Best regards,
> >> >
> >> > Weijie
> >> >
> >> >
> >> > weijie guo  于2024年1月30日周二 14:35写道:
> >> >
> >> > > Hi Xuannan,
> >> > >
> >> > > > 1. +1 to only use XXXParititionStream if users only need to use
> the
> >> > > configurable PartitionStream.  If there are use cases for both,
> >> > > perhaps we could use `ProcessConfigurableNonKeyedPartitionStream` or
> >> > > `ConfigurableNonKeyedPartitionStream` for simplicity.
> >> > >
> >> > > As for why we need both, you can refer to my reply to Yunfeng's
> first
> >> > > question. As for the name, I can accept
> >> > > ProcessConfigurableNonKeyedPartitionStream or keep the status quo.
> >> But I
> >> > > don't want to change it to ConfigurableNonKeyedPartitionStream, the
> >> > reason
> >> > > is the same, because the configuration is applied to the Process
> >> rather
> >> > > than the Stream.
> >> > >
> >> > > > Should we allow users to set custom configurations through the
> >> > > `ProcessConfigurable` interface and access these configurations in
> the
> >> > > `ProcessFunction` via `RuntimeContext`? I believe it would be useful
> >> > > for process function developers to be able to define custom
> >> > > configurations.
> >> > >
> >> > > If I understand you correctly, you want to set custom properties for
> >> > > processing. The current configurations are mostly for the runtime
> >> engine,
> >> > > such as determining the underlying 

Re: [DISCUSS] FLIP-409: DataStream V2 Building Blocks: DataStream, Partitioning and ProcessFunction

2024-02-19 Thread Xintong Song
Thanks for the updates. LGTM.

Best,

Xintong



On Mon, Feb 19, 2024 at 10:51 AM weijie guo 
wrote:

> Thanks for the reply, Xintong.
>
> Based on your comments, I made the following changes to this FLIP:
>
> 1. Renaming `TwoInputStreamProcessFunction` and
> `BroadcastTwoInputStreamProcessFunction` to
> `TwoInputNonBroadcastStreamProcessFunction` and
> `TwoInputBroadcastStreamProcessFunction`, respectively.
>
> 2. Making `NonPartitionedContext` extend `RuntimeContext`.
>
> > Some of these changes also affect FLIP-410. I noticed that FLIP-410 is
> also updated accordingly. It would be nice to also mention those changes in
> the FLIP-410 discussion thread.
>
> Yes, I've now mentioned those updates in the FLIP-410 discussion thread.
>
>
> Best regards,
>
> Weijie
>
>
> Xintong Song  于2024年2月5日周一 10:58写道:
>
> > Thanks for updating the FLIP, Weijie.
> >
> > I think separating the TwoInputProcessFunction according to whether the
> > input stream contains BroadcastStream makes sense.
> >
> > I have a few more comments.
> > 1. I'd suggest the names `TwoInputNonBroadcastStreamProcessFunction` and
> > `TwoInputBroadcastStreamProcessFunction` for the separated methods.
> > 2. I'd suggest making `NonPartitionedContext` extend `RuntimeContext`.
> > Otherwise, for all the functionalities that `RuntimeContext` provides, we
> > need to duplicate them for `NonPartitionedContext`.
> > 3. Some of these changes also affect FLIP-410. I noticed that FLIP-410 is
> > also updated accordingly. It would be nice to also mention those changes
> in
> > the FLIP-410 discussion thread.
> >
> > Best,
> >
> > Xintong
> >
> >
> >
> > On Sun, Feb 4, 2024 at 11:23 AM weijie guo 
> > wrote:
> >
> > > Hi Xuannan and Xintong,
> > >
> > > Good point! After further consideration, I feel that we should make the
> > > Broadcast + NonKeyed/Keyed process function different from the normal
> > > TwoInputProcessFunction. Because the record from the broadcast input
> > indeed
> > > correspond to all partitions, while the record from the non-broadcast
> > edge
> > > have explicit partitions.
> > >
> > > When we consider the data of broadcast input, it is only valid to do
> > > something on all the partitions at once, such as things like
> > > `applyToKeyedState`. Similarly, other operations(e.g, endOfInput) that
> do
> > > not determine the current partition should also only be allowed to
> > perform
> > > on all partitions. This FLIP has been updated.
> > >
> > > Best regards,
> > >
> > > Weijie
> > >
> > >
> > > Xintong Song  于2024年2月1日周四 11:31写道:
> > >
> > > > OK, I see your point.
> > > >
> > > > I think the demand for updating states and emitting outputs upon
> > > receiving
> > > > a broadcast record makes sense. However, the way
> > > > `KeyedBroadcastProcessFunction` supports this may not be optimal.
> E.g.,
> > > if
> > > > `Collector#collect` is called in `processBroadcastElement` but
> outside
> > of
> > > > `Context#applyToKeyedState`, the result can be undefined.
> > > >
> > > > Currently in this FLIP, a `TwoInputStreamProcessFunction` is not
> aware
> > of
> > > > which input is KeyedStream and which is BroadcastStream, which makes
> > > > supporting things like `applyToKeyedState` difficult. I think we can
> > > > provide a built-in function similar to
> `KeyedBroadcastProcessFunction`
> > on
> > > > top of `TwoInputStreamProcessFunction` to address this demand.
> > > >
> > > > WDYT?
> > > >
> > > >
> > > > Best,
> > > >
> > > > Xintong
> > > >
> > > >
> > > >
> > > > On Thu, Feb 1, 2024 at 10:41 AM Xuannan Su 
> > > wrote:
> > > >
> > > > > Hi Weijie and Xingtong,
> > > > >
> > > > > Thanks for the reply! Please see my comments below.
> > > > >
> > > > > > Does this mean if we want to support (KeyedStream,
> BroadcastStream)
> > > ->
> > > > > > (KeyedStream), we must make sure that no data can be output upon
> > > > > processing
> > > > > > records from the input BroadcastStream? That's probably a
> > reasonable
> > > > > > limitation.
> > > > >
> > > > > I don't think that the requirement for supporting (KeyedStream,
> > > > > BroadcastStream) -> (KeyedStream) is that no data can be output
> upon
> > > > > processing the BroadcastStream. For instance, in the current
> > > > > `KeyedBroadcastProcessFunction`, we use Context#applyToKeyedState
> to
> > > > > produce output results, which can be keyed in the same manner as
> the
> > > > > keyed input stream, upon processing data from the BroadcastStream.
> > > > > Therefore, I believe it only requires that the user must ensure
> that
> > > > > the output is keyed in the same way as the input, in this case, the
> > > > > same way as the keyed input stream. I think this requirement is
> > > > > consistent with that of (KeyedStream, KeyedStream) ->
> (KeyedStream).
> > > > > Thus, I believe that supporting (KeyedStream, BroadcastStream) ->
> > > > > (KeyedStream) will not introduce complexity for the users. WDYT?
> > > > >
> > > > > Best regards,
> > > > > Xuannan
> > > > >
> > > > >
> > > > 

Re: Making CollectSinkFunction to wait till all results are consumed

2024-02-19 Thread Alexey Leonov-Vendrovskiy
Hey, Piotrek,

Thanks for the response and thoughts!

I don't think it is possible to tell the difference between different cases
-- when someone would not want the results back from the instantiated
CollectSinkFunction.
It is not clear why this would be the case, unless it is some test flow
that explicitly does unusual things.

Regards,
-Alexey


On Wed, Feb 14, 2024 at 12:27 AM Piotr Nowojski 
wrote:

> Hi!
>
> Interesting catch. I think the current master branch behaviour is broken.
> The chance to lose some records on `endInput`
> is simply a critical bug. The limited buffer size is still problematic, as
> it can surprise users. Having said that, the newly
> proposed behaviour without any buffer is also problematic.
>
> Maybe whenever the user wants to collect the results, before or when
> calling `env.execute()` we should spawn a new
> thread that would asynchronously collect results from the
> `CollectSinkFunction`? I'm not sure but maybe hooking this
> logic up to whenever `CollectStreamSink` is being used is the way to go?
>
> One thing I'm not sure about is whether there are scenarios/existing code
> paths where someone wants to use the
> `CollectSinkFunction` but doesn't want the results to be read
> automatically? And if there are, can we tell them apart?
>
> Best,
> Piotrek
>
> pon., 12 lut 2024 o 08:48 Alexey Leonov-Vendrovskiy 
> napisał(a):
>
> > Hey all,
> >
> > We propose to slightly change the behavior of the CollectSinkFunction
> > <
> >
> https://github.com/apache/flink/blob/6f4d31f1b79afbde6c093b5d40ac83fe7524e303/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkFunction.java#L129
> > >
> > and make it wait till all the result from the buffer is consumed by the
> > client, before shutting it down.
> >
> > Overall protocol and all the other behavior stay the same.
> >
> > This would be a way to guarantee result availability upon the job
> > completion. Today, the tail of the result is stored in an accumulator,
> and
> > gets stored in the job manager. There is an opportunity for this part of
> > the result to get lost, after the job is claimed to be
> > successfully "completed". Waiting till all the results are consumed while
> > the job is running is a natural way to achieve availability. Once the job
> > is done, we are certain all the results are consumed.
> >
> > This change would be achieved by overriding the endInput() method
> > in CollectSinkOperator, and passing the call to CollectSinkFunction to
> wait
> > till the buffer is empty.
> >
> > The old behavior could be enabled via a configuration flag (to be added).
> >
> > A notable side-effect of the change is that any invocation
> > of StreamExecutionEnvironment.execute() (synchronous execution) with a
> > pipeline with CollectSinkFunction in it, would effectively block waiting
> > for the results to get consumed. This would require running the consumer
> on
> > a different thread. Though note, it is* already the case* when the result
> > is larger that what can fit into the CollectSinkFunction's buffer.  Take
> a
> > look at flink-end-to-end-tests/test-scripts/test_quickstarts.sh in the
> > current state of the repo: if we change the parameter numRecords to be
> > 1,000,000, the test locks and waits forever. So, the only difference with
> > the change would be that in similar setups it would wait on any buffer
> size
> > > 0. It makes behavior consistent for results of any non-zero size.
> >
> >
> > Let me know your thoughts.
> >
> > Thanks,
> > Alexey
> >
>


Re: [VOTE] Release flink-connector-parent 1.1.0 release candidate #2

2024-02-19 Thread Rui Fan
Thanks for driving this, Etienne!

+1 (non-binding)

- Verified checksum and signature
- Verified pom content
- Build source on my Mac with jdk8
- Verified no binaries in source
- Checked staging repo on Maven central
- Checked source code tag
- Reviewed web PR

Best,
Rui

On Tue, Feb 20, 2024 at 10:33 AM Qingsheng Ren  wrote:

> Thanks for driving this, Etienne!
>
> +1 (binding)
>
> - Checked release note
> - Verified checksum and signature
> - Verified pom content
> - Verified no binaries in source
> - Checked staging repo on Maven central
> - Checked source code tag
> - Reviewed web PR
> - Built Kafka connector from source with parent pom in staging repo
>
> Best,
> Qingsheng
>
> On Tue, Feb 20, 2024 at 1:34 AM Etienne Chauchot 
> wrote:
>
> > Hi everyone,
> > Please review and vote on the release candidate #2 for the version
> > 1.1.0, as follows:
> > [ ] +1, Approve the release
> > [ ] -1, Do not approve the release (please provide specific comments)
> >
> >
> > The complete staging area is available for your review, which includes:
> > * JIRA release notes [1],
> > * the official Apache source release to be deployed to dist.apache.org
> > [2], which are signed with the key with fingerprint
> > D1A76BA19D6294DD0033F6843A019F0B8DD163EA [3],
> > * all artifacts to be deployed to the Maven Central Repository [4],
> > * source code tag v1.1.0-rc2 [5],
> > * website pull request listing the new release [6].
> >
> > * confluence wiki: connector parent upgrade to version 1.1.0 that will
> > be validated after the artifact is released (there is no PR mechanism on
> > the wiki) [7]
> >
> >
> > The vote will be open for at least 72 hours. It is adopted by majority
> > approval, with at least 3 PMC affirmative votes.
> >
> > Thanks,
> > Etienne
> >
> > [1]
> >
> >
> https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315522=12353442
> > [2]
> >
> >
> https://dist.apache.org/repos/dist/dev/flink/flink-connector-parent-1.1.0-rc2
> > [3] https://dist.apache.org/repos/dist/release/flink/KEYS
> > [4]
> https://repository.apache.org/content/repositories/orgapacheflink-1707
> > [5]
> >
> >
> https://github.com/apache/flink-connector-shared-utils/releases/tag/v1.1.0-rc2
> >
> > [6] https://github.com/apache/flink-web/pull/717
> >
> > [7]
> >
> >
> https://cwiki.apache.org/confluence/display/FLINK/Externalized+Connector+development
> >
>


Re: [VOTE] Release flink-connector-parent 1.1.0 release candidate #2

2024-02-19 Thread Qingsheng Ren
Thanks for driving this, Etienne!

+1 (binding)

- Checked release note
- Verified checksum and signature
- Verified pom content
- Verified no binaries in source
- Checked staging repo on Maven central
- Checked source code tag
- Reviewed web PR
- Built Kafka connector from source with parent pom in staging repo

Best,
Qingsheng

On Tue, Feb 20, 2024 at 1:34 AM Etienne Chauchot 
wrote:

> Hi everyone,
> Please review and vote on the release candidate #2 for the version
> 1.1.0, as follows:
> [ ] +1, Approve the release
> [ ] -1, Do not approve the release (please provide specific comments)
>
>
> The complete staging area is available for your review, which includes:
> * JIRA release notes [1],
> * the official Apache source release to be deployed to dist.apache.org
> [2], which are signed with the key with fingerprint
> D1A76BA19D6294DD0033F6843A019F0B8DD163EA [3],
> * all artifacts to be deployed to the Maven Central Repository [4],
> * source code tag v1.1.0-rc2 [5],
> * website pull request listing the new release [6].
>
> * confluence wiki: connector parent upgrade to version 1.1.0 that will
> be validated after the artifact is released (there is no PR mechanism on
> the wiki) [7]
>
>
> The vote will be open for at least 72 hours. It is adopted by majority
> approval, with at least 3 PMC affirmative votes.
>
> Thanks,
> Etienne
>
> [1]
>
> https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315522=12353442
> [2]
>
> https://dist.apache.org/repos/dist/dev/flink/flink-connector-parent-1.1.0-rc2
> [3] https://dist.apache.org/repos/dist/release/flink/KEYS
> [4] https://repository.apache.org/content/repositories/orgapacheflink-1707
> [5]
>
> https://github.com/apache/flink-connector-shared-utils/releases/tag/v1.1.0-rc2
>
> [6] https://github.com/apache/flink-web/pull/717
>
> [7]
>
> https://cwiki.apache.org/confluence/display/FLINK/Externalized+Connector+development
>


Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread Paul Lam
Congrats, Jiabao!

Best,
Paul Lam

> 2024年2月20日 10:29,Zakelly Lan  写道:
> 
>> Congrats! Jiabao!



Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread Zakelly Lan
Congratulations, Jiabao!


Best,
Zakelly

On Mon, Feb 19, 2024 at 9:40 PM Jing Ge  wrote:

> Congrats! Jiabao!
>
> Best regards,
> Jing
>
> On Mon, Feb 19, 2024 at 2:38 PM Jeyhun Karimov 
> wrote:
>
> > Congratulations, Jiabao!
> > Well deserved!
> >
> > On Mon, Feb 19, 2024 at 2:26 PM gongzhongqiang <
> gongzhongqi...@apache.org>
> > wrote:
> >
> > > Congratulations, Jiabao!
> > >
> > > Best,
> > > Zhongqiang Gong
> > >
> > > Qingsheng Ren  于2024年2月19日周一 17:53写道:
> > >
> > > > Hi everyone,
> > > >
> > > > On behalf of the PMC, I'm happy to announce Jiabao Sun as a new Flink
> > > > Committer.
> > > >
> > > > Jiabao began contributing in August 2022 and has contributed 60+
> > commits
> > > > for Flink main repo and various connectors. His most notable
> > contribution
> > > > is being the core author and maintainer of MongoDB connector, which
> is
> > > > fully functional in DataStream and Table/SQL APIs. Jiabao is also the
> > > > author of FLIP-377 and the main contributor of JUnit 5 migration in
> > > runtime
> > > > and table planner modules.
> > > >
> > > > Beyond his technical contributions, Jiabao is an active member of our
> > > > community, participating in the mailing list and consistently
> > > volunteering
> > > > for release verifications and code reviews with enthusiasm.
> > > >
> > > > Please join me in congratulating Jiabao for becoming an Apache Flink
> > > > committer!
> > > >
> > > > Best,
> > > > Qingsheng (on behalf of the Flink PMC)
> > > >
> > >
> >
>


[VOTE] Release flink-connector-parent 1.1.0 release candidate #2

2024-02-19 Thread Etienne Chauchot

Hi everyone,
Please review and vote on the release candidate #2 for the version 
1.1.0, as follows:

[ ] +1, Approve the release
[ ] -1, Do not approve the release (please provide specific comments)


The complete staging area is available for your review, which includes:
* JIRA release notes [1],
* the official Apache source release to be deployed to dist.apache.org 
[2], which are signed with the key with fingerprint 
D1A76BA19D6294DD0033F6843A019F0B8DD163EA [3],

* all artifacts to be deployed to the Maven Central Repository [4],
* source code tag v1.1.0-rc2 [5],
* website pull request listing the new release [6].

* confluence wiki: connector parent upgrade to version 1.1.0 that will 
be validated after the artifact is released (there is no PR mechanism on 
the wiki) [7]



The vote will be open for at least 72 hours. It is adopted by majority 
approval, with at least 3 PMC affirmative votes.


Thanks,
Etienne

[1] 
https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315522=12353442
[2] 
https://dist.apache.org/repos/dist/dev/flink/flink-connector-parent-1.1.0-rc2

[3] https://dist.apache.org/repos/dist/release/flink/KEYS
[4] https://repository.apache.org/content/repositories/orgapacheflink-1707
[5] 
https://github.com/apache/flink-connector-shared-utils/releases/tag/v1.1.0-rc2


[6] https://github.com/apache/flink-web/pull/717

[7] 
https://cwiki.apache.org/confluence/display/FLINK/Externalized+Connector+development


[ANNOUNCE] Apache flink-connector-mongodb 1.1.0 released

2024-02-19 Thread Leonard Xu
The Apache Flink community is very happy to announce the release of Apache
flink-connector-mongodb 1.1.0. This release is compatible with Flink 1.17.x and 
1.18.x series.

Apache Flink® is an open-source stream processing framework for
distributed, high-performing, always-available, and accurate data streaming
applications.

The release is available for download at:
https://flink.apache.org/downloads.html

The full release notes are available in Jira:
https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315522=12353483

We would like to thank all contributors of the Apache Flink community who
made this release possible!

Best,
Leonard

[jira] [Created] (FLINK-34470) Transactional message + Table api kafka source with 'latest-offset' scan bound mode causes indefinitely hanging

2024-02-19 Thread dongwoo.kim (Jira)
dongwoo.kim created FLINK-34470:
---

 Summary: Transactional message + Table api kafka source with 
'latest-offset' scan bound mode causes indefinitely hanging
 Key: FLINK-34470
 URL: https://issues.apache.org/jira/browse/FLINK-34470
 Project: Flink
  Issue Type: Bug
  Components: Connectors / Kafka
Affects Versions: 1.17.1
Reporter: dongwoo.kim


h2. Summary  
Hi we have faced issue with transactional message and table api kafka source. 
If we configure *'scan.bounded.mode'* to *'latest-offset'* flink sql request 
timeouts after  hanging. We can always reproduce this unexpected behavior by 
following below steps.


h2.  How to reproduce
1. Deploy transactional producer and stop after producing certain amount of 
messages
2. Configure *'scan.bounded.mode'* to *'latest-offset'* and submit simple query 
such as count(*)
3. Flink sql job gets stucked and timeouts.

h2. Cause
Transaction producer always produces [control 
records|https://kafka.apache.org/documentation/#controlbatch] at the end of the 
transaction. And these controll messages are not polled by 
{*}consumer.poll(){*}. (It is filtered internally). In 
*KafkaPartitionSplitReader* code it finishes split only when 
*lastRecord.offset() >= stoppingOffset - 1* condition is met. This might work 
well with non transactional messages or streaming environment but in some batch 
use cases it causes unexpected hanging.

h2. Proposed solution
Adding *consumer.position(tp) >= stoppingOffset* condition to the if statement. 
By this KafkaPartitionSplitReader is available to finish the split even when 
the stopping offset is configured to control record's offset. 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread Jing Ge
Congrats! Jiabao!

Best regards,
Jing

On Mon, Feb 19, 2024 at 2:38 PM Jeyhun Karimov  wrote:

> Congratulations, Jiabao!
> Well deserved!
>
> On Mon, Feb 19, 2024 at 2:26 PM gongzhongqiang 
> wrote:
>
> > Congratulations, Jiabao!
> >
> > Best,
> > Zhongqiang Gong
> >
> > Qingsheng Ren  于2024年2月19日周一 17:53写道:
> >
> > > Hi everyone,
> > >
> > > On behalf of the PMC, I'm happy to announce Jiabao Sun as a new Flink
> > > Committer.
> > >
> > > Jiabao began contributing in August 2022 and has contributed 60+
> commits
> > > for Flink main repo and various connectors. His most notable
> contribution
> > > is being the core author and maintainer of MongoDB connector, which is
> > > fully functional in DataStream and Table/SQL APIs. Jiabao is also the
> > > author of FLIP-377 and the main contributor of JUnit 5 migration in
> > runtime
> > > and table planner modules.
> > >
> > > Beyond his technical contributions, Jiabao is an active member of our
> > > community, participating in the mailing list and consistently
> > volunteering
> > > for release verifications and code reviews with enthusiasm.
> > >
> > > Please join me in congratulating Jiabao for becoming an Apache Flink
> > > committer!
> > >
> > > Best,
> > > Qingsheng (on behalf of the Flink PMC)
> > >
> >
>


Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread Jeyhun Karimov
Congratulations, Jiabao!
Well deserved!

On Mon, Feb 19, 2024 at 2:26 PM gongzhongqiang 
wrote:

> Congratulations, Jiabao!
>
> Best,
> Zhongqiang Gong
>
> Qingsheng Ren  于2024年2月19日周一 17:53写道:
>
> > Hi everyone,
> >
> > On behalf of the PMC, I'm happy to announce Jiabao Sun as a new Flink
> > Committer.
> >
> > Jiabao began contributing in August 2022 and has contributed 60+ commits
> > for Flink main repo and various connectors. His most notable contribution
> > is being the core author and maintainer of MongoDB connector, which is
> > fully functional in DataStream and Table/SQL APIs. Jiabao is also the
> > author of FLIP-377 and the main contributor of JUnit 5 migration in
> runtime
> > and table planner modules.
> >
> > Beyond his technical contributions, Jiabao is an active member of our
> > community, participating in the mailing list and consistently
> volunteering
> > for release verifications and code reviews with enthusiasm.
> >
> > Please join me in congratulating Jiabao for becoming an Apache Flink
> > committer!
> >
> > Best,
> > Qingsheng (on behalf of the Flink PMC)
> >
>


Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread gongzhongqiang
Congratulations, Jiabao!

Best,
Zhongqiang Gong

Qingsheng Ren  于2024年2月19日周一 17:53写道:

> Hi everyone,
>
> On behalf of the PMC, I'm happy to announce Jiabao Sun as a new Flink
> Committer.
>
> Jiabao began contributing in August 2022 and has contributed 60+ commits
> for Flink main repo and various connectors. His most notable contribution
> is being the core author and maintainer of MongoDB connector, which is
> fully functional in DataStream and Table/SQL APIs. Jiabao is also the
> author of FLIP-377 and the main contributor of JUnit 5 migration in runtime
> and table planner modules.
>
> Beyond his technical contributions, Jiabao is an active member of our
> community, participating in the mailing list and consistently volunteering
> for release verifications and code reviews with enthusiasm.
>
> Please join me in congratulating Jiabao for becoming an Apache Flink
> committer!
>
> Best,
> Qingsheng (on behalf of the Flink PMC)
>


[jira] [Created] (FLINK-34469) Implement TableDistribution toString

2024-02-19 Thread Timo Walther (Jira)
Timo Walther created FLINK-34469:


 Summary: Implement TableDistribution toString
 Key: FLINK-34469
 URL: https://issues.apache.org/jira/browse/FLINK-34469
 Project: Flink
  Issue Type: Sub-task
  Components: Table SQL / API
Reporter: Timo Walther


The newly added TableDistribution misses a toString implementation.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread Jiabao Sun
Thanks, everyone! 
It is great to be part of such an active and collaborative community!

On 2024/02/19 11:37:28 Feng Jin wrote:
> Congratulations, Jiabao!
> 
> 
> Best,
> Feng
> 
> On Mon, Feb 19, 2024 at 7:35 PM Sergey Nuyanzin  wrote:
> 
> > Congratulations, Jiabao!
> >
> > On Mon, Feb 19, 2024 at 12:26 PM Yanquan Lv  wrote:
> >
> > > Congratulations, Jiabao.
> > >
> > > He Wang  于2024年2月19日周一 19:21写道:
> > >
> > > > Congrats, Jiabao!
> > > >
> > > > On Mon, Feb 19, 2024 at 7:19 PM Benchao Li 
> > wrote:
> > > >
> > > > > Congrats, Jiabao!
> > > > >
> > > > > Zhanghao Chen  于2024年2月19日周一 18:42写道:
> > > > > >
> > > > > > Congrats, Jiaba!
> > > > > >
> > > > > > Best,
> > > > > > Zhanghao Chen
> > > > > > 
> > > > > > From: Qingsheng Ren 
> > > > > > Sent: Monday, February 19, 2024 17:53
> > > > > > To: dev ; jiabao...@apache.org <
> > > > > jiabao...@apache.org>
> > > > > > Subject: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun
> > > > > >
> > > > > > Hi everyone,
> > > > > >
> > > > > > On behalf of the PMC, I'm happy to announce Jiabao Sun as a new
> > Flink
> > > > > > Committer.
> > > > > >
> > > > > > Jiabao began contributing in August 2022 and has contributed 60+
> > > > commits
> > > > > > for Flink main repo and various connectors. His most notable
> > > > contribution
> > > > > > is being the core author and maintainer of MongoDB connector, which
> > > is
> > > > > > fully functional in DataStream and Table/SQL APIs. Jiabao is also
> > the
> > > > > > author of FLIP-377 and the main contributor of JUnit 5 migration in
> > > > > runtime
> > > > > > and table planner modules.
> > > > > >
> > > > > > Beyond his technical contributions, Jiabao is an active member of
> > our
> > > > > > community, participating in the mailing list and consistently
> > > > > volunteering
> > > > > > for release verifications and code reviews with enthusiasm.
> > > > > >
> > > > > > Please join me in congratulating Jiabao for becoming an Apache
> > Flink
> > > > > > committer!
> > > > > >
> > > > > > Best,
> > > > > > Qingsheng (on behalf of the Flink PMC)
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > >
> > > > > Best,
> > > > > Benchao Li
> > > > >
> > > >
> > >
> >
> >
> > --
> > Best regards,
> > Sergey
> >
> 


Re: FW: RE: [DISCUSS] FLIP-314: Support Customized Job Lineage Listener

2024-02-19 Thread Martijn Visser
I'm a bit confused: did we add new interfaces after FLIP-314 was
accepted? If so, please move the new interfaces to a new FLIP and
start a separate vote. We can't retrospectively change an accepted
FLIP with new interfaces and a new vote.

On Mon, Feb 19, 2024 at 3:22 AM Yong Fang  wrote:
>
> Hi all,
>
> If there are no more feedbacks, I will start a vote for the new interfaces
> in the next day, thanks
>
> Best,
> Fang Yong
>
> On Thu, Feb 8, 2024 at 1:30 PM Yong Fang  wrote:
>
> > Hi devs,
> >
> > According to the online-discussion in FLINK-3127 [1] and
> > offline-discussion with Maciej Obuchowski and Zhenqiu Huang, we would like
> > to update the lineage vertex relevant interfaces in FLIP-314 [2] as follows:
> >
> > 1. Introduce `LineageDataset` which represents source and sink in
> > `LineageVertex`. The fields in `LineageDataset` are as follows:
> > /* Name for this particular dataset. */
> > String name;
> > /* Unique name for this dataset's storage, for example, url for jdbc
> > connector and location for lakehouse connector. */
> > String namespace;
> > /* Facets for the lineage vertex to describe the particular
> > information of dataset, such as schema and config. */
> > Map facets;
> >
> > 2. There may be multiple datasets in one `LineageVertex`, for example,
> > kafka source or hybrid source. So users can get dataset list from
> > `LineageVertex`:
> > /** Get datasets from the lineage vertex. */
> > List datasets();
> >
> > 3. There will be built in facets for config and schema. To describe
> > columns in table/sql jobs and datastream jobs, we introduce
> > `DatasetSchemaField`.
> > /** Builtin config facet for dataset. */
> > @PublicEvolving
> > public interface DatasetConfigFacet extends LineageDatasetFacet {
> > Map config();
> > }
> >
> > /** Field for schema in dataset. */
> > public interface DatasetSchemaField {
> > /** The name of the field. */
> > String name();
> > /** The type of the field. */
> > T type();
> > }
> >
> > Thanks for valuable inputs from @Maciej and @Zhenqiu. And looking forward
> > to your feedback, thanks
> >
> > Best,
> > Fang Yong
> >
> > On Mon, Sep 25, 2023 at 1:18 PM Shammon FY  wrote:
> >
> >> Hi David,
> >>
> >> Do you want the detailed topology for Flink job? You can get
> >> `JobDetailsInfo` in `RestCusterClient` with the submitted job id, it has
> >> `String jsonPlan`. You can parse the json plan to get all steps and
> >> relations between them in a Flink job. Hope this can help you, thanks!
> >>
> >> Best,
> >> Shammon FY
> >>
> >> On Tue, Sep 19, 2023 at 11:46 PM David Radley 
> >> wrote:
> >>
> >>> Hi there,
> >>> I am looking at the interfaces. If I am reading it correctly,there is
> >>> one relationship between the source and sink and this relationship
> >>> represents the operational lineage. Lineage is usually represented as 
> >>> asset
> >>> -> process - > asset – see for example
> >>> https://egeria-project.org/features/lineage-management/overview/#the-lineage-graph
> >>>
> >>> Maybe I am missing it, but it seems to be that it would be useful to
> >>> store the process in the lineage graph.
> >>>
> >>> It is useful to have the top level lineage as source -> Flink job ->
> >>> sink. Where the Flink job is the process, but also to have this asset ->
> >>> process -> asset pattern for each of the steps in the job. If this is
> >>> present, please could you point me to it,
> >>>
> >>>   Kind regards, David.
> >>>
> >>>
> >>>
> >>>
> >>>
> >>> From: David Radley 
> >>> Date: Tuesday, 19 September 2023 at 16:11
> >>> To: dev@flink.apache.org 
> >>> Subject: [EXTERNAL] RE: [DISCUSS] FLIP-314: Support Customized Job
> >>> Lineage Listener
> >>> Hi,
> >>> I notice that there is an experimental lineage integration for Flink
> >>> with OpenLineage https://openlineage.io/docs/integrations/flink  . I
> >>> think this feature would allow for a superior Flink OpenLineage 
> >>> integration,
> >>> Kind regards, David.
> >>>
> >>> From: XTransfer 
> >>> Date: Tuesday, 19 September 2023 at 15:47
> >>> To: dev@flink.apache.org 
> >>> Subject: [EXTERNAL] Re: [DISCUSS] FLIP-314: Support Customized Job
> >>> Lineage Listener
> >>> Thanks Shammon for this proposal.
> >>>
> >>> That’s helpful for collecting the lineage of Flink tasks.
> >>> Looking forward to its implementation.
> >>>
> >>> Best,
> >>> Jiabao
> >>>
> >>>
> >>> > 2023年9月18日 20:56,Leonard Xu  写道:
> >>> >
> >>> > Thanks Shammon for the informations, the comment makes the lifecycle
> >>> clearer.
> >>> > +1
> >>> >
> >>> >
> >>> > Best,
> >>> > Leonard
> >>> >
> >>> >
> >>> >> On Sep 18, 2023, at 7:54 PM, Shammon FY  wrote:
> >>> >>
> >>> >> Hi devs,
> >>> >>
> >>> >> After discussing with @Qingsheng, I fixed a minor issue of the
> >>> lineage lifecycle in `StreamExecutionEnvironment`. I have added the 
> >>> comment
> >>> to explain that the lineage information in `StreamExecutionEnvironment`
> 

[jira] [Created] (FLINK-34468) Implement Lineage Interface in Cassandra Connector

2024-02-19 Thread Zhenqiu Huang (Jira)
Zhenqiu Huang created FLINK-34468:
-

 Summary: Implement Lineage Interface in Cassandra Connector
 Key: FLINK-34468
 URL: https://issues.apache.org/jira/browse/FLINK-34468
 Project: Flink
  Issue Type: Sub-task
  Components: Connectors / Cassandra
Reporter: Zhenqiu Huang






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (FLINK-34467) Implement Lineage Interface in Jdbc Connector

2024-02-19 Thread Zhenqiu Huang (Jira)
Zhenqiu Huang created FLINK-34467:
-

 Summary: Implement Lineage Interface in Jdbc Connector
 Key: FLINK-34467
 URL: https://issues.apache.org/jira/browse/FLINK-34467
 Project: Flink
  Issue Type: Sub-task
  Components: Connectors / JDBC
Affects Versions: 1.19.0
Reporter: Zhenqiu Huang






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (FLINK-34466) Implement Lineage Interface in Kafka Connector

2024-02-19 Thread Zhenqiu Huang (Jira)
Zhenqiu Huang created FLINK-34466:
-

 Summary: Implement Lineage Interface in Kafka Connector
 Key: FLINK-34466
 URL: https://issues.apache.org/jira/browse/FLINK-34466
 Project: Flink
  Issue Type: Sub-task
  Components: Connectors / Kafka
Affects Versions: 1.19.0
Reporter: Zhenqiu Huang






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread Feng Jin
Congratulations, Jiabao!


Best,
Feng

On Mon, Feb 19, 2024 at 7:35 PM Sergey Nuyanzin  wrote:

> Congratulations, Jiabao!
>
> On Mon, Feb 19, 2024 at 12:26 PM Yanquan Lv  wrote:
>
> > Congratulations, Jiabao.
> >
> > He Wang  于2024年2月19日周一 19:21写道:
> >
> > > Congrats, Jiabao!
> > >
> > > On Mon, Feb 19, 2024 at 7:19 PM Benchao Li 
> wrote:
> > >
> > > > Congrats, Jiabao!
> > > >
> > > > Zhanghao Chen  于2024年2月19日周一 18:42写道:
> > > > >
> > > > > Congrats, Jiaba!
> > > > >
> > > > > Best,
> > > > > Zhanghao Chen
> > > > > 
> > > > > From: Qingsheng Ren 
> > > > > Sent: Monday, February 19, 2024 17:53
> > > > > To: dev ; jiabao...@apache.org <
> > > > jiabao...@apache.org>
> > > > > Subject: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun
> > > > >
> > > > > Hi everyone,
> > > > >
> > > > > On behalf of the PMC, I'm happy to announce Jiabao Sun as a new
> Flink
> > > > > Committer.
> > > > >
> > > > > Jiabao began contributing in August 2022 and has contributed 60+
> > > commits
> > > > > for Flink main repo and various connectors. His most notable
> > > contribution
> > > > > is being the core author and maintainer of MongoDB connector, which
> > is
> > > > > fully functional in DataStream and Table/SQL APIs. Jiabao is also
> the
> > > > > author of FLIP-377 and the main contributor of JUnit 5 migration in
> > > > runtime
> > > > > and table planner modules.
> > > > >
> > > > > Beyond his technical contributions, Jiabao is an active member of
> our
> > > > > community, participating in the mailing list and consistently
> > > > volunteering
> > > > > for release verifications and code reviews with enthusiasm.
> > > > >
> > > > > Please join me in congratulating Jiabao for becoming an Apache
> Flink
> > > > > committer!
> > > > >
> > > > > Best,
> > > > > Qingsheng (on behalf of the Flink PMC)
> > > >
> > > >
> > > >
> > > > --
> > > >
> > > > Best,
> > > > Benchao Li
> > > >
> > >
> >
>
>
> --
> Best regards,
> Sergey
>


Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread Sergey Nuyanzin
Congratulations, Jiabao!

On Mon, Feb 19, 2024 at 12:26 PM Yanquan Lv  wrote:

> Congratulations, Jiabao.
>
> He Wang  于2024年2月19日周一 19:21写道:
>
> > Congrats, Jiabao!
> >
> > On Mon, Feb 19, 2024 at 7:19 PM Benchao Li  wrote:
> >
> > > Congrats, Jiabao!
> > >
> > > Zhanghao Chen  于2024年2月19日周一 18:42写道:
> > > >
> > > > Congrats, Jiaba!
> > > >
> > > > Best,
> > > > Zhanghao Chen
> > > > 
> > > > From: Qingsheng Ren 
> > > > Sent: Monday, February 19, 2024 17:53
> > > > To: dev ; jiabao...@apache.org <
> > > jiabao...@apache.org>
> > > > Subject: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun
> > > >
> > > > Hi everyone,
> > > >
> > > > On behalf of the PMC, I'm happy to announce Jiabao Sun as a new Flink
> > > > Committer.
> > > >
> > > > Jiabao began contributing in August 2022 and has contributed 60+
> > commits
> > > > for Flink main repo and various connectors. His most notable
> > contribution
> > > > is being the core author and maintainer of MongoDB connector, which
> is
> > > > fully functional in DataStream and Table/SQL APIs. Jiabao is also the
> > > > author of FLIP-377 and the main contributor of JUnit 5 migration in
> > > runtime
> > > > and table planner modules.
> > > >
> > > > Beyond his technical contributions, Jiabao is an active member of our
> > > > community, participating in the mailing list and consistently
> > > volunteering
> > > > for release verifications and code reviews with enthusiasm.
> > > >
> > > > Please join me in congratulating Jiabao for becoming an Apache Flink
> > > > committer!
> > > >
> > > > Best,
> > > > Qingsheng (on behalf of the Flink PMC)
> > >
> > >
> > >
> > > --
> > >
> > > Best,
> > > Benchao Li
> > >
> >
>


-- 
Best regards,
Sergey


Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread Yanquan Lv
Congratulations, Jiabao.

He Wang  于2024年2月19日周一 19:21写道:

> Congrats, Jiabao!
>
> On Mon, Feb 19, 2024 at 7:19 PM Benchao Li  wrote:
>
> > Congrats, Jiabao!
> >
> > Zhanghao Chen  于2024年2月19日周一 18:42写道:
> > >
> > > Congrats, Jiaba!
> > >
> > > Best,
> > > Zhanghao Chen
> > > 
> > > From: Qingsheng Ren 
> > > Sent: Monday, February 19, 2024 17:53
> > > To: dev ; jiabao...@apache.org <
> > jiabao...@apache.org>
> > > Subject: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun
> > >
> > > Hi everyone,
> > >
> > > On behalf of the PMC, I'm happy to announce Jiabao Sun as a new Flink
> > > Committer.
> > >
> > > Jiabao began contributing in August 2022 and has contributed 60+
> commits
> > > for Flink main repo and various connectors. His most notable
> contribution
> > > is being the core author and maintainer of MongoDB connector, which is
> > > fully functional in DataStream and Table/SQL APIs. Jiabao is also the
> > > author of FLIP-377 and the main contributor of JUnit 5 migration in
> > runtime
> > > and table planner modules.
> > >
> > > Beyond his technical contributions, Jiabao is an active member of our
> > > community, participating in the mailing list and consistently
> > volunteering
> > > for release verifications and code reviews with enthusiasm.
> > >
> > > Please join me in congratulating Jiabao for becoming an Apache Flink
> > > committer!
> > >
> > > Best,
> > > Qingsheng (on behalf of the Flink PMC)
> >
> >
> >
> > --
> >
> > Best,
> > Benchao Li
> >
>


Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread Matthias Pohl
Congratulations, Jiabao!

On Mon, Feb 19, 2024 at 12:21 PM He Wang  wrote:

> Congrats, Jiabao!
>
> On Mon, Feb 19, 2024 at 7:19 PM Benchao Li  wrote:
>
> > Congrats, Jiabao!
> >
> > Zhanghao Chen  于2024年2月19日周一 18:42写道:
> > >
> > > Congrats, Jiaba!
> > >
> > > Best,
> > > Zhanghao Chen
> > > 
> > > From: Qingsheng Ren 
> > > Sent: Monday, February 19, 2024 17:53
> > > To: dev ; jiabao...@apache.org <
> > jiabao...@apache.org>
> > > Subject: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun
> > >
> > > Hi everyone,
> > >
> > > On behalf of the PMC, I'm happy to announce Jiabao Sun as a new Flink
> > > Committer.
> > >
> > > Jiabao began contributing in August 2022 and has contributed 60+
> commits
> > > for Flink main repo and various connectors. His most notable
> contribution
> > > is being the core author and maintainer of MongoDB connector, which is
> > > fully functional in DataStream and Table/SQL APIs. Jiabao is also the
> > > author of FLIP-377 and the main contributor of JUnit 5 migration in
> > runtime
> > > and table planner modules.
> > >
> > > Beyond his technical contributions, Jiabao is an active member of our
> > > community, participating in the mailing list and consistently
> > volunteering
> > > for release verifications and code reviews with enthusiasm.
> > >
> > > Please join me in congratulating Jiabao for becoming an Apache Flink
> > > committer!
> > >
> > > Best,
> > > Qingsheng (on behalf of the Flink PMC)
> >
> >
> >
> > --
> >
> > Best,
> > Benchao Li
> >
>


Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread He Wang
Congrats, Jiabao!

On Mon, Feb 19, 2024 at 7:19 PM Benchao Li  wrote:

> Congrats, Jiabao!
>
> Zhanghao Chen  于2024年2月19日周一 18:42写道:
> >
> > Congrats, Jiaba!
> >
> > Best,
> > Zhanghao Chen
> > 
> > From: Qingsheng Ren 
> > Sent: Monday, February 19, 2024 17:53
> > To: dev ; jiabao...@apache.org <
> jiabao...@apache.org>
> > Subject: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun
> >
> > Hi everyone,
> >
> > On behalf of the PMC, I'm happy to announce Jiabao Sun as a new Flink
> > Committer.
> >
> > Jiabao began contributing in August 2022 and has contributed 60+ commits
> > for Flink main repo and various connectors. His most notable contribution
> > is being the core author and maintainer of MongoDB connector, which is
> > fully functional in DataStream and Table/SQL APIs. Jiabao is also the
> > author of FLIP-377 and the main contributor of JUnit 5 migration in
> runtime
> > and table planner modules.
> >
> > Beyond his technical contributions, Jiabao is an active member of our
> > community, participating in the mailing list and consistently
> volunteering
> > for release verifications and code reviews with enthusiasm.
> >
> > Please join me in congratulating Jiabao for becoming an Apache Flink
> > committer!
> >
> > Best,
> > Qingsheng (on behalf of the Flink PMC)
>
>
>
> --
>
> Best,
> Benchao Li
>


Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread Benchao Li
Congrats, Jiabao!

Zhanghao Chen  于2024年2月19日周一 18:42写道:
>
> Congrats, Jiaba!
>
> Best,
> Zhanghao Chen
> 
> From: Qingsheng Ren 
> Sent: Monday, February 19, 2024 17:53
> To: dev ; jiabao...@apache.org 
> Subject: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun
>
> Hi everyone,
>
> On behalf of the PMC, I'm happy to announce Jiabao Sun as a new Flink
> Committer.
>
> Jiabao began contributing in August 2022 and has contributed 60+ commits
> for Flink main repo and various connectors. His most notable contribution
> is being the core author and maintainer of MongoDB connector, which is
> fully functional in DataStream and Table/SQL APIs. Jiabao is also the
> author of FLIP-377 and the main contributor of JUnit 5 migration in runtime
> and table planner modules.
>
> Beyond his technical contributions, Jiabao is an active member of our
> community, participating in the mailing list and consistently volunteering
> for release verifications and code reviews with enthusiasm.
>
> Please join me in congratulating Jiabao for becoming an Apache Flink
> committer!
>
> Best,
> Qingsheng (on behalf of the Flink PMC)



-- 

Best,
Benchao Li


Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread Zhanghao Chen
Congrats, Jiaba!

Best,
Zhanghao Chen

From: Qingsheng Ren 
Sent: Monday, February 19, 2024 17:53
To: dev ; jiabao...@apache.org 
Subject: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

Hi everyone,

On behalf of the PMC, I'm happy to announce Jiabao Sun as a new Flink
Committer.

Jiabao began contributing in August 2022 and has contributed 60+ commits
for Flink main repo and various connectors. His most notable contribution
is being the core author and maintainer of MongoDB connector, which is
fully functional in DataStream and Table/SQL APIs. Jiabao is also the
author of FLIP-377 and the main contributor of JUnit 5 migration in runtime
and table planner modules.

Beyond his technical contributions, Jiabao is an active member of our
community, participating in the mailing list and consistently volunteering
for release verifications and code reviews with enthusiasm.

Please join me in congratulating Jiabao for becoming an Apache Flink
committer!

Best,
Qingsheng (on behalf of the Flink PMC)


Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread Lincoln Lee
Congratulations Jiabao!

Best,
Lincoln Lee


Leonard Xu  于2024年2月19日周一 18:25写道:

> Congratulations, Jiabao! Well deserved.
>
>
> Best,
> Leonard
>
>
> > 2024年2月19日 下午6:21,David Radley  写道:
> >
> > Congratulations Jiabao!
> >
> > From: Swapnal Varma 
> > Date: Monday, 19 February 2024 at 10:14
> > To: dev@flink.apache.org 
> > Subject: [EXTERNAL] Re: [ANNOUNCE] New Apache Flink Committer - Jiabao
> Sun
> > Congratulations Jiabao!
> >
> > Best,
> > Swapnal
> >
> > On Mon, 19 Feb 2024, 15:37 weijie guo, 
> wrote:
> >
> >> Congratulations, Jiabao :)
> >>
> >> Best regards,
> >>
> >> Weijie
> >>
> >>
> >> Hang Ruan  于2024年2月19日周一 18:04写道:
> >>
> >>> Congratulations, Jiabao!
> >>>
> >>> Best,
> >>> Hang
> >>>
> >>> Qingsheng Ren  于2024年2月19日周一 17:53写道:
> >>>
>  Hi everyone,
> 
>  On behalf of the PMC, I'm happy to announce Jiabao Sun as a new Flink
>  Committer.
> 
>  Jiabao began contributing in August 2022 and has contributed 60+
> >> commits
>  for Flink main repo and various connectors. His most notable
> >> contribution
>  is being the core author and maintainer of MongoDB connector, which is
>  fully functional in DataStream and Table/SQL APIs. Jiabao is also the
>  author of FLIP-377 and the main contributor of JUnit 5 migration in
> >>> runtime
>  and table planner modules.
> 
>  Beyond his technical contributions, Jiabao is an active member of our
>  community, participating in the mailing list and consistently
> >>> volunteering
>  for release verifications and code reviews with enthusiasm.
> 
>  Please join me in congratulating Jiabao for becoming an Apache Flink
>  committer!
> 
>  Best,
>  Qingsheng (on behalf of the Flink PMC)
> 
> >>>
> >>
> >
> > Unless otherwise stated above:
> >
> > IBM United Kingdom Limited
> > Registered in England and Wales with number 741598
> > Registered office: PO Box 41, North Harbour, Portsmouth, Hants. PO6 3AU
>
>


Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread Leonard Xu
Congratulations, Jiabao! Well deserved.


Best,
Leonard


> 2024年2月19日 下午6:21,David Radley  写道:
> 
> Congratulations Jiabao!
> 
> From: Swapnal Varma 
> Date: Monday, 19 February 2024 at 10:14
> To: dev@flink.apache.org 
> Subject: [EXTERNAL] Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun
> Congratulations Jiabao!
> 
> Best,
> Swapnal
> 
> On Mon, 19 Feb 2024, 15:37 weijie guo,  wrote:
> 
>> Congratulations, Jiabao :)
>> 
>> Best regards,
>> 
>> Weijie
>> 
>> 
>> Hang Ruan  于2024年2月19日周一 18:04写道:
>> 
>>> Congratulations, Jiabao!
>>> 
>>> Best,
>>> Hang
>>> 
>>> Qingsheng Ren  于2024年2月19日周一 17:53写道:
>>> 
 Hi everyone,
 
 On behalf of the PMC, I'm happy to announce Jiabao Sun as a new Flink
 Committer.
 
 Jiabao began contributing in August 2022 and has contributed 60+
>> commits
 for Flink main repo and various connectors. His most notable
>> contribution
 is being the core author and maintainer of MongoDB connector, which is
 fully functional in DataStream and Table/SQL APIs. Jiabao is also the
 author of FLIP-377 and the main contributor of JUnit 5 migration in
>>> runtime
 and table planner modules.
 
 Beyond his technical contributions, Jiabao is an active member of our
 community, participating in the mailing list and consistently
>>> volunteering
 for release verifications and code reviews with enthusiasm.
 
 Please join me in congratulating Jiabao for becoming an Apache Flink
 committer!
 
 Best,
 Qingsheng (on behalf of the Flink PMC)
 
>>> 
>> 
> 
> Unless otherwise stated above:
> 
> IBM United Kingdom Limited
> Registered in England and Wales with number 741598
> Registered office: PO Box 41, North Harbour, Portsmouth, Hants. PO6 3AU



RE: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread David Radley
Congratulations Jiabao!

From: Swapnal Varma 
Date: Monday, 19 February 2024 at 10:14
To: dev@flink.apache.org 
Subject: [EXTERNAL] Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun
Congratulations Jiabao!

Best,
Swapnal

On Mon, 19 Feb 2024, 15:37 weijie guo,  wrote:

> Congratulations, Jiabao :)
>
> Best regards,
>
> Weijie
>
>
> Hang Ruan  于2024年2月19日周一 18:04写道:
>
> > Congratulations, Jiabao!
> >
> > Best,
> > Hang
> >
> > Qingsheng Ren  于2024年2月19日周一 17:53写道:
> >
> > > Hi everyone,
> > >
> > > On behalf of the PMC, I'm happy to announce Jiabao Sun as a new Flink
> > > Committer.
> > >
> > > Jiabao began contributing in August 2022 and has contributed 60+
> commits
> > > for Flink main repo and various connectors. His most notable
> contribution
> > > is being the core author and maintainer of MongoDB connector, which is
> > > fully functional in DataStream and Table/SQL APIs. Jiabao is also the
> > > author of FLIP-377 and the main contributor of JUnit 5 migration in
> > runtime
> > > and table planner modules.
> > >
> > > Beyond his technical contributions, Jiabao is an active member of our
> > > community, participating in the mailing list and consistently
> > volunteering
> > > for release verifications and code reviews with enthusiasm.
> > >
> > > Please join me in congratulating Jiabao for becoming an Apache Flink
> > > committer!
> > >
> > > Best,
> > > Qingsheng (on behalf of the Flink PMC)
> > >
> >
>

Unless otherwise stated above:

IBM United Kingdom Limited
Registered in England and Wales with number 741598
Registered office: PO Box 41, North Harbour, Portsmouth, Hants. PO6 3AU


Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread Swapnal Varma
Congratulations Jiabao!

Best,
Swapnal

On Mon, 19 Feb 2024, 15:37 weijie guo,  wrote:

> Congratulations, Jiabao :)
>
> Best regards,
>
> Weijie
>
>
> Hang Ruan  于2024年2月19日周一 18:04写道:
>
> > Congratulations, Jiabao!
> >
> > Best,
> > Hang
> >
> > Qingsheng Ren  于2024年2月19日周一 17:53写道:
> >
> > > Hi everyone,
> > >
> > > On behalf of the PMC, I'm happy to announce Jiabao Sun as a new Flink
> > > Committer.
> > >
> > > Jiabao began contributing in August 2022 and has contributed 60+
> commits
> > > for Flink main repo and various connectors. His most notable
> contribution
> > > is being the core author and maintainer of MongoDB connector, which is
> > > fully functional in DataStream and Table/SQL APIs. Jiabao is also the
> > > author of FLIP-377 and the main contributor of JUnit 5 migration in
> > runtime
> > > and table planner modules.
> > >
> > > Beyond his technical contributions, Jiabao is an active member of our
> > > community, participating in the mailing list and consistently
> > volunteering
> > > for release verifications and code reviews with enthusiasm.
> > >
> > > Please join me in congratulating Jiabao for becoming an Apache Flink
> > > committer!
> > >
> > > Best,
> > > Qingsheng (on behalf of the Flink PMC)
> > >
> >
>


Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread Rui Fan
Congratulations, Jiabao!

Best,
Rui

On Mon, Feb 19, 2024 at 6:07 PM weijie guo 
wrote:

> Congratulations, Jiabao :)
>
> Best regards,
>
> Weijie
>
>
> Hang Ruan  于2024年2月19日周一 18:04写道:
>
> > Congratulations, Jiabao!
> >
> > Best,
> > Hang
> >
> > Qingsheng Ren  于2024年2月19日周一 17:53写道:
> >
> > > Hi everyone,
> > >
> > > On behalf of the PMC, I'm happy to announce Jiabao Sun as a new Flink
> > > Committer.
> > >
> > > Jiabao began contributing in August 2022 and has contributed 60+
> commits
> > > for Flink main repo and various connectors. His most notable
> contribution
> > > is being the core author and maintainer of MongoDB connector, which is
> > > fully functional in DataStream and Table/SQL APIs. Jiabao is also the
> > > author of FLIP-377 and the main contributor of JUnit 5 migration in
> > runtime
> > > and table planner modules.
> > >
> > > Beyond his technical contributions, Jiabao is an active member of our
> > > community, participating in the mailing list and consistently
> > volunteering
> > > for release verifications and code reviews with enthusiasm.
> > >
> > > Please join me in congratulating Jiabao for becoming an Apache Flink
> > > committer!
> > >
> > > Best,
> > > Qingsheng (on behalf of the Flink PMC)
> > >
> >
>


Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread weijie guo
Congratulations, Jiabao :)

Best regards,

Weijie


Hang Ruan  于2024年2月19日周一 18:04写道:

> Congratulations, Jiabao!
>
> Best,
> Hang
>
> Qingsheng Ren  于2024年2月19日周一 17:53写道:
>
> > Hi everyone,
> >
> > On behalf of the PMC, I'm happy to announce Jiabao Sun as a new Flink
> > Committer.
> >
> > Jiabao began contributing in August 2022 and has contributed 60+ commits
> > for Flink main repo and various connectors. His most notable contribution
> > is being the core author and maintainer of MongoDB connector, which is
> > fully functional in DataStream and Table/SQL APIs. Jiabao is also the
> > author of FLIP-377 and the main contributor of JUnit 5 migration in
> runtime
> > and table planner modules.
> >
> > Beyond his technical contributions, Jiabao is an active member of our
> > community, participating in the mailing list and consistently
> volunteering
> > for release verifications and code reviews with enthusiasm.
> >
> > Please join me in congratulating Jiabao for becoming an Apache Flink
> > committer!
> >
> > Best,
> > Qingsheng (on behalf of the Flink PMC)
> >
>


Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread Hang Ruan
Congratulations, Jiabao!

Best,
Hang

Qingsheng Ren  于2024年2月19日周一 17:53写道:

> Hi everyone,
>
> On behalf of the PMC, I'm happy to announce Jiabao Sun as a new Flink
> Committer.
>
> Jiabao began contributing in August 2022 and has contributed 60+ commits
> for Flink main repo and various connectors. His most notable contribution
> is being the core author and maintainer of MongoDB connector, which is
> fully functional in DataStream and Table/SQL APIs. Jiabao is also the
> author of FLIP-377 and the main contributor of JUnit 5 migration in runtime
> and table planner modules.
>
> Beyond his technical contributions, Jiabao is an active member of our
> community, participating in the mailing list and consistently volunteering
> for release verifications and code reviews with enthusiasm.
>
> Please join me in congratulating Jiabao for becoming an Apache Flink
> committer!
>
> Best,
> Qingsheng (on behalf of the Flink PMC)
>


[jira] [Created] (FLINK-34465) Python py38-cython: commands failed, Bash exited with code '1

2024-02-19 Thread lincoln lee (Jira)
lincoln lee created FLINK-34465:
---

 Summary: Python py38-cython: commands failed, Bash exited with 
code '1
 Key: FLINK-34465
 URL: https://issues.apache.org/jira/browse/FLINK-34465
 Project: Flink
  Issue Type: Bug
  Components: API / Python
Affects Versions: 1.18.0
Reporter: Sergey Nuyanzin
Assignee: Huang Xingbo


{noformat}
Apr 06 05:55:13 ___ summary 

Apr 06 05:55:13 ERROR:   py37-cython: commands failed
Apr 06 05:55:13   py38-cython: commands succeeded
Apr 06 05:55:13   py39-cython: commands succeeded
Apr 06 05:55:13   py310-cython: commands succeeded
{noformat}
in logs there is such error for 37 not sure if it is related
{noformat}
Apr 06 04:26:30  ERROR at setup of 
ProcessWindowTests.test_count_sliding_window 
Apr 06 04:26:30 
Apr 06 04:26:30 cls = 
Apr 06 04:26:30 
Apr 06 04:26:30 @classmethod
Apr 06 04:26:30 def setUpClass(cls):
Apr 06 04:26:30 super(PyFlinkStreamingTestCase, cls).setUpClass()
Apr 06 04:26:30 cls.env.set_parallelism(2)
Apr 06 04:26:30 >   cls.env.set_runtime_mode(RuntimeExecutionMode.STREAMING)
Apr 06 04:26:30 
Apr 06 04:26:30 pyflink/testing/test_case_utils.py:193: 
Apr 06 04:26:30 _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
_ _ _ _ _ _ _ _ 
 {noformat}
[https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=47960=logs=821b528f-1eed-5598-a3b4-7f748b13f261=6bb545dd-772d-5d8c-f258-f5085fba3295=24593]



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread Qingsheng Ren
Hi everyone,

On behalf of the PMC, I'm happy to announce Jiabao Sun as a new Flink
Committer.

Jiabao began contributing in August 2022 and has contributed 60+ commits
for Flink main repo and various connectors. His most notable contribution
is being the core author and maintainer of MongoDB connector, which is
fully functional in DataStream and Table/SQL APIs. Jiabao is also the
author of FLIP-377 and the main contributor of JUnit 5 migration in runtime
and table planner modules.

Beyond his technical contributions, Jiabao is an active member of our
community, participating in the mailing list and consistently volunteering
for release verifications and code reviews with enthusiasm.

Please join me in congratulating Jiabao for becoming an Apache Flink
committer!

Best,
Qingsheng (on behalf of the Flink PMC)


[jira] [Created] (FLINK-34464) actions/cache@v4 times out

2024-02-19 Thread Matthias Pohl (Jira)
Matthias Pohl created FLINK-34464:
-

 Summary: actions/cache@v4 times out
 Key: FLINK-34464
 URL: https://issues.apache.org/jira/browse/FLINK-34464
 Project: Flink
  Issue Type: Bug
  Components: Build System / CI, Test Infrastructure
Reporter: Matthias Pohl


[https://github.com/apache/flink/actions/runs/7953599167/job/21710058433#step:4:125]

Pulling the docker image stalled. This should be a temporary issue:
{code:java}
/usr/bin/docker exec  
601a5a6e68acf3ba38940ec7a07e08d7c57e763ca0364070124f71bc2f708bc3 sh -c "cat 
/etc/*release | grep ^ID"
120Received 260046848 of 1429155280 (18.2%), 248.0 MBs/sec
121Received 545259520 of 1429155280 (38.2%), 260.0 MBs/sec
[...]
Received 914358272 of 1429155280 (64.0%), 0.0 MBs/sec
21645Received 914358272 of 1429155280 (64.0%), 0.0 MBs/sec
21646Error: The operation was canceled. {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (FLINK-34463) Open catalog in CatalogManager should use proper context classloader

2024-02-19 Thread jrthe42 (Jira)
jrthe42 created FLINK-34463:
---

 Summary: Open catalog in CatalogManager should use proper context 
classloader
 Key: FLINK-34463
 URL: https://issues.apache.org/jira/browse/FLINK-34463
 Project: Flink
  Issue Type: Bug
  Components: Table SQL / Runtime
Affects Versions: 1.18.1, 1.17.2
Reporter: jrthe42


When we try to create a catalog in CatalogManager, if the catalog jar is added 
using `ADD JAR` and the catalog itself requires SPI mechanism, the operation 
may fail.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[RESULT][VOTE] Release flink-connector-mongodb 1.1.0, release candidate #2

2024-02-19 Thread Leonard Xu
I'm happy to announce that we have unanimously approved this release.

There are 6 approving votes, 3 of which are binding:

* Jiabao Sun (non-binding) 
* gongzhongqiang (non-binding)
* Hang Ruan (non-binding)
* Danny Cranmer (binding)
* Martijn Visser (binding)
* Leonard Xu (binding)

There are no disapproving votes.

Thanks all! and I’ll complete the release soon and announce it soon after this 
email.

Best,
Leonard

Re: [VOTE] Release flink-connector-mongodb v1.1.0, release candidate #2

2024-02-19 Thread Leonard Xu
Thanks all for the voting, I’ll summarize the result in another mail.

Best,
Leonard


> 2024年2月19日 下午4:46,Leonard Xu  写道:
> 
> +1 (binding)
> 
> - built from source code succeeded
> - verified signatures
> - verified hashsums 
> - checked the contents contains jar and pom files in apache repo 
> - checked Github release tag 
> - checked release notes
> 
> Best,
> Leonard
> 
>> 2024年2月8日 下午11:37,Martijn Visser  写道:
>> 
>> +1 (binding)
>> 
>> - Validated hashes
>> - Verified signature
>> - Verified that no binaries exist in the source archive
>> - Build the source with Maven
>> - Verified licenses
>> - Verified web PRs
>> 
>> On Wed, Jan 31, 2024 at 10:41 AM Danny Cranmer  
>> wrote:
>>> 
>>> Thanks for driving this Leonard!
>>> 
>>> +1 (binding)
>>> 
>>> - Release notes look ok
>>> - Signatures/checksums of source archive are good
>>> - Verified there are no binaries in the source archive
>>> - Built sources locally successfully
>>> - v1.0.0-rc2 tag exists in github
>>> - Tag build passing on CI [1]
>>> - Contents of Maven dist look complete
>>> - Verified signatures/checksums of binary in maven dist is correct
>>> - Verified NOTICE files and bundled dependencies
>>> 
>>> Thanks,
>>> Danny
>>> 
>>> [1]
>>> https://github.com/apache/flink-connector-mongodb/actions/runs/7709467379
>>> 
>>> On Wed, Jan 31, 2024 at 7:54 AM gongzhongqiang 
>>> wrote:
>>> 
 +1(non-binding)
 
 - Signatures and Checksums are good
 - No binaries in the source archive
 - Tag is present
 - Build successful with jdk8 on ubuntu 22.04
 
 
 Leonard Xu  于2024年1月30日周二 18:23写道:
 
> Hey all,
> 
> Please help review and vote on the release candidate #2 for the version
> v1.1.0 of the
> Apache Flink MongoDB Connector as follows:
> 
> [ ] +1, Approve the release
> [ ] -1, Do not approve the release (please provide specific comments)
> 
> The complete staging area is available for your review, which includes:
> * JIRA release notes [1],
> * The official Apache source release to be deployed to dist.apache.org
> [2],
> which are signed with the key with fingerprint
> 5B2F6608732389AEB67331F5B197E1F1108998AD [3],
> * All artifacts to be deployed to the Maven Central Repository [4],
> * Source code tag v1.1.0-rc2 [5],
> * Website pull request listing the new release [6].
> 
> The vote will be open for at least 72 hours. It is adopted by majority
> approval, with at least 3 PMC affirmative votes.
> 
> 
> Best,
> Leonard
> [1]
> 
 https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315522=12353483
> [2]
> 
 https://dist.apache.org/repos/dist/dev/flink/flink-connector-mongodb-1.1.0-rc2/
> [3] https://dist.apache.org/repos/dist/release/flink/KEYS
> [4]
> https://repository.apache.org/content/repositories/orgapacheflink-1705/
> [5] https://github.com/apache/flink-connector-mongodb/tree/v1.1.0-rc2
> [6] https://github.com/apache/flink-web/pull/719
 
> 



[jira] [Created] (FLINK-34462) Session window with negative parameter throws unclear exception

2024-02-19 Thread Shuai Xu (Jira)
Shuai Xu created FLINK-34462:


 Summary: Session window with negative parameter throws unclear 
exception
 Key: FLINK-34462
 URL: https://issues.apache.org/jira/browse/FLINK-34462
 Project: Flink
  Issue Type: Improvement
  Components: Table SQL / Planner
Affects Versions: 1.19.0
Reporter: Shuai Xu


Set invalid parameter in session window get unclear error.
{code:java}
// add test in WindowAggregateITCase
def testEventTimeSessionWindowWithInvalidName(): Unit = {
  val sql =
"""
  |SELECT
  |  window_start,
  |  window_end,
  |  COUNT(*),
  |  SUM(`bigdec`),
  |  MAX(`double`),
  |  MIN(`float`),
  |  COUNT(DISTINCT `string`),
  |  concat_distinct_agg(`string`)
  |FROM TABLE(
  |   SESSION(TABLE T1, DESCRIPTOR(rowtime), INTERVAL '-5' SECOND))
  |GROUP BY window_start, window_end
""".stripMargin

  val sink = new TestingAppendSink
  tEnv.sqlQuery(sql).toDataStream.addSink(sink)
  env.execute()
} 

{code}
{code:java}
java.lang.AssertionError: Sql optimization: Assertion error: null at 
org.apache.flink.table.planner.plan.optimize.program.FlinkVolcanoProgram.optimize(FlinkVolcanoProgram.scala:79)
 at 
org.apache.flink.table.planner.plan.optimize.program.FlinkChainedProgram.$anonfun$optimize$1(FlinkChainedProgram.scala:59)
 at 
scala.collection.TraversableOnce.$anonfun$foldLeft$1(TraversableOnce.scala:156) 
at 
scala.collection.TraversableOnce.$anonfun$foldLeft$1$adapted(TraversableOnce.scala:156)
 at scala.collection.Iterator.foreach(Iterator.scala:937) at 
scala.collection.Iterator.foreach$(Iterator.scala:937) at 
scala.collection.AbstractIterator.foreach(Iterator.scala:1425) at 
scala.collection.IterableLike.foreach(IterableLike.scala:70) at 
scala.collection.IterableLike.foreach$(IterableLike.scala:69) at 
scala.collection.AbstractIterable.foreach(Iterable.scala:54) at 
scala.collection.TraversableOnce.foldLeft(TraversableOnce.scala:156) at 
scala.collection.TraversableOnce.foldLeft$(TraversableOnce.scala:154) at 
scala.collection.AbstractTraversable.foldLeft(Traversable.scala:104) at 
org.apache.flink.table.planner.plan.optimize.program.FlinkChainedProgram.optimize(FlinkChainedProgram.scala:55)
 at 
org.apache.flink.table.planner.plan.optimize.StreamCommonSubGraphBasedOptimizer.optimizeTree(StreamCommonSubGraphBasedOptimizer.scala:176)
 at 
org.apache.flink.table.planner.plan.optimize.StreamCommonSubGraphBasedOptimizer.doOptimize(StreamCommonSubGraphBasedOptimizer.scala:83)
 at 
org.apache.flink.table.planner.plan.optimize.CommonSubGraphBasedOptimizer.optimize(CommonSubGraphBasedOptimizer.scala:87)
 at 
org.apache.flink.table.planner.delegation.PlannerBase.optimize(PlannerBase.scala:320)
 at 
org.apache.flink.table.planner.delegation.PlannerBase.translate(PlannerBase.scala:178)
 at 
org.apache.flink.table.api.bridge.internal.AbstractStreamTableEnvironmentImpl.toStreamInternal(AbstractStreamTableEnvironmentImpl.java:224)
 at 
org.apache.flink.table.api.bridge.internal.AbstractStreamTableEnvironmentImpl.toStreamInternal(AbstractStreamTableEnvironmentImpl.java:219)
 at 
org.apache.flink.table.api.bridge.scala.internal.StreamTableEnvironmentImpl.toDataStream(StreamTableEnvironmentImpl.scala:151)
 at 
org.apache.flink.table.api.bridge.scala.internal.StreamTableEnvironmentImpl.toDataStream(StreamTableEnvironmentImpl.scala:128)
 at 
org.apache.flink.table.api.bridge.scala.TableConversions.toDataStream(TableConversions.scala:60)
 at 
org.apache.flink.table.planner.runtime.stream.sql.WindowAggregateITCase.testEventTimeSessionWindowWithInvalidName(WindowAggregateITCase.scala:1239)
 at java.lang.reflect.Method.invoke(Method.java:498) at 
java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183) at 
java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193) at 
java.util.stream.ReferencePipeline$2$1.accept(ReferencePipeline.java:175) at 
java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193) at 
java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183) at 
java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193) at 
java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193) at 
java.util.Iterator.forEachRemaining(Iterator.java:116) at 
scala.collection.convert.Wrappers$IteratorWrapper.forEachRemaining(Wrappers.scala:26)
 at 
java.util.Spliterators$IteratorSpliterator.forEachRemaining(Spliterators.java:1801)
 at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482) at 
java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472) at 
java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150) 
at 
java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
 at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) at 

[jira] [Created] (FLINK-34461) MongoDB weekly builds fail with time out on Flink 1.18.1 for JDK17

2024-02-19 Thread Martijn Visser (Jira)
Martijn Visser created FLINK-34461:
--

 Summary: MongoDB weekly builds fail with time out on Flink 1.18.1 
for JDK17
 Key: FLINK-34461
 URL: https://issues.apache.org/jira/browse/FLINK-34461
 Project: Flink
  Issue Type: Bug
  Components: Connectors / MongoDB
Affects Versions: mongodb-1.1.0
Reporter: Martijn Visser


The weekly tests for MongoDB consistently time out for the v1.0 branch while 
testing Flink 1.18.1 for JDK17:

https://github.com/apache/flink-connector-mongodb/actions/runs/7770329490/job/21190387348

https://github.com/apache/flink-connector-mongodb/actions/runs/7858349600/job/21443232301

https://github.com/apache/flink-connector-mongodb/actions/runs/7945225005/job/21691624903




--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [VOTE] Release flink-connector-mongodb v1.1.0, release candidate #2

2024-02-19 Thread Leonard Xu
+1 (binding)

- built from source code succeeded
- verified signatures
- verified hashsums 
- checked the contents contains jar and pom files in apache repo 
- checked Github release tag 
- checked release notes

Best,
Leonard

> 2024年2月8日 下午11:37,Martijn Visser  写道:
> 
> +1 (binding)
> 
> - Validated hashes
> - Verified signature
> - Verified that no binaries exist in the source archive
> - Build the source with Maven
> - Verified licenses
> - Verified web PRs
> 
> On Wed, Jan 31, 2024 at 10:41 AM Danny Cranmer  
> wrote:
>> 
>> Thanks for driving this Leonard!
>> 
>> +1 (binding)
>> 
>> - Release notes look ok
>> - Signatures/checksums of source archive are good
>> - Verified there are no binaries in the source archive
>> - Built sources locally successfully
>> - v1.0.0-rc2 tag exists in github
>> - Tag build passing on CI [1]
>> - Contents of Maven dist look complete
>> - Verified signatures/checksums of binary in maven dist is correct
>> - Verified NOTICE files and bundled dependencies
>> 
>> Thanks,
>> Danny
>> 
>> [1]
>> https://github.com/apache/flink-connector-mongodb/actions/runs/7709467379
>> 
>> On Wed, Jan 31, 2024 at 7:54 AM gongzhongqiang 
>> wrote:
>> 
>>> +1(non-binding)
>>> 
>>> - Signatures and Checksums are good
>>> - No binaries in the source archive
>>> - Tag is present
>>> - Build successful with jdk8 on ubuntu 22.04
>>> 
>>> 
>>> Leonard Xu  于2024年1月30日周二 18:23写道:
>>> 
 Hey all,
 
 Please help review and vote on the release candidate #2 for the version
 v1.1.0 of the
 Apache Flink MongoDB Connector as follows:
 
 [ ] +1, Approve the release
 [ ] -1, Do not approve the release (please provide specific comments)
 
 The complete staging area is available for your review, which includes:
 * JIRA release notes [1],
 * The official Apache source release to be deployed to dist.apache.org
 [2],
 which are signed with the key with fingerprint
 5B2F6608732389AEB67331F5B197E1F1108998AD [3],
 * All artifacts to be deployed to the Maven Central Repository [4],
 * Source code tag v1.1.0-rc2 [5],
 * Website pull request listing the new release [6].
 
 The vote will be open for at least 72 hours. It is adopted by majority
 approval, with at least 3 PMC affirmative votes.
 
 
 Best,
 Leonard
 [1]
 
>>> https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315522=12353483
 [2]
 
>>> https://dist.apache.org/repos/dist/dev/flink/flink-connector-mongodb-1.1.0-rc2/
 [3] https://dist.apache.org/repos/dist/release/flink/KEYS
 [4]
 https://repository.apache.org/content/repositories/orgapacheflink-1705/
 [5] https://github.com/apache/flink-connector-mongodb/tree/v1.1.0-rc2
 [6] https://github.com/apache/flink-web/pull/719
>>>