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

Enforce query timeout on query compilation in the multi-stage query engine #14946

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

Conversation

yashmayya
Copy link
Collaborator

  • In some rare scenarios, query compilation itself can take a very long time in the multi-stage query engine (see Multi-stage: Perf issue when IN expression has a lot of entries #13617 for example).
  • In these cases, the query timeout is not enforced, and the response can take arbitrarily long.
  • This patch fixes the issue by enforcing the query timeout on the compilation phase as well. Subsequent operations like query dispatch and reduce will operate with the remaining timeout duration.

@yashmayya yashmayya added enhancement multi-stage Related to the multi-stage query engine labels Jan 30, 2025
@yashmayya yashmayya requested a review from gortiz January 30, 2025 10:21
Comment on lines 189 to 191
Future<QueryEnvironment.QueryPlannerResult> queryPlanResultFuture = _queryCompileExecutor.submit(
() -> queryEnvironment.planQuery(query, sqlNodeAndOptions, requestId));

Copy link
Contributor

Choose a reason for hiding this comment

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

we need to do the same with explain plan

LOGGER.warn("Caught exception planning request {}: {}, {}", requestId, query, consolidatedMessage);
_brokerMetrics.addMeteredGlobalValue(BrokerMeter.REQUEST_COMPILATION_EXCEPTIONS, 1);
if (e.getMessage().matches(".* Column .* not found in any table'")) {
if (t.getMessage().matches(".* Column .* not found in any table'")) {
Copy link
Contributor

Choose a reason for hiding this comment

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

just an error I've found: t.getMessage may be null. Given you are touching this line, can you also modify this line to check if the message is null?

@@ -112,6 +118,7 @@ public MultiStageBrokerRequestHandler(PinotConfiguration config, String brokerId
CommonConstants.MultiStageQueryRunner.KEY_OF_MULTISTAGE_EXPLAIN_INCLUDE_SEGMENT_PLAN,
CommonConstants.MultiStageQueryRunner.DEFAULT_OF_MULTISTAGE_EXPLAIN_INCLUDE_SEGMENT_PLAN);
_queryThrottler = queryThrottler;
_queryCompileExecutor = Executors.newCachedThreadPool(new NamedThreadFactory("query-compile-executor"));
Copy link
Contributor

Choose a reason for hiding this comment

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

I was fixing this issue myself, and I was just thinking about the executor. I was thinking of reusing another executor (because we already have too many of them!), but I didn't find a clear executor to use. Anyway, I don't think using a cached thread pool is a good idea. Wouldn't it be better to use a fixed number of threads?

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

There's a cached thread pool executor in BrokerAdminApiApplication that could be moved up to BaseBrokerStarter and then shared between the broker request handler and the admin api application. I'm not sure I agree on cached thread pools being a bad idea here though. Isn't it a great use case for one? We have many short lived tasks that are executed with timeouts here (we're also cancelling them if timeouts are exceeded but I'm not sure how Calcite code handles these interrupts in the planning / optimization phase). Why would it be better to use a fixed thread pool here?

Copy link
Collaborator Author

@yashmayya yashmayya Jan 30, 2025

Choose a reason for hiding this comment

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

Also, we're using Jersey's ManagedAsync annotation on the API resource methods that call these broker request handler methods. So they're being executed on threads from the Jersey managed internal executor service. If that's a fixed thread pool, I suppose we could make ours similar as well (because we don't need any more threads than the number of request processing threads) but I'm not able to find more info about that one.

Copy link
Contributor

@gortiz gortiz Jan 31, 2025

Choose a reason for hiding this comment

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

My reason for using a fixed number of threads is that optimizing a query is a purely CPU-bound task. What is the advantage of having more threads than CPUs in this case? In fact, I would use fewer threads than CPUs.

Imagine a situation where users are running tons of queries that are expensive to parse. In that case, having a large number of threads running optimizations would be expensive and make the system unresponsive. In the same case, using a fixed thread pool smaller than the number of CPUs would queue up queries, but other parts of the system shouldn't be affected.

Remember that Jersey can easily spawn hundreds of threads because it assumes most work will be blocking. And IIRC, the new MSE query throttling mechanism is applied after the query is optimized.

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

Thanks for the explanation, those are very convincing arguments! I've updated this to be a fixed thread pool executor with the number of threads being numCores / 2.

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

Imagine a situation where users are running tons of queries that are expensive to parse. In that case, having a large number of threads running optimizations would be expensive and make the system unresponsive. In the same case, using a fixed thread pool smaller than the number of CPUs would queue up queries, but other parts of the system shouldn't be affected.

My main concern is potential increase in query latency due to this queueing but I think the only scenario where that would occur is the one you said with tons of queries that are super expensive to compile / optimize - and in this scenario, I think stability is anyway going to be more important to maintain than query latency.

@codecov-commenter
Copy link

codecov-commenter commented Jan 30, 2025

Codecov Report

Attention: Patch coverage is 4.16667% with 23 lines in your changes missing coverage. Please review.

Project coverage is 63.69%. Comparing base (59551e4) to head (b825874).
Report is 1654 commits behind head on master.

Files with missing lines Patch % Lines
...requesthandler/MultiStageBrokerRequestHandler.java 4.16% 23 Missing ⚠️
Additional details and impacted files
@@             Coverage Diff              @@
##             master   #14946      +/-   ##
============================================
+ Coverage     61.75%   63.69%   +1.94%     
- Complexity      207     1480    +1273     
============================================
  Files          2436     2712     +276     
  Lines        133233   152071   +18838     
  Branches      20636    23496    +2860     
============================================
+ Hits          82274    96864   +14590     
- Misses        44911    47919    +3008     
- Partials       6048     7288    +1240     
Flag Coverage Δ
custom-integration1 100.00% <ø> (+99.99%) ⬆️
integration 100.00% <ø> (+99.99%) ⬆️
integration1 100.00% <ø> (+99.99%) ⬆️
integration2 0.00% <ø> (ø)
java-11 63.67% <4.16%> (+1.97%) ⬆️
java-21 63.57% <4.16%> (+1.94%) ⬆️
skip-bytebuffers-false 63.69% <4.16%> (+1.94%) ⬆️
skip-bytebuffers-true 63.54% <4.16%> (+35.81%) ⬆️
temurin 63.69% <4.16%> (+1.94%) ⬆️
unittests 63.69% <4.16%> (+1.94%) ⬆️
unittests1 56.23% <ø> (+9.34%) ⬆️
unittests2 34.01% <4.16%> (+6.27%) ⬆️

Flags with carried forward coverage won't be shown. Click here to find out more.

☔ View full report in Codecov by Sentry.
📢 Have feedback on the report? Share it here.

@yashmayya yashmayya requested a review from gortiz February 7, 2025 05:16
@gortiz
Copy link
Contributor

gortiz commented Feb 7, 2025

Approved. There are some conflicts that prevent the merge

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
enhancement multi-stage Related to the multi-stage query engine
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants