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

Planning time limit #7213

Merged
merged 2 commits into from
Apr 21, 2021
Merged

Conversation

skrzypo987
Copy link
Member

No description provided.

@cla-bot cla-bot bot added the cla-signed label Mar 8, 2021
@skrzypo987 skrzypo987 force-pushed the skrzypo/024-planning-time-limit branch from 6a60698 to 2582129 Compare March 22, 2021 14:04
@skrzypo987
Copy link
Member Author

Added a test that freezes during planning. To my amazement it got interrupted properly. Changed docs so that setting this limit will not guarantee instant termination of the query
Also changed the default to 100 days. Let's keep it backward-compatible and move the default value discussion to subsequent PR.

@findepi
Copy link
Member

findepi commented Mar 23, 2021

move the default value discussion to subsequent PR.

do you plan to make it?

@skrzypo987
Copy link
Member Author

do you plan to make it?

Sure, it is easy. Just need to open a PR and propose an absurd default value. Then a lot of of Trino seniors will come arguing about the actual value and settle with something.
(Disclaimer: That was a joke)

import static java.lang.Integer.MAX_VALUE;
import static org.assertj.core.api.Assertions.assertThatThrownBy;

// Tests relies on timeouts so additional query runners may disturb
Copy link
Member

Choose a reason for hiding this comment

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

what did you mean here?

Copy link
Member Author

Choose a reason for hiding this comment

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

The test needs to finish between 5s and 10s time window. Any task running in the background may increase chance of flakiness.

Copy link
Member

Choose a reason for hiding this comment

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

I think the comment is not complete

@skrzypo987 skrzypo987 added the WIP label Mar 26, 2021
@skrzypo987 skrzypo987 force-pushed the skrzypo/024-planning-time-limit branch from 2582129 to 4b9c92d Compare March 26, 2021 11:29
@skrzypo987 skrzypo987 force-pushed the skrzypo/024-planning-time-limit branch from 4b9c92d to 029c936 Compare March 29, 2021 10:47
@@ -13,6 +13,18 @@ The maximum allowed time for a query to be actively executing on the
cluster, before it is terminated. Compared to the run time below, execution
time does not include analysis, query planning or wait times in a queue.

``query.max-planning-time``
Copy link
Member

Choose a reason for hiding this comment

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

please review @mosabua

Copy link
Contributor

Choose a reason for hiding this comment

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

LGTM.

import static java.lang.Integer.MAX_VALUE;
import static org.assertj.core.api.Assertions.assertThatThrownBy;

// Tests relies on timeouts so additional query runners may disturb
Copy link
Member

Choose a reason for hiding this comment

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

I think the comment is not complete

