Status
Current state: Under Discussion
...
Page properties | |||
---|---|---|---|
|
...
...
JIRA:
...
|
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
...
Code Block | ||
---|---|---|
| ||
DataStream<String> input = ... // User's HBaseAsyncFunc doesn't change. AsyncFunction<String, Tuple<String, String>> asyncFunc = new HBaseAsyncFunc... // Create an async retry strategy via utility class or a user defined strategy. AsyncRetryStrategy asyncRetryStrategy = new AsyncRetryStrategies.FixedDelayRetryStrategyBuilder(3, 100L) // maxAttempts=3, fixedDelay=100ms .retryIfResult(RetryPredicates.EMPTY_RESULT_PREDICATE) .retryIfException(RetryPredicates.HAS_EXCEPTION_PREDICATE) .build(); // ordered output mode AsyncDataStream.orderedWaitWithRetyorderedWaitWithRetry(input, asyncFunc, timeout, TimeUnit.MILLISECONDS, 100, asyncRetryStrategy); // or use unordered output mode |
...
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 { /** /** whetherAdd thean nextAsyncWaitOperator attemptwith canan happen. */ boolean canRetry(int currentAttempts); /** the delay time of next attempt. */AsyncRetryStrategy to support retry of AsyncFunction. The order to process input records is guaranteed to be the same as long getBackoffTimeMillis(); * input ones. /** * An@param Optionalin JavaInput {@Predicate@link DataStream} that defines a condition on* asyncFunction's@param futurefunc result{@link whichAsyncFunction} * will trigger a later reattempt operation, @param timeout from first invoke to final completion of asynchronous operation, may include multiple retries, and will be calledreset in beforecase user's ResultFuture#complete.of failover */ @param timeUnit of defaultthe Optional<Predicate<Collection<T>>> resultPredicate() { return Optional.empty();given timeout * @param asyncRetryStrategy {@link AsyncRetryStrategy} for {@link AsyncFunction}. } /** * @param <IN> Type of input record * An Optional Java {@Predicate} that defines a condition on asyncFunction's exception which @param <OUT> Type of output record * @return A new {@link SingleOutputStreamOperator}. */ will trigger a laterpublic reattemptstatic operation<IN, willOUT> be called before user'sSingleOutputStreamOperator<OUT> orderedWaitWithRetry( * ResultFuture#completeExceptionally. */ DataStream<IN> in, AsyncFunction<IN, defaultOUT> Optional<Predicate<Throwable>> exceptionPredicate() { func, long timeout, TimeUnit timeUnit, AsyncRetryStrategy<OUT> asyncRetryStrategy) return Optional.empty(); } } |
Proposed Changes
The current processing flow chart (processEelment & Watermark):
Tried two implementations (will update the benchmark result later for proper choice):
- 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)
1. based on timer trigger
// The unorderedWaitWithRetry method is symmetrical. |
Proposed Changes
The main processing flow after retry added:
...
- Add a
Set<RetryableResultHandlerDelegator>
(delayQueue) for in-flight retry elements which stay uncompleted in worker queue.Add 3 attributes (currentAttempts, startTime, nextBackOffTime) to StreamRecordQueueEntry for retry statequeue (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:
- When snapshot, write all queue entry with current retry status (currentAttempts, startTime, nextBackOffTime) to the state
- When recover from state, restore retry status and fire a new process if necessary
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)
...
no changes
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
...
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.
A new list state with async attempt status was added in AsyncWaitOperator to restore the retry after recoveryNo state changes in AsyncWaitOperator.
Test Plan
Add more test cases in AsyncWaitOperatorTest
Rejected Alternatives
We tried two implementations:
- based on timer trigger (as described above)
- based on DelayQueue with a pull fashion
and 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.
Comparation of the two implementations:
Solution | Pros | Cons |
based on timer | 1. simple | 1. rely on timer service and more timer instances |
based on DelayQueue | 1. no timer instance | 1. rely on java DelayQueue and Slightly more complex |
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.
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:
...