Thanks Hongshun for picking up the ticket and analyzing the detail deeply.
As you have discussed with Qingsheng offline, I think we can update the content
to FLIP-288[1] and then start the FLIP discussion.
Best,
Leonard
[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-288%3A+Enable+D
Hi Hongshun
Agree with @Etienne. And can you describe this process and the problems in
a figure? Thanks
Best,
Shammon FY
On Wed, Mar 15, 2023 at 5:15 PM Etienne Chauchot
wrote:
> Hi,
>
> Why not track this in a FLIP and a ticket and link this discussion thread.
>
> My 2 cents
>
> Etienne
>
> L
Thanks for your advise! I will do this later.
Best, Hongshun
On Wed, Mar 15, 2023 at 5:15 PM Etienne Chauchot
wrote:
> Hi,
>
> Why not track this in a FLIP and a ticket and link this discussion thread.
>
> My 2 cents
>
> Etienne
>
> Le 15/03/2023 à 10:01, Hongshun Wang a écrit :
> > Hi devs,
Hi,
Why not track this in a FLIP and a ticket and link this discussion thread.
My 2 cents
Etienne
Le 15/03/2023 à 10:01, Hongshun Wang a écrit :
Hi devs,
I’d like to join this discussion. CC:Qingsheng
As discussed above, new partitions after the first discovery should be
consumed from EARLI
Hi devs,
I’d like to join this discussion. CC:Qingsheng
As discussed above, new partitions after the first discovery should be
consumed from EARLIEST offset.
However, when KafkaSourceEnumerator restarts after a job failure, it cannot
distinguish between unassigned partitions as first-discovered o
erience. The counter argument
>>> last
>>> >> > time
>>> >> > > when I proposed to enable auto partition discovery was mainly due
>>> to
>>> >> the
>>> >> > > concern from the Flink users. There were arguments that sometimes
>>> >> users
>>&
gt;> want
>> >> > > to do this by restarting the job manually so they can avoid
>> unnoticed
>> >> > > changes in the jobs.
>> >> > >
>> >> > > Given that in the old Flink source, by default the auto partition
>> >> > discovery
>&
iscovery is much preferred. So I think we
> >> should
> >> > > do it.
> >> > >
> >> > > I am not worried about the performance. The new Kafka source will
> only
> >> > have
> >> > > the SplitEnumerator sending metad
ests when the feature is
>> > enabled.
>> > > It is actually much cheaper than the old Kafka source where every
>> > > subtask does that.
>> > >
>> > > Thanks,
>> > >
>> > > Jiangjie (Becket) Qin
>> > >
t; >
> > > Thanks,
> > >
> > > Jiangjie (Becket) Qin
> > >
> > >
> > >
> > > On Sat, Jan 14, 2023 at 11:46 AM Yun Tang wrote:
> > >
> > > > +1 for this proposal and thanks Qingsheng for driving this
ng this.
> > >
> > > Considering the interval, we also set the value as 5min, equivalent to
> > the
> > > default value of metadata.max.age.ms.
> > >
> > >
> > > Best
> > > Yun Tang
> > > ___
nks Qingsheng for driving this.
> >
> > Considering the interval, we also set the value as 5min, equivalent to
> the
> > default value of metadata.max.age.ms.
> >
> >
> > Best
> > Yun Tang
> > ____________
> > From: Benc
Thanks Qingsheng for the proposal. +1 for the change.
On Fri, Jan 13, 2023 at 11:32 AM Qingsheng Ren wrote:
> Hi devs,
>
> I’d like to start a discussion about enabling the dynamic partition
> discovery feature by default in Kafka source. Dynamic partition discovery
> [1] is a useful feature in
tadata.max.age.ms.
>
>
> Best
> Yun Tang
>
> From: Benchao Li
> Sent: Friday, January 13, 2023 23:06
> To: dev@flink.apache.org
> Subject: Re: [DISCUSS] Enabling dynamic partition discovery by default in
> Kafka source
>
> +1, we&
@flink.apache.org
Subject: Re: [DISCUSS] Enabling dynamic partition discovery by default in Kafka
source
+1, we've enabled this by default (10mins) in our production for years.
Jing Ge 于2023年1月13日周五 22:22写道:
> +1 for the proposal that makes users' daily work easier and therefore makes
+1, we've enabled this by default (10mins) in our production for years.
Jing Ge 于2023年1月13日周五 22:22写道:
> +1 for the proposal that makes users' daily work easier and therefore makes
> Flink more attractive.
>
> Best regards,
> Jing
>
>
> On Fri, Jan 13, 2023 at 11:27 AM Qingsheng Ren wrote:
>
>
+1 for the proposal that makes users' daily work easier and therefore makes
Flink more attractive.
Best regards,
Jing
On Fri, Jan 13, 2023 at 11:27 AM Qingsheng Ren wrote:
> Thanks everyone for joining the discussion!
>
> @Martijn:
>
> > All newly discovered partitions will be consumed from th
Thanks everyone for joining the discussion!
@Martijn:
> All newly discovered partitions will be consumed from the earliest offset
possible.
Thanks for the reminder! I checked the logic of KafkaSource and found that
new partitions will start from the offset initializer specified by the user
inste
+1 on the overall direction, it's an important feature.
I've had a look on the latest master and looks like removed partition
handling is not yet added but I think this is essential.
https://github.com/apache/flink/blob/28c3e1a3923ba560b559a216985c1abeb794ebaa/flink-connectors/flink-connector-kaf
Thanks for this proposal, Qingsheng!
If you want to be a little conservative with the default, 5 minutes might be
better than 30 seconds.
The equivalent config in Kafka seems to be metadata.max.age.ms
(https://kafka.apache.org/documentation/#consumerconfigs_metadata.max.age.ms),
which has a de
This sounds like a no-brainer +1
Two things that seem to be obvious, but might be good to double check:
1. All newly discovered partitions will be consumed from the earliest
offset possible. That's how it's documented for version 1.12 [1], but not
for later versions, which is why I would like to
Thanks Qingsheng for driving this, enable the dynamic partition discovery would
be very useful for kafka topic scale partitions scenarios.
+1 for the change.
CC: Becket
Best,
Leonard
> On Jan 13, 2023, at 3:15 PM, Jark Wu wrote:
>
> +1 for the change. I think this is beneficial for user
+1
It’s hard to imagine why someone would not enable this in prod anyways.
Gyula
On Fri, 13 Jan 2023 at 08:17, Jark Wu wrote:
> +1 for the change. I think this is beneficial for users and is compatible.
>
> Best,
> Jark
>
> On Fri, 13 Jan 2023 at 14:22, 何军 wrote:
>
> > >
> > > +1 for this id
+1 for the change. I think this is beneficial for users and is compatible.
Best,
Jark
On Fri, 13 Jan 2023 at 14:22, 何军 wrote:
> >
> > +1 for this idea, we have enabled kafka dynamic partition discovery in
> all
> > jobs.
> >
> >
>
>
> +1 for this idea, we have enabled kafka dynamic partition discovery in all
> jobs.
>
>
25 matches
Mail list logo