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

Add thread timeout that interrupts query analyzer thread #17650

Merged
merged 1 commit into from
Apr 19, 2022

Conversation

vanekjar
Copy link
Contributor

@vanekjar vanekjar commented Apr 14, 2022

It was discovered that runaway regex can run for very long or even end up in infinite loop and block the query analyzer thread. The fix introduces a thread timeout that interrupts the runaway regex matcher after a certain timeout.

Example of a query that can cause issues:

select REGEXP_EXTRACT('runaway_regex-is-evaluated-infinitely - xxx"}', '.*runaway_(.*?)+-+xxx.*')

Test plan - unit test

== RELEASE NOTES ==

General Changes
* Add support to set timeout on the query analyzer runtime to prevent long running query analysis like complex regular expressions. The timeout duration can be set by the configuration property `planner.query-analyzer-timeout` or session property `query_analyzer_timeout`.

@linux-foundation-easycla
Copy link

linux-foundation-easycla bot commented Apr 14, 2022

CLA Signed

The committers listed above are authorized under a signed CLA.

  • ✅ login: vanekjar / name: Jaromir Vanek (66d0456dab2bdd7bcb179ce263fd76a1f3d44cc2)

@vanekjar vanekjar force-pushed the thread-watchdog branch 2 times, most recently from 1d32620 to 908b40d Compare April 14, 2022 23:23
@aweisberg aweisberg self-requested a review April 14, 2022 23:56
Copy link
Contributor

@aweisberg aweisberg left a comment

Choose a reason for hiding this comment

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

This is great!

The release note should mention the actual configuration property, also want it in the docs. I think the layout there doesn't present a good location for this. Maybe add a Planner section for this? I think the name might also need some rethinking, but I am not 100% sure what it should be.

For the PR description. It's not just REGEXP_EXTRACT it's generally possible to write a regular expression and use several functions. I would just say regular expressions can end up being very long running or infinite.

@@ -224,6 +224,8 @@

private double hyperloglogStandardErrorWarningThreshold = 0.004;

private Duration queryAnalyzerTimeout = new Duration(30, SECONDS);
Copy link
Contributor

Choose a reason for hiding this comment

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

The query analyzer is basically the whole planner and there is a 3 minute timeout on the iterative optimizer. I would say this should default to the same as the iterative optimizer timeout otherwise the cases where the iterative optimizer takes a long time will also get canceled/interrupt.

