Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Handle interruption in RetryDriver and TaskRunner #18964

Merged

Conversation

pettyjamesm
Copy link
Contributor

Cross port of trinodb/trino#15803 with one additional commit to fix handling of S3 client exceptions when interrupted. At a high level, this PR:

  1. Adds AbortedException to the list of RetryDriver#stopOn exceptions for all S3 client operations in PrestoS3FileSystem and S3SelectLineRecordReader. When the S3 client receives an InterruptedException internally during an API call operation, it re-sets the current threads interrupted flag, but re-throws an AbortedException. When this occurs, the retry driver should stop attempting retries.
  2. Adds logic to RetryDriver exception handling to stop retries when an InterruptedException is caught or Thread.currentThread().isInterrupted(). Failing to do so could result in drivers that were interrupted as part of task cancellation running significantly longer than necessary by proceeding to retry / backoff instead of exiting.
  3. Changes TaskRunner to continue processing new splits instead of terminating when the current thread was interrupted as a result of task cancellation interrupting the current driver being processed without TaskExecutor having been shut down. Before this change, those interruptions would cause the TaskRunner to stop processing new splits and submit a new TaskRunner into the cached threadpool executor, which could needlessly create new worker threads.
== NO RELEASE NOTE ==

@pettyjamesm pettyjamesm requested a review from a team as a code owner January 24, 2023 14:58
@pettyjamesm pettyjamesm force-pushed the improve-interrupted-handling branch from 8df783b to e05a499 Compare January 24, 2023 17:30
Copy link
Contributor

@ajaygeorge ajaygeorge left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Some comments

@pettyjamesm pettyjamesm force-pushed the improve-interrupted-handling branch 2 times, most recently from 2285985 to 394f561 Compare January 31, 2023 23:47
Copy link
Contributor

@ajaygeorge ajaygeorge left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

When a thread is interrupted during an S3 client operation the SDK
client internally catches the InterruptedException, sets the thread back
to interrupted, and throws an AbortedException instead of
InterruptedException. When this occurs, the RetryDriver should stop
attempting retries. This change adds AbortedException to the stopOn list
for all retry drivers in PrestoS3FileSystem and S3SelectLineRecordReader
Stop attempting retries in Hive's RetryDriver if an InterruptedException
is caught or when the current thread is interrupted and immediately throw
the current exception. Otherwise, RetryDriver might interfere with drivers
terminating in a timely manner after tasks are terminated via a
cancel, failure, or abort.
Drivers may leave the TaskRunner thread's interrupt flag set during the
course of processing, but doing so should not result in the TaskRunner
terminating its own processing loop until the TaskExecutor is closed.

Instead of allowing the interrupt to terminate the current task runner's
loop and re-creating a new runner to replace the interrupted one, we can
clear the interrupt flag after each iteration. Otherwise, TaskRunners
that were interrupted would have to replace themselves and end up
creating unnecessary threads in the cached threadpool executor in the
process.
@pettyjamesm pettyjamesm force-pushed the improve-interrupted-handling branch from 394f561 to db99445 Compare February 2, 2023 20:38
@pettyjamesm pettyjamesm merged commit 6c4459c into prestodb:master Feb 3, 2023
@pettyjamesm pettyjamesm deleted the improve-interrupted-handling branch February 3, 2023 16:08
@@ -139,6 +139,11 @@ public <V> V run(String callableName, Callable<V> callable)
return callable.call();
}
catch (Exception e) {
// Immediately stop retry attempts once an interrupt has been received
if (e instanceof InterruptedException || Thread.currentThread().isInterrupted()) {

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry for the late comment, Checking the isInterrupted will clear the interrupt flag on the thread, is that expected ? I am not sure who is responsible for clearing the interrupt flag in the Presto codebase.

Sleep on line 163, tries to rightly set the interrupted flag.

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My bad, isInterrupted does not clear the flag, so it is good. but on handling the interruptedException, there are still differences.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We’re rethrowing InterruptedException (or whatever other exception was thrown if the current thread is interrupted) but not “handing” it other than to ensure that we stop retrying and let the exception propagate.

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks this makes sense

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants