Lincoln Thanks! Looks very good.
Best,
Jingsong
On Tue, May 31, 2022 at 12:09 PM Lincoln Lee wrote:
> Thanks Jingsong for the reminder! There is indeed some cleanup work to be
> done, I'll update the doc today.
>
> Best,
> Lincoln Lee
>
>
> Jingsong Li 于2022年5月31日周二 11:29写道:
>
> > Hi Lincoln,
Thanks Jingsong for the reminder! There is indeed some cleanup work to be
done, I'll update the doc today.
Best,
Lincoln Lee
Jingsong Li 于2022年5月31日周二 11:29写道:
> Hi Lincoln,
>
> Some minor comments for FLIP document (Does not affect voting):
>
> - Maybe we can remove `The current processing f
Hi Lincoln,
Some minor comments for FLIP document (Does not affect voting):
- Maybe we can remove `The current processing flow chart` part, at first
impression I thought this was part of `Proposed Changes`, it's right under
`Proposed Changes`.
- About `We tried two implementations`, I think we ca
Hi everyone,
I started a vote for this FLIP [1], please vote there or ask additional
questions here. [2]
[1] https://lists.apache.org/thread/qp9y1k0gldxymzmrso0xgsrwh15n6clc
[2] https://lists.apache.org/thread/pgm3bf8vd5vqchlm29n6cro0gz4pbd3g
Best,
Lincoln Lee
Jark Wu 于2022年5月27日周五 14:44写道:
Thanks, Lincoln, the updated comments look good to me.
Best,
Jark
On Fri, 27 May 2022 at 14:21, Lincoln Lee wrote:
> Hi Jark & developers,
>
> I'm fine with this, and minor changes:
>
> "timeout from first invoke to final completion of asynchronous operation,
> may include multiple retries, and
Hi Jark & developers,
I'm fine with this, and minor changes:
"timeout from first invoke to final completion of asynchronous operation,
may include multiple retries, and will be reset in case of failover"
The FLIP[1] was updated including two changes:
1. generic type naming, use OUT instead of T
Hi Lincoln,
What do you think about
"timeout for the asynchronous operation from the first invoke to finally
complete, which may across multiple retry attempts".
Best,
Jark
On Wed, 25 May 2022 at 20:29, Lincoln Lee wrote:
> Hi Jark,
>
> Thanks for your feedback!
>
> for 2) good advice for the
Hi Jark,
Thanks for your feedback!
for 2) good advice for the generic type naming, use OUT instead of T for
the async scenario can be better.
for 3) your concern makes sense to me, we should make the change more
explicitly to users, especially the api itself (although the documentation
is necess
Hi Lincoln,
Thanks for proposing this retry feature for the async operator, this would
be very helpful for FLIP-234.
It's glad to see the vivid discussion, and the following are my thoughts:
1) +1 w/o retry state.
It's very tricky and hard to implement a semantic exact state for retry
(currentAtt
Hi everyone,
Gen Luo, Yun Gao and I had a long offline discussion about the
implementation of the recovery part. The key point was should we store the
retry state and do the recovery after the job restart?
We reached a consensus not to store the retry state for now, which is the
clearest for user
Hi Gen Luo,
You're right, the total cost time include the failover-restart time. So
when the failover time exceeds the retry timeout set by the user, in fact,
all the data to be retry after recovery will have no additional retry
opportunities, which is equivalent to normal data. In such circumstan
Hi Lincoln,
Thanks for the explanation. I understand your thought, but I'm a little
confused by the additional detail.
Is the startTime when the record is processed for the first time? And the
cost time is counted based on it even after a job recovers from a failover
or is restarted? For the failo
Thanks Gen Luo!
Agree with you that prefer the simpler design.
I’d like to share my thoughts on this choice: whether store the retry state
or not only affect the recovery logic, not the per-record processing, so I
just compare the two:
1. w/ retry state: simple recovery but lost precision
2. w/o
Hi Lincoln,
Thanks for the quick reply.
1. I understand when restarting a job with a savepoint, the retry state can
ensure the total retry attempts and delay is expected. However, when
failover happens while a job is running, the remaining attempts recorded in
the state are actually redid, and
Hi Gen Luo,
Thanks a lot for your feedback!
1. About the retry state:
I considered dropping the retry state which really simplifies state changes
and avoids compatibility handling. The only reason I changed my mind was
that it might be lossy to the user. Elements that has been tried several
ti
Thank Lincoln for the proposal!
The FLIP looks good to me. I'm in favor of the timer based implementation,
and I'd like to share some thoughts.
I'm thinking if we have to store the retry status in the state. I suppose
the retrying requests can just submit as the first attempt when the job
restore
Hi everyone,
By comparing the two internal implementations of delayed retries, we
prefer the timer-based solution, which obtains precise delay control
through simple logic and only needs to pay (what we consider to be
acceptable) timer instance cost for the retry element. The FLIP[1] doc has
b
Hi Jinsong,
Good question!
The delayQueue is very similar to incompleteElements in
UnorderedStreamElementQueue, it only records the references of in-flight
retry elements, the core value is for the ease of a fast scan when force
flush during endInput and less refactor for existing logic.
Users n
Thanks Lincoln for your reply.
I'm a little confused about the relationship between Ordered/Unordered
Queue and DelayQueue. Why do we need to have a DelayQueue?
Can we remove the DelayQueue and put the state of the retry in the
StreamRecordQueueEntry (seems like it's already in the FLIP)
The adva
Hi Jinsong,
Thanks for your feedback! Let me try to answer the two questions:
For q1: Motivation
Yes, users can implement retries themselves based on the external async
client, but this requires each user to do similar things, and if we can
support retries uniformly, user code would become much s
Thank Lincoln for the proposal.
## Motivation:
> asyncInvoke and callback functions are executed synchronously by the main
thread, which is not suitable adding long time blocking operations, and
introducing additional thread will bring extra complexity for users
According to the documentation of
Dear Flink developers,
I would like to open a discussion on FLIP 232 [1], for an extension of
AsyncWaitOperator to support retry for user's asyncFunction.
To do so, new user interface will added to define the trigger condition for
retry and when should retry. Internally, a delayed retry mechanis
22 matches
Mail list logo