Thread planningThread = currentThread();
stateMachine.getStateChange(PLANNING).addListener(() -> {
if (stateMachine.getQueryState() == FAILED) {
planningThread.interrupt();
Copy link
Member

Choose a reason for hiding this comment

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

This might not be a planning thread anymore (when planning is finished) when query goes to FAILED state. In fact, this executor thread might be running planning for some other query. I think interruption needs to run in critical section:

  1. planning thread is setting planningThread=null when planning is finished
  2. other thread is calling interrupt

Copy link
Member Author

Choose a reason for hiding this comment

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

The listener will fire only when the state changes from PLANNING to FAILED. Once planning is complete the state changes immediately to STARTING and the thread will not get interrupted.

Copy link
Member Author

Choose a reason for hiding this comment

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

Ok, you are right, the thread might be reused. I will sync it.

Copy link
Member Author

Choose a reason for hiding this comment

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

Fixed. PTAL

@skrzypo987 skrzypo987 force-pushed the skrzypo/024-planning-time-limit branch 2 times, most recently from 4bb9a59 to b6069c5 Compare March 30, 2021 14:27
@mosabua mosabua added the needs-docs This pull request requires changes to the documentation label Mar 30, 2021
@skrzypo987 skrzypo987 force-pushed the skrzypo/024-planning-time-limit branch from b6069c5 to c665258 Compare March 31, 2021 06:43
Copy link
Member

@sopel39 sopel39 left a comment

Choose a reason for hiding this comment

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

lgtm % comments

@@ -284,6 +285,20 @@ public QueryManagerConfig setQueryMaxExecutionTime(Duration queryMaxExecutionTim
return this;
}

@NotNull
@MinDuration("1ns")
Copy link
Member

Choose a reason for hiding this comment

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

nit: why not 0? Duration itself can be 0.

Copy link
Member Author

Choose a reason for hiding this comment

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

Removed the constraint to be consistent with max-execution-time

{
assertThatThrownBy(() -> queryRunner.execute("SELECT * FROM t1 WHERE col = 'abc'"))
.hasMessageContaining("Query exceeded the maximum planning time limit of 5.00s");
assertThat(interrupted).isTrue();
Copy link
Member

Choose a reason for hiding this comment

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

This is racy, planning thread might have hiccup and set interrupted = true later than queryRunner.execute returns.

Use https://docs.oracle.com/javase/7/docs/api/java/util/concurrent/CountDownLatch.html instead.

@@ -129,6 +134,9 @@
private final CostCalculator costCalculator;
private final DynamicFilterService dynamicFilterService;

@Nullable
private Thread planningThread;
Copy link
Member

Choose a reason for hiding this comment

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

Why is this a field in the class?

Copy link
Member Author

Choose a reason for hiding this comment

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

It is used in async lambdas and cannot be a variable. So I could either decorate it in some AtomicReference type of object of make it a field.

Copy link
Member

Choose a reason for hiding this comment

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

It doesn't need to be a field, then. You just need to do:

final Thread planningThread = currentThread();
stateMachine.getStateChange(PLANNING).addListener(() -> {
    if (stateMachine.getQueryState() == FAILED) {
        synchronized (this) {
            planningThread.interrupt();
        }
    }
}, directExecutor());

Copy link
Member Author

Choose a reason for hiding this comment

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

It is being set to null later on

Copy link
Member

Choose a reason for hiding this comment

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

It's not clear to me why we need that. The only reason would be to prevent the wrong thread from being interrupted, but that can't really happen. If the query transitions to FAILED during planning, it must still be executing the start() method, since all the planning methods are synchronous. It is only then that the thread is interrupted, which means it will be the right thread.

I can't think of a combination of actions that would result in start() completing and the query not already be in a state other than PLANNING, which means the interruption wouldn't happen beyond that point.

Copy link
Member Author

Choose a reason for hiding this comment

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

This was implemented to prevent the interruption in an unlikely case of the thread being already reused for another query. This can be simulated by placing long sleep just before the interrupt() method. That way we can interrupt the next query executed in this thread.

Copy link
Member

Choose a reason for hiding this comment

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

State change listener can execute any time in the future (even when planningThread started serving another query)

Copy link
Member

@martint martint Apr 19, 2021

Choose a reason for hiding this comment

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

Fair enough. The use of directExecutor() in the addListener call threw me off.

However, if can happen only if the planning fails from the call to fail() in the catch at the bottom of this method.

Still, I would write this code as follows to keep the state and logic self-contained and not have to reason about a class-level field, it's lifetime beyond this method (or other methods, etc). Ideally, we should be able to remove listeners, but there's no facility to do so:

final AtomicReference<Thread> planningThread = new AtomicReference<>(currentThread());
stateMachine.getStateChange(PLANNING).addListener(() -> {
    if (stateMachine.getQueryState() == FAILED) {
        synchronized (this) {
            Thread thread = planningThread.get();
            if (thread != null) {
                thread.interrupt();
            }
        }
    }
}, directExecutor());

try {
    PlanRoot plan = planQuery();
    // DynamicFilterService needs plan for query to be registered.
    // Query should be registered before dynamic filter suppliers are requested in distribution planning.
    registerDynamicFilteringQuery(plan);
    planDistribution(plan);
}
finally {
    synchronized (this) {
        planningThread.set(null);
        // clear the interrupted flag in case there was a race condition where
        // the planning thread was interrupted right after planning completes above
        Thread.interrupted();
    }
}

Copy link
Member

Choose a reason for hiding this comment

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

However, if can happen only if the planning fails from the call to fail() in the catch at the bottom of this method.

Why only then? Query might be interrupted via timeout and thread.interrupt(); might be delivered in arbitrary time after that.

Other than that keeping logic self-contained within method makes sense

Copy link
Member

Choose a reason for hiding this comment

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

Why only then? Query might be interrupted via timeout and thread.interrupt(); might be delivered in arbitrary time after that.

Ah yes, scratch that. I was thinking of the case where the interrupt causes those methods to throw.

if (stateMachine.getQueryState() == FAILED) {
synchronized (this) {
if (planningThread != null) {
planningThread.interrupt();
Copy link
Member

Choose a reason for hiding this comment

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

How does this relate to the interrupt delivered here


?

Copy link
Member Author

Choose a reason for hiding this comment

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

It is a different thread. queryExecutionFuture.cancel(true) interrupts thread started in LocalDispatchQueryFactory::createDispatchQuery.
The actual planning thread is started from that thread in LocalDispatchQuery::startExecution

Copy link
Member

@sopel39 sopel39 left a comment

Choose a reason for hiding this comment

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

Did you push you changes?

@skrzypo987 skrzypo987 force-pushed the skrzypo/024-planning-time-limit branch from c665258 to 053066b Compare April 6, 2021 11:03
Copy link
Member

@findepi findepi left a comment

Choose a reason for hiding this comment

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

(just skimming)

@skrzypo987 skrzypo987 force-pushed the skrzypo/024-planning-time-limit branch from 053066b to f45d628 Compare April 8, 2021 10:56
@mosabua mosabua removed the needs-docs This pull request requires changes to the documentation label Apr 8, 2021
@mosabua mosabua added the docs label Apr 8, 2021
@skrzypo987
Copy link
Member Author

@martint PTAL if the current state is ok for you

@skrzypo987 skrzypo987 removed the WIP label Apr 15, 2021
@@ -129,6 +134,9 @@
private final CostCalculator costCalculator;
private final DynamicFilterService dynamicFilterService;

@Nullable
private Thread planningThread;
Copy link
Member

Choose a reason for hiding this comment

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

It doesn't need to be a field, then. You just need to do:

final Thread planningThread = currentThread();
stateMachine.getStateChange(PLANNING).addListener(() -> {
    if (stateMachine.getQueryState() == FAILED) {
        synchronized (this) {
            planningThread.interrupt();
        }
    }
}, directExecutor());

@skrzypo987 skrzypo987 force-pushed the skrzypo/024-planning-time-limit branch from f45d628 to 018e9ea Compare April 16, 2021 13:06
@skrzypo987
Copy link
Member Author

@martint I used the planningTime from QueryStateTimer to check the timeout. One commit less and not using executionStartTime. PTAL

Session defaultSession = testSessionBuilder()
.setCatalog("mock")
.setSchema("default")
.setSystemProperty(QUERY_MAX_PLANNING_TIME, "5s")
Copy link
Member

Choose a reason for hiding this comment

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

We need to make this much tighter for this test or come up with a better way to test this.

Copy link
Member Author

Choose a reason for hiding this comment

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

What exactly do you mean?
The feature is about making best effort to stop the planning, so ensuring it works in some cases should be enough IMO

Copy link
Member

Choose a reason for hiding this comment

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

I'm saying that 5s is to long to sleep in a test.

PlanRoot plan = planQuery();
// DynamicFilterService needs plan for query to be registered.
// Query should be registered before dynamic filter suppliers are requested in distribution planning.
registerDynamicFilteringQuery(plan);
planDistribution(plan);

synchronized (this) {
if (Thread.interrupted()) {
Copy link
Member

Choose a reason for hiding this comment

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

This shouldn't be necessary. It should be taken care of by the thread pool.

Copy link
Member

Choose a reason for hiding this comment

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

This thread does not go into pool immediately. This might not be a problem, but it seems cleaner to clear interrupted flag since we interrupt planning specifically. wdyt?

Copy link
Member

Choose a reason for hiding this comment

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

If planning is interrupted before this line, the above calls (planQuery, registerDynamicFilteringQuery, planDistribution) will throw and this will not be reached.

The only case where it can happen is if there is a race condition where the planning completes just about the time the timeout kicks in and interrupts the thread right after planDistribution returns or while it's in some part that doesn't get affected by the interrupt so it doesn't throw. It definitely deserves a comment, and the debug statement is not very useful. Also see my comment above on how I'd restructure this section.

Copy link
Member

Choose a reason for hiding this comment

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

If planning is interrupted before this line, the above calls (planQuery, registerDynamicFilteringQuery, planDistribution) will throw and this will not be reached.

That will only happen if these methods will throw interrupted exception on IO which might not be the case.

Also see my comment above on how I'd restructure this section.

+1

Copy link
Member

Choose a reason for hiding this comment

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

That will only happen if these methods will throw interrupted exception on IO which might not be the case.

That should not happen unless the code can do something reasonable with a lack of result from that IO. If we're silently ignoring those interruptions and making results up, that's a bigger problem.

@@ -129,6 +134,9 @@
private final CostCalculator costCalculator;
private final DynamicFilterService dynamicFilterService;

@Nullable
private Thread planningThread;
Copy link
Member

Choose a reason for hiding this comment

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

It's not clear to me why we need that. The only reason would be to prevent the wrong thread from being interrupted, but that can't really happen. If the query transitions to FAILED during planning, it must still be executing the start() method, since all the planning methods are synchronous. It is only then that the thread is interrupted, which means it will be the right thread.

I can't think of a combination of actions that would result in start() completing and the query not already be in a state other than PLANNING, which means the interruption wouldn't happen beyond that point.

@skrzypo987 skrzypo987 force-pushed the skrzypo/024-planning-time-limit branch from 018e9ea to a153e60 Compare April 20, 2021 07:28
Copy link
Member

@martint martint left a comment

Choose a reason for hiding this comment

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

Looks good, but let's change the default max planning time to something more reasonable (e.g., 10 minutes). 100 days is meaningless -- we might as well make it optional and default to no limit specified.

@skrzypo987 skrzypo987 force-pushed the skrzypo/024-planning-time-limit branch 3 times, most recently from c3d8789 to 2c495b7 Compare April 21, 2021 08:09
@sopel39
Copy link
Member

sopel39 commented Apr 21, 2021

mind automation

@skrzypo987
Copy link
Member Author

There are some problems with GHA at the moment, I will restart it later

skrzypo987 added 2 commits April 21, 2021 11:14
When the query state changes from PLANNING to FAILED,
due to error or explicit cancel, the planning thread will get
interrupted possibly freeing some of the resources.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Development

Successfully merging this pull request may close these issues.

7 participants