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 property to enable eager building of plan as part of new native plan checker SPI #23649

Open
wants to merge 2 commits into
base: master
Choose a base branch
from

Conversation

BryanCutler
Copy link

@BryanCutler BryanCutler commented Sep 13, 2024

Description

This is a prerequisite for the native plan checker SPI being worked on in #23596 and discussed in https://github.com/prestodb/rfcs/blob/main/RFC-0008-plan-checker.md.

The new property query-execution-fail-fast-enabled will enable the eager building and validation of a logical plan so that any errors or incompatibilities in the plan will cause the query to fail quickly, before cluster resources are assigned and keep queries with invalid plans from holding slots in the queue.

Motivation and Context

The new SPI for a native plan checker, being added in #23596, would use this property so that a plan can be sent to the native sidecar and incompatibles with velox would be caught before actually sending the plan to a native worker to start execution. However, there is also benefit in enabling this property for non-native clusters since all plan checkers are run once the plan is built.

Impact

Test Plan

Manual testing has been done with the property enabled and verified that queries that get queued up will be allowed to be validated and, if fail, removed from the queue before execution begins.

Contributor checklist

  • Please make sure your submission complies with our development, formatting, commit message, and attribution guidelines.
  • PR description addresses the issue accurately and concisely. If the change is non-trivial, a GitHub Issue is referenced.
  • Documented new properties (with its default value), SQL syntax, functions, or other functionality.
  • If release notes are required, they follow the release notes guidelines.
  • Adequate tests were added if applicable.
  • CI passed.

Release Notes

Please follow release notes guidelines and fill in the release notes below.

== RELEASE NOTES ==

General Changes
* Add new property `query-execution-fail-fast-enabled` for eager building of validation of a logical plan. :pr:`23649`

@BryanCutler
Copy link
Author

cc @tdcmeehan @rschlussel

Copy link
Contributor

@steveburnett steveburnett left a comment

Choose a reason for hiding this comment

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

LGTM! (docs)

Pull branch, local doc build, looks good. Thanks!

@jaystarshot
Copy link
Member

The plan is always built and validated before execution

@BryanCutler
Copy link
Author

The plan is always built and validated before execution

@jaystarshot yes they are - I suppose I worded the description poorly. The goal here is to make sure the plan is validated before cluster resources are assigned to run the query. Specifically, with a native cluster when there are plans that are incompatible with Velox, they do not fail until evaluated on a worker. Eagerly checking the plan will allow such failures to happen before running at the point where resources are assigned and also help remove these queries from holding slots in the queue.

@@ -2969,4 +2971,17 @@ public FeaturesConfig setInlineProjectionsOnValues(boolean isInlineProjectionsOn
this.isInlineProjectionsOnValuesEnabled = isInlineProjectionsOnValuesEnabled;
return this;
}

@Config("query-execution-fail-fast-enabled")
Copy link
Contributor

Choose a reason for hiding this comment

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

maybe call this validate-plan-before-queueing? Is that what this is for?

Copy link
Author

@BryanCutler BryanCutler Sep 16, 2024

Choose a reason for hiding this comment

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

Yeah, it could be named better. However, it isn't specific for only queries that have been queued. It starts building/validating the plan as soon as SqlQueryExecution is constructed instead of waiting until SqlQueryExecution.start() is invoked, when ManagedQueryExecution.startWaitingForResources() has already completed.

Perhaps something like eagerly-validate-plan-before-resources ?

Copy link
Contributor

Choose a reason for hiding this comment

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

IIUC, I think queueing will be a more accurate term, because checking for resources happens after queueing.

Copy link
Author

Choose a reason for hiding this comment

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

Sure, my point was that not all queries end up on the queue and might be started right away. In that case this property would still be used and could validate before waiting for resources completes. So I didn't want it to sound like this is only useful if workload has heavy queuing, but if validate-plan-before-queueing is the consensus, it's fine with me.

Copy link
Contributor

Choose a reason for hiding this comment

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

Just explaining my thinking a bit here, rather than thinking of queueing as something that happens to a query, I think of it as a component in Presto. But I see your point as this phrase may not be familiar to folks who are not Presto developers.

What about eager-plan-validation-enabled, and in the documentation we describe it more precisely?

