rkhachatryan commented on a change in pull request #18931:
URL: https://github.com/apache/flink/pull/18931#discussion_r816821759
##########
File path:
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/RetryingExecutor.java
##########
@@ -151,18 +193,29 @@ public void run() {
}
private void handleError(Exception e) {
- LOG.info("execution attempt {} failed: {}", current,
e.getMessage());
- // prevent double completion in case of a timeout and another
failure
- boolean attemptTransition = attemptCompleted.compareAndSet(false,
true);
- if (attemptTransition && !actionCompleted.get()) {
- long nextAttemptDelay = retryPolicy.retryAfter(current, e);
- if (nextAttemptDelay == 0L) {
- executorService.submit(next());
- } else if (nextAttemptDelay > 0L) {
- executorService.schedule(next(), nextAttemptDelay,
MILLISECONDS);
- } else {
- actionCompleted.set(true);
- }
+ if (!attemptCompleted.compareAndSet(false, true) ||
actionCompleted.get()) {
+ // either this attempt was already completed (e.g. timed out);
+ // or another attempt completed the task
+ return;
+ }
+ LOG.debug("execution attempt {} failed: {}", current,
e.getMessage());
+ long nextAttemptDelay = retryPolicy.retryAfter(current, e);
+ if (nextAttemptDelay >= 0L) {
+ activeAttempts.incrementAndGet();
+ scheduleNext(nextAttemptDelay, next());
+ }
+ if (activeAttempts.decrementAndGet() == 0
+ && actionCompleted.compareAndSet(false, true)) {
+ LOG.info("failed with {} attempts: {}", current,
e.getMessage());
Review comment:
The exception should be propagated all the way to the
`CheckpointCoordinator`; `AsyncCheckpointRunnable` will also log the full
exception.
Here, I only wanted to make debugging easier by logging the number of
attempts and the exact timestamp. Having one more place with a stack trace
would pollute the logs. WDYT?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]