Hi Piotr,
Thanks for the comments. Let me try to understand your concerns and
hopefully address the concerns.
>> What would happen if there are two (or more) operator coordinators with
conflicting desired checkpoint trigger behaviour
With the proposed change, there won't exist any "*conflicting*
d to be processed. Otherwise, up-to-date data will be overwritten by
> out-of-date data which turns out to be unexpected results in real business
> scenarios.
>
>
> Best regards,
> Jing
>
> [1]
>
> https://github.com/apache/flink/blob/fadde2a378aac4293676944dd513291919a481e
I am not sure how to address the use-case mentioned in the motivation
section, with the pluggable checkpoint trigger + metrics. Can you help
provide the definition of these APIs and kindly explain how that works to
address the mentioned use-case.
In the mentioned use-case, users want to have two
> > makes sense to build it as a generic feature in Flink. Looking forward to
> > it. However, for some user cases, e.g. when users were aware of the
> bounded
> > sources (in the HybridSource) and care more about the throughput, the
> > dynamic adjustment migh
frequent interval in the continuous phase in face of
a spike in the number of pending records buffered in the source operator.
On Tue, May 30, 2023 at 9:17 AM Dong Lin wrote:
> Hi Piotrek,
>
> Thanks for providing more details of the alternative approach!
>
> If I understan
us spectrum to gradually adjust the
> interval, by replacing the first if/else
> check with a weighted average:
>
> int maxBackPressureTime = getSubtaskMaxBackPressuredTimeMsPerSecond();
> long nextCheckpointTs = lastCheckpointTs + slowCheckpointInterval *
> maxBackPressureTime + fastCheckpoi
docs/ops/metrics/#io
> [2]
>
> https://nightlies.apache.org/flink/flink-docs-release-1.17/docs/deployment/metric_reporters/
> [3]
>
> https://nightlies.apache.org/flink/flink-docs-master/docs/ops/metrics/#end-to-end-latency-tracking
> [4]
> https://cwiki.apache.org/confluence/d
ing the interval hence the trigger logic could be implemented on
> Flink or external systems like Flink Kubernetes Operator?
> Wdyt? I think the REST API proposal here sounds more and more interesting.
>
>
> Best Regards,
> Ahmed Hamdy
>
>
> On Wed, 31 May 2023 at 07:59,
Hi Piotr,
Thanks for the explanations. I have some followup questions below.
On Fri, Jun 2, 2023 at 10:55 PM Piotr Nowojski wrote:
> Hi All,
>
> Thanks for chipping in the discussion Ahmed!
>
> Regarding using the REST API. Currently I'm leaning towards implementing
> this feature inside the Fl
+1 (binding)
Thank you Archit for driving the FLIP.
Cheers,
Dong
On Tue, Jun 6, 2023 at 12:55 AM Archit Goyal
wrote:
> Hi everyone,
>
> Thanks for all the feedback for FLIP-312: Add Yarn ACLs to Flink
> Containers<
> https://cwiki.apache.org/confluence/display/FLINK/FLIP+312%3A+Add+Yarn+ACLs+t
Thanks Feng for the FLIP.
+1(binding)
Cheers,
Dong
On Wed, Jun 14, 2023 at 10:35 AM Feng Jin wrote:
> Hi everyone
>
> Thanks for all the feedback about the FLIP-295: Support lazy initialization
> of catalogs and persistence of catalog configurations[1].
> [2] is the discussion thread.
>
>
> I'
y sure
> that's a (for 95-99% of cases)
> solvable problem. If not, there is always the hacky solution, that could be
> even integrated into this above
> mentioned algorithm as a short circuit to always reach `slow-interval`.
>
> Apart of that, you picked 3 minutes as the check
se case sufficiently (imo they can), then
>
To be clear, I think the solution based on the CheckpointTrigger +
algorithm can be useful in some cases (e.g. when the source has
event-time). And I would be happy to draft a FLIP separately to take
advantage of this idea.
For the use case targeted by
; metrics that are used by various auto scaling solutions (like Flink K8s
> operator's
> autosaler) would be better. The hacky solution I proposed to:
> 1. show you that the generic solution is simply a superset of your proposal
> 2. if you are adamant that busyness/backpressured/rec
Hi everyone,
We would like to start voting for the Flink ML 2.3.0 release. This
release primarily
provides the ability to run Flink ML on Flink 1.15, 1.16 and 1.17.
Please review and vote on the release candidate #1 for version 2.3.0 of
Apache Flink ML as follows.
[ ] +1, Approve the release
[ ]
ger parallelism/memory/cpu
> for job under backlog status in the future.
>
> In short, the updated FLIP looks good to me.
>
>
> Best,
> Leonard
>
>
> > On Jun 22, 2023, at 12:07 PM, Dong Lin wrote:
> >
> > Hi Piotr,
> >
> > Thanks again for p
o me!
> >
> > Thanks Dong, Yunfeng and all for your discussion and design.
> >
> > Best,
> > Jingsong
> >
> > On Tue, Jun 27, 2023 at 3:35 PM Jark Wu wrote:
> > >
> > > Thank you Dong for driving this FLIP.
> > >
> > > The
t; >
> > The FLIP looks good to me. This new version is clearer to understand.
> >
> > Best,
> > Hang
> >
> > Dong Lin 于2023年6月27日周二 16:53写道:
> >
> > > Thanks Jack, Jingsong, and Zhu for the review!
> > >
> > > Thanks Zhu for
tps://cwiki.apache.org/confluence/display/FLINK/FLIP-328%3A+Allow+source+operators+to+determine+isProcessingBacklog+based+on+event-time+lag>)
to allow users to determine isBacklog dynamically based on the event-time
lag and/or source backpressure metrics.
On Thu, Jun 29, 2023 at 10:49 AM Do
Hi all,
We would like to start the vote for FLIP-309: Support using larger
checkpointing interval when source is processing backlog [1]. This FLIP was
discussed in this thread [2].
Flink 1.18 release will feature freeze on July 11. We hope to make this
feature available in Flink 1.18.
The vote w
Hi Martijn,
Thanks for your feedback! Please see my replhy inline.
On Thu, Jun 29, 2023 at 4:35 PM Martijn Visser
wrote:
> Hi Dong and Yunfeng,
>
> Thanks for the FLIP. What's not clear for me is what's the expected
> behaviour when the allowed latency can't be met, for whatever reason.
> Given
Hi Shammon,
Thanks for your comments. Please see my reply inline.
On Thu, Jun 29, 2023 at 6:01 PM Shammon FY wrote:
> Hi Dong and Yunfeng,
>
> Thanks for bringing up this discussion.
>
> As described in the FLIP, the differences between `end-to-end latency` and
> `table.exec.mini-batch.allow-la
I will also vote +1 (binding).
- Verified checksums and GPG for all maven artifacts and source
distributions
- Successfully run the Flink ML Python/Java quickstart using source
distributions.
- Verified that the source distributions do not contain any unwanted
binaries.
- Built the source distribu
Hi all,
I'm happy to announce that we have unanimously approved this release [1].
There are 5 approving votes, 3 of which are binding:
- Dian Fu (binding)
- Xingbo Huang (binding)
- Dong Lin (binding)
- Xin Jiang (non-binding)
- Zhipeng Zhang (non-binding)
There are no disapproving
Hi all,
I'm happy to announce that we have unanimously approved this release [1].
There are 5 approving votes, 3 of which are binding:
- Dian Fu (binding)
- Xingbo Huang (binding)
- Dong Lin (binding)
- Xin Jiang (non-binding)
- Zhipeng Zhang (non-binding)
There are no disapproving
Sorry, I used the wrong title. I just sent another email with the right
title to announce the voting result.
On Fri, Jun 30, 2023 at 7:26 AM Dong Lin wrote:
> Hi all,
>
> I'm happy to announce that we have unanimously approved this release [1].
>
> There are 5 approving v
ferred to
throw exception only when there is indeed error. If we don't need to check
a checkpoint, it is preferred to not trigger the checkpoint in the first
place. And I think adding SplitEnumeratorContext#setIsProcessingBacklog is
probably not that much of a big deal.
Thanks for all the comment
Hi Chesnay, can you put your comments in the discussion thread, so that we
can continue the technical discussion there?
Thanks for the FLIP.
+1 (binding)
On Fri, Jun 30, 2023 at 5:39 PM Becket Qin wrote:
> Hi folks,
>
> I'd like to start the VOTE for FLIP-321[1] which proposes to introduce an
> API deprecation process to Flink. The discussion thread for the FLIP can be
> found here[2].
>
> The vote will be open
uepeng Pan"
> >>> 收件人: "dev"
> >>> 发送时间: 星期四, 2023年 6 月 29日 下午 8:21:14
> >>> 主题: Re: [VOTE] FLIP-309: Support using larger checkpointing interval
> when
> >>> source is processing backlog
> >>>
> >>> +1 non-
Hi all,
I am opening this thread to discuss FLIP-327: Support stream-batch unified
operator to improve job throughput when processing backlog data. The design
doc can be found at
https://cwiki.apache.org/confluence/display/FLINK/FLIP-327%3A+Support+stream-batch+unified+operator+to+improve+job+thro
back of your
proposed solution, and then you repeat the same ask and insist this is
possible.
If you can find a solution that wouldn't require modification of every
source and still address the target use-case well, could you please kindly
rephrase your solution so that we can revisit it?
APIs.
If you prefer to have a better solution with simpler APIs and yet same or
better correctness/performance for the target use-case, could you please
kindly explain its API design so that we can continue the discussion?
Best,
Dong
On Mon, Jul 3, 2023 at 6:39 PM Dong Lin wrote:
> Hi Pi
Hi Jing,
Thanks for the comments! Please find below my comments, which are based on
the offline discussion with Xuannan.
On Wed, Jul 5, 2023 at 1:36 AM Jing Ge wrote:
> Hi Xuannan, Hi Dong
>
> Thanks for the Proposal! After reading the FLIP, I'd like to ask some
> questions:
>
> 1. Naming conve
Hi Piotr,
I am sorry if you feel unhappy or upset with us for not following/fixing
your proposal. It is not my intention to give you this feeling. After all,
we are all trying to make Flink better, to support more use-case with the
most maintainable code. I hope you understand that just like you,
ial benefit
of propagating isBacklog from source operators to downstream operators is
that we might take advantage of features in FLIP-327, dynamically disable
checkpoint, and apply sorting to optimize the operator throughput. IMHO,
such optimization is probably easier to understand and operate
ter thinking through this,
I still could not find a use-case where we actually need this flexibility.
In particular, for cases where a user has explicitly configured
pipeline.object-reuse to true, that means the user already knows (or takes
the responsibility of ensuring) that correctness can be ac
tures proposed in FLIP-325 and FLIP-327 to optimize the
above use-case.
What do you think?
Best,
Dong
>
> Best,
> Piotrek
>
> niedz., 2 lip 2023 o 16:15 Dong Lin napisał(a):
>
> > Hi all,
> >
> > I am opening this thread to discuss FLIP-327: Support stream-b
committed to balance between
> > throughput and latency.
> >
> > 6.
> >
> > > Add RecordAttributesBuilder and RecordAttributes that extends
> > StreamElement to provide operator with essential
> > > information about the records they receive, such as
gt;
Yeah, thanks for the suggestion. I have updated the FLIP as suggested.
Best,
Dong
>
> Best regards,
> Jing
>
> On Thu, Jul 6, 2023 at 2:38 PM Dong Lin wrote:
>
> > Hi Shammon,
> >
> > Thanks for your comments. Please see my reply inline.
> >
>
;
> Best regards,
> Jing
>
> On Thu, Jul 6, 2023 at 12:30 PM Dong Lin wrote:
>
>> Hi Jing,
>>
>> Thank you for the detailed explanation. Please see my reply inline.
>>
>> On Thu, Jul 6, 2023 at 3:17 AM Jing Ge wrote:
>>
>>> Hi Xuannan, Hi
e ways you mentioned for how to derive isBacklog.
>
Sure, I have updated the FLIP to include the following information:
"In the future, we expect IsProcessingBacklog can very likely be determined
using the following strategies ..."
Best,
Dong
>
> WDYT?
>
> Best regards,
>
se FLIPs
in collaboration.
The overall vision behind these FLIPs is to maximize Flink performance for
stream-batch unified use-case (e.g. those use-cases that need to process a
bounded set of stale data followed by an unbounded set of fresh data).
Currently, even though Flink can generate correctness r
Hi Stefan,
Thanks for all the comments! That is really helpful and I have updated the
FLIP based on your comments. Please see my reply inline.
On Mon, Jul 10, 2023 at 10:23 PM Stefan Richter
wrote:
> Hi,
>
> After reading through the discussion, I think the FLIP should provide
> additional deta
.
>
>
Thank you for catching this issue. It is fixed now.
Best,
Dong
>
>
> Best regards
> Jing
>
>
> On Thu, Jul 6, 2023 at 1:24 PM Dong Lin wrote:
>
> > Hi Piotr,
> >
> > Thanks for your comments! Please see my reply inline.
> >
> > On We
e travelling,
> being aggregated and what will be done with that information.
> (from SourceReader/SplitEnumerator to some
> "component" aggregating it, and then ... ?)
> 2. For me "processing backlog" doesn't necessarily equate to &quo
I will read the summary that you have just published later, but I think
> we don't need to block this FLIP on the
> existence of that high level summary.
>
> wt., 11 lip 2023 o 17:49 Dong Lin napisał(a):
>
> > Hi Piotr and everyone,
> >
> > I have doc
future, we plan to incrementally optimize other aggregation
operation (e.g. aggregate) on the DataStream API when EndOfStreamWindows is
used as the window assigner.
Best,
Dong
>
> Best regards,
> Jing
>
>
> On Tue, Jul 11, 2023 at 2:58 PM Dong Lin wrote:
>
> > Hi Jing,
&
?
> Or do you envision that in the future users will be configuring only:
> - execution.end-to-end-latency
> and only optionally:
> - execution.checkpointing.interval-during-backlog
> ?
>
> Best Piotrek
>
> PS, I will read the summary that you have just published later, but I think
>
so reference future work, that a solution that would allow set
> `isProcessingBacklog` for sources like Kafka will be introduced via
> FLIP-328 [1].
>
> Best,
> Piotrek
>
> [1]
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-328%3A+Allow+source+operators+to+d
Hi all,
We would like to start the vote for FLIP-309: Support using larger
checkpointing interval when source is processing backlog [1]. This FLIP was
discussed in this thread [2].
The vote will be open until at least July 21st (at least 72 hours), following
the consensus voting process.
Cheers,
upport starting with
> batch mode and switching only once to
> streaming, and design a follow up with switching back and forth?
>
Sure, that sounds good to me. I am happy to split this FLIP into two FLIPs
so that we can make incremental progress.
Best,
Dong
> I'm looking forw
)
- Piotr Nowojski (binding)
- Jark Wu (binding)
- Jingsong Li (binding)
- Leonard Xu (binding)
- Guowei Ma (binding)
- Yu Xia (binding)
- Zhu Zhu (binding)
- Stefan Richter (binding)
- Dong Lin (binding)
- Hang Ruan
There is no disapproving vote.
Cheers,
Dong
[1] https://cwiki.apache.org/confluence
and forth.
>
Cool, I added the following statement to the motivation section.
"NOTE: this FLIP focuses only on the capability to switch from batch to
stream mode. If there is any extra API needed to support switching from
stream to batch mode, we will discuss them in a follow-up FLIP."
Hi David,
Thank you for the detailed comments and the suggestion of this alternative
approach.
I agree with you that this alternative can also address the target use-case
with the same correctness. In comparison to the current FLIP, this
alternative indeed introduces much less complexity to the F
Hi Xintong,
Thanks for the detailed explanation of your concern. Let me chime in and
provide my thoughts on this issue.
Please see my comments for each point inline.
Overall, it seems that the main concern with this FLIP is that the 2%
throughput saving might not be worth the added implementatio
t,
Dong
> > Maybe we can revisit the need for such a config when we introduce/discuss
> > the capability to switch backlog from false to true in the future. What
> do
> > you think?
>
> Sure, we can do that.
>
> Best,
> Piotrek
>
> niedz., 23 lip 2023
take back and may cause problems for future developments.
> For this specific assumption, it doesn't come from any contract. It just
> happens to be true with the current implementation.
>
> Best,
>
> Xintong
>
>
>
> On Fri, Aug 11, 2023 at 10:43 AM Dong Lin wrote:
ncern as
> discussed above?
>
> We should do this independently from marking the APIs/config options as
> `@Experimental`
>
> Best,
> Piotrek
>
> pt., 11 sie 2023 o 14:55 Dong Lin napisał(a):
>
> > Hi Piotr,
> >
> > Thanks for the reply!
> >
&
Hi all,
Jinhao (cc'ed) and I are opening this thread to discuss FLIP-331: Support
EndOfStreamWindows and isOutputOnEOF operator attribute to optimize task
deployment. The design doc can be found at
https://cwiki.apache.org/confluence/display/FLINK/FLIP-331%3A+Support+EndOfStreamWindows++and+isOutp
Thanks Wencong for initiating the discussion.
+1 for the proposal.
On Fri, Sep 1, 2023 at 12:00 PM Wencong Liu wrote:
> Hi devs,
>
> I would like to start a discussion on FLIP-357: Deprecate Iteration API of
> DataStream [1].
>
> Currently, the Iteration API of DataStream is incomplete. For ins
Thanks Allison for proposing the FLIP.
+1 (binding)
On Fri, Sep 8, 2023 at 4:21 AM Allison Chang
wrote:
> Hi everyone,
>
> Would like to start the VOTE for FLIP-323<
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-323%3A+Support+Attached+Execution+on+Flink+Application+Completion+for+Ba
Hi Jark,
Thanks for the comments. Please see my comments inline.
On Sat, Sep 9, 2023 at 4:13 PM Jark Wu wrote:
> Hi Xuannan,
>
> I leave my comments inline.
>
> > In the case where a user wants to
> > use a CDC source and also determine backlog status based on watermark
> > lag, we still need t
hey don't
need to handle the case where backlog switches from true to false.
Although we can move the limitation from FLIP-328 to FLIP-327, it will just
become harder to explain to Flink developers that "source can switch
backlog from false to true but you don't need to handle th
+1 (binding)
- Verified that the source release can be built successfully.
- Verified that the checksum and gpg files match the corresponding source
release files and maven artifacts.
- Verified that the source archives do not contain any binary file.
- Checked that the source code tag looks good.
Thank you Rui for the proposal.
+1 (binding)
On Wed, Sep 13, 2023 at 10:52 AM Rui Fan <1996fan...@gmail.com> wrote:
> Hi all,
>
> Thanks for all the feedback about the FLIP-334:
> Decoupling autoscaler and kubernetes and
> support the Standalone Autoscaler[1].
> This FLIP was discussed in [2].
>
Thanks for the FLIP!
+1(binding)
On Wed, Sep 13, 2023 at 9:16 PM Gyula Fóra wrote:
> Hi All!
>
> Thanks for all the feedback on FLIP-361: Improve GC Metrics [1][2]
>
> I'd like to start a vote for it. The vote will be open for at least 72
> hours unless there is an objection or insufficient vot
Thanks Archit for the FLIP.
+1 (binding)
Regards,
Dong
On Thu, Sep 14, 2023 at 1:47 AM Archit Goyal
wrote:
> Hi everyone,
>
> Thanks for reviewing the FLIP-355 Add parent dir of files to classpath
> using yarn.provided.lib.dirs :
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP+355%3
Thanks Wencong for the FLIP.
+1 (binding)
On Thu, Sep 14, 2023 at 12:36 PM Wencong Liu wrote:
> Hi dev,
>
>
> I'd like to start a vote on FLIP-357.
>
>
> Discussion thread:
> https://lists.apache.org/thread/shf77phc0wzlbj06jsfj3nclxnm2mrv5
> FLIP:
> https://cwiki.apache.org/confluence/display/F
ode switching back
> and forth.
> However, I think this should be a limitation of FLIP-327, not FLIP-328.
> IIUC,
> FLIP-309 doesn't have this limitation, right? I just don't understand
> what's the
> challenge to switch a flag?
>
> Best,
> Jark
>
>
;s backlog status to false".
Would this answer your question?
Best,
Dong
> understand. From the API interface "ctx.setIsProcessingBacklog(boolean)",
> it allows users to invoke "setIsProcessingBacklog(false)". And FLIP-309
> also says "MySQL CDC
is
operator will sort data internally, which means that Flink runtime should
not additionally sort its inputs. So overall the Flink job can comply with
the semantics of these two attributes consistently.
Thanks again for taking time to review this FLIP. Please let me know what
you think.
Best regards,
> on
> > EndOfStreamWindows have the attribute 'isInternalSorterSupported' set to
> > true,
> > indicating support for the 'mixed mode', they also have isOutputOnEOF set
> > to true,
> > which suggests that the upstream operators should be executed in
ly if no rule says it should be set to true.
BTW, I can understand that setIsProcessingBacklog(false) appears to suggest
the job's backlog is set to false. We can certainly update its Java doc to
make the semantics clearer. If you have any suggestion on a better name for
this method, we can a
the input of the one-input
> > > operator to be automatically sorted during backlog processing. When
> > > combined with the state backend optimization introduced in FLIP-325
> [1],
> > > all the keyed single-input operators can achieve similar performance as
> > i
ving
flink-streaming-java depend on flink-runtime, I agree it is preferred to
avoid introducing more dependencies like this.
I have updated the FLIP to let RecordAttributes extend StreamElement.
Best,
Dong
> The rest of your replies make sense to me.
>
> Best,
>
> Xintong
>
>
t;setIsProcessingBacklog(false)" first, but the watermark
> lag
> exceeds the configured value.
>
> This is the conflict I'm concerned about.
>
> Best,
> Jark
>
> On Fri, 15 Sept 2023 at 12:00, Dong Lin wrote:
>
> > Hi Jark,
> >
> > Please see
o make semantics
above clearer for Flink users.
And I would be happy to discuss/explain this design offline when you have
time.
Thanks,
Dong
>
> Best,
> Jark
>
> On Tue, 19 Sept 2023 at 09:13, Dong Lin wrote:
>
> > Hi Jark,
> >
> > Do you have time to comm
Hi all,
Thank you for the comments!
If there is no further comment, we will open the voting thread in 3 days.
Cheers,
Xuannan and Dong
ave not reached agreement on which option is better. It will be
really helpful if we can get more comments on these options.
Thanks,
Dong
On Tue, Sep 19, 2023 at 11:26 AM Dong Lin wrote:
> Hi Jark,
>
> Thanks for the reply. Please see my comments inline.
>
> On Tue, Sep 19, 2023 at 10
Hi all,
We would like to start the vote for FLIP-327: Support switching from batch
to stream mode to improve throughput when processing backlog data [1]. This
FLIP was discussed in this thread [2].
The vote will be open until at least Sep 27th (at least 72
hours), following the consensus voting p
>
> > 2) The semantics of `#setIsProcessingBacklog(false)` is that the given
> > source instance will have watermarkLag=false.
>
> ?
>
> Best,
> Piotrek
>
> czw., 21 wrz 2023 o 15:28 Dong Lin napisał(a):
>
> > Hi all,
> >
> > Jark and I discu
ng Liu
> wrote:
> >
> > > Hi Dong & Jinhao,
> > >
> > > Thanks for your clarification! +1
> > >
> > > Best regards,
> > > Wencong
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> >
23 at 10:16 PM Yuepeng Pan
> > > wrote:
> > >
> > > > +1(non-binding), thank you for driving this proposal.
> > > >
> > > > Best,
> > > > Yuepeng Pan.
> > > > At 2023-09-22 14:07:45, "Dong Lin" wrote:
> > &g
(binding)
- Xintong Song (binding)
- Dong Lin (binding)
- Yuepeng Pan (non-binding)
- Venkatakrishnan Sowrirajan (non-binding)
- Ahmed Hamdy (non-binding)
There is no disapproving vote.
Cheers,
Dong
[1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-327
%3A+Support+switching+from+batch
Thanks for the FLIP!
+1 (binding)
Xuannan Su 于2023年10月19日 周四10:30写道:
> Hi all,
>
> We would like to start the vote for FLIP-329: Add operator attribute
> to specify support for object-reuse[1]. This FLIP was discussed in
> this thread [2].
>
> The vote will be open until at least Oct 22nd (at le
Thanks for the FLIP!
+1 (binding)
On Wed, Oct 18, 2023 at 10:25 AM Xuannan Su wrote:
> Hi all,
>
> We would like to start the vote for FLIP-328: Allow source operators
> to determine isProcessingBacklog based on watermark lag [1]. This FLIP
> was discussed in this thread [2].
>
> The vote will
Hi Ken,
Sorry for the late reply. I didn't notice this email from you until now.
In this scenario you described above, I don't think operator2 will see the
result modified by operato1. Note that object re-use applies only to the
transmission of data between operators in the same operator chain. B
Hi Lu,
I am not actively working on Flink and this JIRA recently. If Xuannan does
not plan to work on this anytime soon, I personally think it will be great
if you can help work on this FLIP. Maybe we can start the voting thread if
there is no further comment on this FLIP.
Xuannan, what do you th
+1 (binding)
Thanks for the FLIP!
On Thu, Dec 1, 2022 at 12:20 PM Shengkai Fang wrote:
> Hi All,
>
> Thanks for all the feedback so far. Based on the discussion[1] we seem
> to have a consensus, so I would like to start a vote on FLIP-273.
>
> The vote will last for at least 72 hours (Dec 5th a
ather than the enumerations, or
> introduce a new custom type strategy which can load customized
> implementations.
> This can not solve the problem immediately, but makes the choice of restart
> strategy more flexiable.
> What do you think about this?
>
> Thanks.
>
>
Hello Chesney,
The overall plan sounds good! Just to double check, is Dec 9th the proposed
cutoff date for the release of those externalized connectors?
Also, will we reserve time for users to verify that the drop-in replacement
from Flink 1.16 to those externalized connectors can work as expecte
a verification. Release testing is
> meant to achieve that.
> Since 1.16.x is unaffected by the removal from the master branch there
> is no risk to existing deployments, while 1.17 is still quite a bit away.
>
> On 02/12/2022 02:11, Dong Lin wrote:
> > Hello Chesney,
> >
> &
Hi Samrat,
Thanks for the FLIP!
Since this is the first proposal for adding a vendor-specific catalog
library in Flink, I think maybe we should also externalize those catalog
libraries similar to how we are externalizing connector libraries. It is
likely that we might want to add catalogs for oth
Hi Dong Lin,
>
> Since this is the first proposal for adding a vendor-specific catalog
> > library in Flink, I think maybe we should also externalize those catalog
> > libraries similar to how we are externalizing connector libraries. It is
> > likely that we might want to add
Hi Matthias,
Thanks for the proposal! Overall I am in favor of making this interface
change to make Flink's codebase more maintainable.
Regarding "Interface change might affect other projects that customize HA
services", are you referring to those projects which hack into Flink's
source code (as
Hi Hang,
Thanks for the FLIP! The FLIP looks good and it is pretty informative.
I have just two minor comments regarding names:
- Would it be useful to rename the config key as
*metrics.scope.jm.job.operator-coordinator* for consistency with
*metrics.scope.jm.job
*(which is not named as *jm-job)?
uire us to touch
> >> the interface again if (for whatever reason) we want to reintroduce
> >> per-component leader election in some form.
> >> The interface change is, strictly speaking, not necessary to provide the
> >> new functionality. But I like the idea o
ght it's a good idea but forgot about FLINK-24038.
>
> Matthias
>
> [1] https://issues.apache.org/jira/browse/FLINK-24038
> [2] https://issues.apache.org/jira/browse/FLINK-25806
>
> On Fri, Dec 9, 2022 at 2:09 AM Dong Lin wrote:
>
> > Hi Matthias,
> >
> &
l metrics backend.
>
> Since I'm on holidays soon, just so no one tries to pull a fast one on
> me, if this were to go to a vote as-is I'd be against it.
>
>
> On 09/12/2022 15:30, Dong Lin wrote:
> > Hi Hang,
> >
> > Thanks for the FLIP! The FLIP looks go
1 - 100 of 255 matches
Mail list logo