*/
public ThreadWatchdog(Thread thread, ScheduledExecutorService executor, Duration timeout)
{
this.thread = requireNonNull(thread);
Copy link
Contributor

Choose a reason for hiding this comment

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

We always provide a message saying which parameter was null.

private void start()
{
this.future = executor.schedule(
thread::interrupt, timeout.toMillis(), TimeUnit.MILLISECONDS);
Copy link
Contributor

Choose a reason for hiding this comment

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

In j.u.c usage you normally always do nanoseconds to allow the highest precision timings to flow through. It doesn't matter here though.

@@ -356,7 +360,10 @@ public int getRunningTaskCount()
@Override
public void start()
{
try (SetThreadName ignored = new SetThreadName("Query-%s", stateMachine.getQueryId())) {
try (SetThreadName ignored = new SetThreadName("Query-%s", stateMachine.getQueryId());
ThreadWatchdog w = new ThreadWatchdog(Thread.currentThread(),
Copy link
Contributor

Choose a reason for hiding this comment

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

I think it's less risky to just scope this to the query analyzer and not all the other methods including starting the scheduler.

That said there might be value in being able to interrupt the entire thing. @highker WDYT?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I assumed the thread may get stuck anywhere, not just the specific discovered case. But considering I named the property query_analyzer_timeout, it probably makes sense to wrap only the analyzer part.

try (SetThreadName ignored = new SetThreadName("Query-%s", stateMachine.getQueryId());
ThreadWatchdog w = new ThreadWatchdog(Thread.currentThread(),
watchdogExecutor,
SystemSessionProperties.getQueryAnalyzerTimeout(getSession()))) {
Copy link
Contributor

Choose a reason for hiding this comment

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

We static import these SystemSessionProperties methods

{
private final Thread thread;
private final ScheduledExecutorService executor;
private final Duration timeout;
Copy link
Contributor

Choose a reason for hiding this comment

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

This class doesn't need any of these since starting is implicit in construction. I would just remove the start method and inline creating the task.

public static ScheduledExecutorService createWatchdogExecutor()
{
ScheduledThreadPoolExecutor executor =
new ScheduledThreadPoolExecutor(1, threadsNamed("thread-watchdog"));
Copy link
Contributor

Choose a reason for hiding this comment

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

Use daemonThreadsNamed

@aweisberg aweisberg requested a review from highker April 15, 2022 00:20
@vanekjar
Copy link
Contributor Author

vanekjar commented Apr 15, 2022

Thanks for the quick review. I fixed all the comments in the code and updated the doc. Please let me know if I matched the wording correctly.

@vanekjar vanekjar force-pushed the thread-watchdog branch 2 times, most recently from 43d5ee2 to 7652399 Compare April 15, 2022 01:28
Copy link
Contributor

@highker highker left a comment

Choose a reason for hiding this comment

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

  • Can we squash the commits into one? Note that we do encourage have multiple small commits. But each commit should be logically self-contained. For this change, one commit is good enough.
  • Can we wrap the commit title and message up to 72 characters per line? Please follow the commit guideline here: https://cbea.ms/git-commit/

* ThreadWatchdog spins up a background task
* that interrupts given thread after the timeout expires.
*/
public class ThreadWatchdog
Copy link
Contributor

Choose a reason for hiding this comment

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

Let's call it TimeoutThread. Same for all other places in this PR. Watchdog doesn't sound very formal.

Comment on lines 44 to 45
this.future = executor.schedule(
thread::interrupt, timeout.roundTo(NANOSECONDS), NANOSECONDS);
Copy link
Contributor

Choose a reason for hiding this comment

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

put into the same line

@Retention(RUNTIME)
@Target({FIELD, PARAMETER, METHOD})
@Qualifier
public @interface ForWatchdog
Copy link
Contributor

Choose a reason for hiding this comment

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

ForTimeoutThread; same for everywhere else

PlanRoot plan;

// setup a thread watchdog in case query analyzer ends up in an infinite loop
try (ThreadWatchdog w = new ThreadWatchdog(Thread.currentThread(),
Copy link
Contributor

Choose a reason for hiding this comment

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

  • move Thread.currentThread() to its own line. We leave the first line empty for function parameters if too long.
  • instead of calling it w, we could call it unused or thread. Usually we spell the full name for variables and definitions

Comment on lines 386 to 387
ScheduledThreadPoolExecutor executor =
new ScheduledThreadPoolExecutor(1, daemonThreadsNamed("thread-watchdog"));
Copy link
Contributor

Choose a reason for hiding this comment

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

  • put into the same line
  • if we only have one thread, would that cause starvation if some bad thread blocks the legit ones?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

if we only have one thread, would that cause starvation if some bad thread blocks the legit ones?

TimeoutThread only executes Thread#interrupt and finishes. It shouldn't get blocked. I believe it's safer than spawning large amount of threads by mistake.

Comment on lines 1120 to 1121
@ConfigDescription("Maximum running time for the query analyzer in case" +
"the processing takes too long or is stuck in an infinite loop.")
Copy link
Contributor

Choose a reason for hiding this comment

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

put into same line

Comment on lines 336 to 337
@Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = "Regexp matching interrupted",
timeOut = 30_000)
Copy link
Contributor

Choose a reason for hiding this comment

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

put into the same line; or if it's too long, keep the first line empty and then one param per line

@vanekjar vanekjar changed the title Add thread watchdog that interrupts query analyzer thread Add thread timeout that interrupts query analyzer thread Apr 19, 2022
@vanekjar vanekjar force-pushed the thread-watchdog branch 3 times, most recently from 36bfe61 to 64794a3 Compare April 19, 2022 17:51
@vanekjar
Copy link
Contributor Author

@highker Thank you for your thorough review and guidance. I updated the PR accordingly.

@highker highker self-requested a review April 19, 2022 19:00
Copy link
Contributor

@highker highker left a comment

Choose a reason for hiding this comment

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

Thanks for the contribution. The test failure is related. Please take a look

@@ -194,7 +194,8 @@ public void testDefaults()
.setMaxStageCountForEagerScheduling(25)
.setHyperloglogStandardErrorWarningThreshold(0.004)
.setPreferMergeJoin(false)
.setRoundRobinShuffleBeforePartialDistinctLimit(false));
.setRoundRobinShuffleBeforePartialDistinctLimit(false)
.setQueryAnalyzerTimeout(new Duration(30, SECONDS)));
Copy link
Contributor

Choose a reason for hiding this comment

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

This will fail the test

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thanks for the heads up. Just fixed.

@highker highker self-assigned this Apr 19, 2022
It was discovered that runaway regex can run for very long or even
end up in infinite loop and block the query analyzer thread. The fix
introduces a thread timeout that interrupts the runaway regex matcher
after a certain timeout.
@highker highker merged commit 1660a27 into prestodb:master Apr 19, 2022
@vanekjar vanekjar deleted the thread-watchdog branch April 19, 2022 22:47
@mshang816 mshang816 mentioned this pull request May 17, 2022
14 tasks
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.

3 participants