...
Code Block | ||||
---|---|---|---|---|
| ||||
public interface AsyncRetryStrategy<T>AsyncRetryStrategy<OUT> extends Serializable { /** whether the next attempt can happen. */ boolean canRetry(int currentAttempts); /** the delay time of next attempt. */ long getBackoffTimeMillis(int currentAttempts); /** the async retry condition defined by an {@link AsyncRetryPredicate}. */ AsyncRetryPredicate<T>AsyncRetryPredicate<OUT> getRetryPredicate(); } |
Code Block |
---|
/** The async retry condition include result {@link Predicate} and exception {@link Predicate}. */ public interface AsyncRetryPredicate<T>AsyncRetryPredicate<OUT> { /** * An Optional Java {@Predicate} that defines a condition on asyncFunction's future result which * will trigger a later reattempt operation, will be called before user's ResultFuture#complete. */ Optional<Predicate<Collection<T>>>Optional<Predicate<Collection<OUT>>> resultPredicate(); /** * An Optional Java {@Predicate} that defines a condition on asyncFunction's exception which * will trigger a later reattempt operation, will be called before user's * ResultFuture#completeExceptionally. */ Optional<Predicate<Throwable>> exceptionPredicate(); } |
...
New method in AsyncDataStream
Code Block | ||||
---|---|---|---|---|
| ||||
@PublicEvolving public interface AsyncRetryStrategy<T> extends Serializable { /** whether the next attempt can* happen. */ boolean canRetry(int currentAttempts); /** the delay time of next attempt. */ long getBackoffTimeMillis(int currentAttempts); /** * An Optional Java {@Predicate} that defines a condition on asyncFunction's future result whichAdd an AsyncWaitOperator with an AsyncRetryStrategy to support retry of AsyncFunction. The order to process input records is guaranteed to be the same as * input ones. * * @param in Input {@link DataStream} * @param func {@link AsyncFunction} * will@param triggertimeout afrom laterfirst reattemptinvoke operation,to willfinal becompletion calledof before user's ResultFuture#complete. */ default Optional<Predicate<Collection<T>>> resultPredicate() {asynchronous operation, may include multiple retries, and will be reset in case of failover * @param timeUnit return Optional.empty(); } /**of the given timeout * @param asyncRetryStrategy {@link AsyncRetryStrategy} for {@link AsyncFunction}. * An@param Optional<IN> JavaType {@Predicate}of thatinput definesrecord a condition on asyncFunction's exception* which @param <OUT> Type of output *record will trigger a later reattempt* operation,@return willA benew called before user's{@link SingleOutputStreamOperator}. * ResultFuture#completeExceptionally./ public */ static <IN, OUT> SingleOutputStreamOperator<OUT> defaultorderedWaitWithRetry( Optional<Predicate<Throwable>> exceptionPredicate() { return Optional.empty(); } } DataStream<IN> in, AsyncFunction<IN, OUT> func, long timeout, TimeUnit timeUnit, AsyncRetryStrategy<OUT> asyncRetryStrategy) // The unorderedWaitWithRetry method is symmetrical. |
Proposed Changes
The current processing flow chart (processEelment & Watermark):
Tried We tried two implementations:
- based on timer trigger (Poc1: https://github.com/lincoln-lil/flink/pull/new/async-retry-timer)
- based on DelayQueue with a pull fashion (Poc2: https://github.com/lincoln-lil/flink/pull/new/async-op-retry)
- trigger
- based on DelayQueue with a pull fashion
and 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.
...
- Add a
Set<RetryableResultHandlerDelegator>
(delayQueue) for in-flight retry elements which stay uncompleted in worker queue (delayQueue is not a real queue, just for fast scan and finish retry when endInput) . - Register a timer for delayed retry and add resultHandlerDelegator to delayQueue if matches retry predicate and satisfies the AsyncRetryStrategy
- When retry time is up, will increment currentAttempts & fire a retry for asyncFunc; When async future complete, will cleanup last retry timer & queue entry if needed.
...
State snapshot and recovery: no changes
2. based on DelayQueue with a pull fashion
The main difference between solution 1. is not rely on the timer service and use a java.concurrent.DelayQueue to save toRetry items (Make StreamRecordQueueEntry retryable by implements Delayed Interface).
And check if any delayed retry element expires and do retry (both in processElement and processWatermark, yes the retry is only triggered by new inputs or watermark, this sacrifice some trigger time precision and some queue check cost, but no new thread here and can reduce timer instances)
The operator open-close and state snapshot-recovery phase are almost the same.
How to deal with retry state and recovery?
Currently we choose not to store the retry state, which is the clearest for users and does not require any new changes to the current recovery behavior.
...
Option
...
RetryState
...
Recovery
...
Pros
...
Cons
...
1. w/o retry state
...
N/A
...
start from scratch
...
simple
...
may result more attempts and longer time for retry elements
...
2. w/ retry state
...
currentAttemps
firstExecTime
...
currentAttemps
new timeout = max{now() - firstExecTime, timeout/maxAttempts}
...
latency first
...
less attemps when restart took too long, maybe a regression when failure rate is non-strong time-dependent
...
3. w/ retry state
...
currentAttemps
costTime
...
currentAttemps
new timeout = timeout - costTime
...
a compromise, compare to 1 & 2
...
if costTime = lastExecTime - firstExecTime, tend to No.1
if costTime = now() - firstExecTime, tend to No.2
How to deal with retry state and recovery?
Currently we choose not to store the retry state, which is the clearest for users and does not require any new changes to the current recovery behavior.
Option | RetryState | Recovery | Pros | Cons |
1. w/o retry state | N/A | start from scratch | simple | may result more attempts and longer time for retry elements |
2. w/ retry state | currentAttemps | currentAttemps | latency first | less attemps when restart took too long, maybe a regression when failure rate is non-strong time-dependent |
3. w/ retry state | currentAttemps | currentAttemps | a compromise, compare to 1 & 2 | if costTime = lastExecTime - firstExecTime, tend to No.1 |
No.2 & No.3 are relatively more complex, and there's a special case (new timeout maybe negative when recovery and need a special process logic) not easy to explain to users clearly.
Changes that users may need to perceive
Note, the new retry feature may result in larger queue capacity requirements, the maximum number can be approximately evaluated as below:
Code Block |
---|
inputRate * retryRate * avgRetryDuration |
For example, for a task with inputRate = 100 records/sec, where 1% of the elements will trigger 1 retry on average, and the average retry time is 60s, the additional queue capacity requirement will be
Code Block |
---|
100 records/sec * 1% * 60s = 60 |
That is, adding more 60 capacity to the work queue may not affect the throughput in unordered output mode , in case of ordered mode, the head element is the key point, and the longer it stays uncompleted, the longer the processing delay provided by the operator, the retry feature may increase the incomplete time of the head element, if in fact more retries are obtained with the same timeout constraint.
When the queue capacity grows(common way to ease the backpressure), the risk of OOM increases. Though in fact, for ListState storage, the theoretical upper limit is Integer.MAX_VALUE, so the queue capacity's limit is the same, but we can't increase the queue capacity too big in production, increase the task parallelism maybe a more viable way.
Compatibility
This change is backwards compatible that jobs can recover from state which was generated by prior flink versions, and if no retry strategy enabled the behavior is as before.
No state changes in AsyncWaitOperator.
Test Plan
Add more test cases in AsyncWaitOperatorTest
Rejected Alternatives
For The Interface
Another choice is combining the two interfaces into one:
Code Block | ||||
---|---|---|---|---|
| ||||
@PublicEvolving
public interface AsyncRetryStrategy<T> extends Serializable {
/** whether the next attempt can happen. */
boolean canRetry(int currentAttempts);
/** the delay time of next attempt. */
long getBackoffTimeMillis(int currentAttempts);
/**
* An Optional Java {@Predicate} that defines a condition on asyncFunction's future result which
* will trigger a later reattempt operation, will be called before user's ResultFuture#complete.
*/
default Optional<Predicate<Collection<T>>> resultPredicate() {
return Optional.empty();
}
/**
* An Optional Java {@Predicate} that defines a condition on asyncFunction's exception which
* will trigger a later reattempt operation, will be called before user's
* ResultFuture#completeExceptionally.
*/
default Optional<Predicate<Throwable>> exceptionPredicate() {
return Optional.empty();
}
} |
For The Changes
Based on DelayQueue with a pull fashion
The main difference between solution 1. is not rely on the timer service and use a java.concurrent.DelayQueue to save toRetry items (Make StreamRecordQueueEntry retryable by implements Delayed Interface).
And check if any delayed retry element expires and do retry (both in processElement and processWatermark, yes the retry is only triggered by new inputs or watermark, this sacrifice some trigger time precision and some queue check cost, but no new thread here and can reduce timer instances)
The operator open-close and state snapshot-recovery phase are almost the same.
No.2 & No.3 are relatively more complex, and there's a special case (new timeout maybe negative when recovery and need a special process logic) not easy to explain to users clearly.
Changes that users may need to perceive
Note, the new retry feature may result in larger queue capacity requirements, the maximum number can be approximately evaluated as below:
Code Block |
---|
inputRate * retryRate * avgRetryDuration |
For example, for a task with inputRate = 100 records/sec, where 1% of the elements will trigger 1 retry on average, and the average retry time is 60s, the additional queue capacity requirement will be
Code Block |
---|
100 records/sec * 1% * 60s = 60 |
That is, adding more 60 capacity to the work queue may not affect the throughput in unordered output mode , in case of ordered mode, the head element is the key point, and the longer it stays uncompleted, the longer the processing delay provided by the operator, the retry feature may increase the incomplete time of the head element, if in fact more retries are obtained with the same timeout constraint.
When the queue capacity grows(common way to ease the backpressure), the risk of OOM increases. Though in fact, for ListState storage, the theoretical upper limit is Integer.MAX_VALUE, so the queue capacity's limit is the same, but we can't increase the queue capacity too big in production, increase the task parallelism maybe a more viable way.
Compatibility
This change is backwards compatible that jobs can recover from state which was generated by prior flink versions, and if no retry strategy enabled the behavior is as before.
No state changes in AsyncWaitOperator.
Test Plan
Add more test cases in AsyncWaitOperatorTest
...
Introduce a memory safe external queue to replace the worker/delay queue
As shown in the processing flow chart above, a delay queue was added to support delayed retry operation. We considered several implementations of the delayed queue and make choice based on two questions:
...