Copy link
Author

Choose a reason for hiding this comment

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

eager-plan-validation-enabled sgtm

@@ -2010,6 +2011,11 @@ public SystemSessionProperties(
"If one input of the cross join is a single row with constant value, remove this cross join and replace with a project node",
featuresConfig.isRemoveCrossJoinWithSingleConstantRow(),
false),
booleanProperty(
QUERY_EXECUTION_FAIL_FAST_ENABLED,
"Enable eager building and validation of logical plan before execution",
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
"Enable eager building and validation of logical plan before execution",
"Enable eager building and validation of logical plan before queueing",

Copy link
Contributor

@tdcmeehan tdcmeehan left a comment

Choose a reason for hiding this comment

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

We could try to write a test for this.

  • We can use the blackhole connector to queue up queries that will never finish, without using a lot resources.
  • One we hit the predetermined queue depth, we then check to see that the next query we send immediately fails.

@@ -243,6 +246,9 @@ private SqlQueryExecution(
}
}
}

// Optionally build and validate plan immediately, before execution begins
planFuture = isQueryExecutionFailFastEnabled(getSession()) ? createLogicalPlanAsync() : null;
Copy link
Contributor

Choose a reason for hiding this comment

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

I'm wondering if we can just inline the createLogicalPlanAsync method, and instead, switch the executor based on the session property.

  • If the flag is disabled, use MoreExecutors.sameThreadExecutor()
  • If the flag is enabled, use a new executor which is injected into this class, and which we have metrics for. Note we define the executor currently used HERE, we could define a new one with more restricted paralellism, and which we could add dedicated metrics for (using a QueryExecutionMBean as you see in the link). One thing we'd need to figure out is how to start the planner in the current thread if the queue were full, without a race, but I think this should be doable.

Copy link
Author

Choose a reason for hiding this comment

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

When flag is enabled, sure I can add a new executor

For when it's disabled, IIUC, using something like MoreExecutors.directExecutor() would cause the plan to be validated immediately in the constructor, which would change the default behavior - I don't think we want to do this. I could still inline createLogicalPlanAsync() if you want to make it more compact?

Copy link
Contributor

Choose a reason for hiding this comment

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

That's a good point, we wouldn't want that! I suppose it should be as you said, if the flag is enabled, submit it to the new executor with restricted parallelism. When we get to line 470, we should try to cancel it without interrupting it--if the cancellation succeeded or if the feature is not enabled, do it in the current thread, otherwise, get() the future.

Copy link
Author

Choose a reason for hiding this comment

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

So when you say try to cancel the future at line 470, is it just in case the ExecutorService hasn't started running the thread, and doing it the current thread would be quicker?

Copy link
Contributor

Choose a reason for hiding this comment

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

Yes, both quicker, and also it won't be subject to the concurrency restraints of the executor it would be running under.

Copy link
Contributor

Choose a reason for hiding this comment

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

also, let's make the executor service parallelism configurable.

Copy link
Author

Choose a reason for hiding this comment

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

ok, good idea

@@ -2969,4 +2971,17 @@ public FeaturesConfig setInlineProjectionsOnValues(boolean isInlineProjectionsOn
this.isInlineProjectionsOnValuesEnabled = isInlineProjectionsOnValuesEnabled;
return this;
}

@Config("query-execution-fail-fast-enabled")
Copy link
Contributor

Choose a reason for hiding this comment

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

IIUC, I think queueing will be a more accurate term, because checking for resources happens after queueing.

@tdcmeehan tdcmeehan self-assigned this Sep 17, 2024
@BryanCutler
Copy link
Author

We could try to write a test for this.
We can use the blackhole connector to queue up queries that will never finish, without using a lot resources.
One we hit the predetermined queue depth, we then check to see that the next query we send immediately fails.

Thanks @tdcmeehan, I can add a test, but is there a way to determine if a query has been queued or what the current queue size is?

@tdcmeehan
Copy link
Contributor

@BryanCutler take a look at TestQueues, which uses the BlackHole connector to deterministically queue, and which uses resource group configuration upfront. If you know that your queries will hang (as the BlackHole connector does), and know the resource group configuration, you can know deterministically when your queries will start to queue.

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.

5 participants