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

[New Feature] Add new configuration options which allows broker to use a bounded Jersey ThreadPool #10614

Merged
merged 4 commits into from
Apr 18, 2023

Conversation

MeihanLi
Copy link
Contributor

@MeihanLi MeihanLi commented Apr 14, 2023

[New Feature] Add new configuration options which allows broker to use a bounded Jersey ThreadPool

Currently Broker uses Jersey default unbounded thread pool to process async requests and it uses No-Op RejectedExecutionHandler.

When a broker is serving very high QPS, a significant amount of threads will be accumulated on the broker. We have seen ~25k threads accumulated in a short time because of this and brokers were taken down very quickly. This is also the root cause of #9019.

This PR
(1) Implements a custom provider BrokerManagedAsyncExecutorProvider by extending ThreadPoolExecutorProvider, which will be automatically picked by Jersey. When async requests can not be accommodated, they will get rejected and customers will receive ERROR code 503 (Service Unavailable).

(2) Add new configuration options below which allow us to use the bounded thread pool and allocate capacities for it. By default, it is disabled.
pinot.broker.enable.bounded.http.async.executor
pinot.broker.http.async.executor.max.pool.size
pinot.broker.http.async.executor.core.pool.size
pinot.broker.http.async.executor.queue.size

By enabling the new configs, endpoints POST /query/sql and GET /query/sql will be impacted.

Test

Tested locally, one example response for a rejected request is

HTTP/1.1 503 Service Unavailable
Content-Type: application/json
Connection: close
Content-Length: 220

Pinot Broker thread pool can not accommodate more requests now. Request is rejected from java.util.concurrent.ThreadPoolExecutor@79c8b52f[Running, pool size = 1, active threads = 1, queued tasks = 1, completed tasks = 6]

The error log emitted by BrokerManagedAsyncExecutorProvider is:
2023/04/14 16:47:26.146 ERROR [BrokerManagedAsyncExecutorProvider] [grizzly-http-server-15] Task java.util.concurrent.FutureTask@dafa6aa[Not completed, task = java.util.concurrent.Executors$RunnableAdapter@74af7b05[Wrapped task = org.glassfish.jersey.server.ServerRuntime$AsyncResponder$2@58e4de25]] rejected from java.util.concurrent.ThreadPoolExecutor@1bebdf0b[Running, pool size = 1, active threads = 1, queued tasks = 1, completed tasks = 6]

@MeihanLi MeihanLi changed the title add an bounded thead pool for brokers [release-notes][bugfix] Add a new configuration which allows broker to use a bounded Jersey ThreadPool Apr 14, 2023
@MeihanLi MeihanLi changed the title [release-notes][bugfix] Add a new configuration which allows broker to use a bounded Jersey ThreadPool [Feature] Add a new configuration which allows broker to use a bounded Jersey ThreadPool Apr 14, 2023
@MeihanLi MeihanLi changed the title [Feature] Add a new configuration which allows broker to use a bounded Jersey ThreadPool [New Feature] Add a new configuration which allows broker to use a bounded Jersey ThreadPool Apr 14, 2023
@MeihanLi MeihanLi marked this pull request as draft April 14, 2023 07:27
@MeihanLi MeihanLi changed the title [New Feature] Add a new configuration which allows broker to use a bounded Jersey ThreadPool [New Feature] Add new configuration options which allows broker to use a bounded Jersey ThreadPool Apr 14, 2023
@codecov-commenter
Copy link

codecov-commenter commented Apr 14, 2023

Codecov Report

Merging #10614 (827c537) into master (13a792f) will increase coverage by 1.36%.
The diff coverage is 70.58%.

@@             Coverage Diff              @@
##             master   #10614      +/-   ##
============================================
+ Coverage     69.03%   70.40%   +1.36%     
+ Complexity     6499     5629     -870     
============================================
  Files          2106     2106              
  Lines        113010   113674     +664     
  Branches      17027    17216     +189     
============================================
+ Hits          78015    80029    +2014     
+ Misses        29515    28050    -1465     
- Partials       5480     5595     +115     
Flag Coverage Δ
integration1 24.74% <8.82%> (?)
integration2 24.08% <8.82%> (+0.03%) ⬆️
unittests1 67.88% <100.00%> (-0.15%) ⬇️
unittests2 14.03% <58.82%> (+0.12%) ⬆️

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

Impacted Files Coverage Δ
...pinot/broker/broker/BrokerAdminApiApplication.java 78.48% <20.00%> (-8.48%) ⬇️
...ker/broker/BrokerManagedAsyncExecutorProvider.java 90.00% <90.00%> (ø)
...a/org/apache/pinot/common/metrics/BrokerMeter.java 100.00% <100.00%> (ø)
...va/org/apache/pinot/spi/utils/CommonConstants.java 27.05% <100.00%> (+2.66%) ⬆️

... and 183 files with indirect coverage changes

📣 We’re building smart automated test selection to slash your CI/CD build times. Learn more

@MeihanLi MeihanLi marked this pull request as ready for review April 14, 2023 17:16
@MeihanLi MeihanLi force-pushed the bounded_broker_threadPool branch from 2729796 to cc7b404 Compare April 14, 2023 23:32
Copy link
Contributor

@ankitsultana ankitsultana left a comment

Choose a reason for hiding this comment

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

Thanks Christina for addressing all the feedback!

@MeihanLi MeihanLi force-pushed the bounded_broker_threadPool branch from 79e7221 to 04a4a2b Compare April 17, 2023 17:14
Runtime.getRuntime().availableProcessors() * 2;
public static final String CONFIG_OF_HTTP_ASYNC_EXECUTOR_QUEUE_SIZE =
"pinot.broker.http.async.executor.queue.size";
public static final int DEFAULT_HTTP_ASYNC_EXECUTOR_QUEUE_SIZE =
Copy link
Contributor

Choose a reason for hiding this comment

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

queue size actually is bounded by memory. but I think availableProcessors() * 2 is OK.

Copy link
Contributor

@jasperjiaguo jasperjiaguo Apr 17, 2023

Choose a reason for hiding this comment

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

Would the default queue size of availableProcessors() * 2 be too small? During spiky request we may drop requests prematurely?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Hi Jasper, I didn't test the default values so they might be small. Can you share what might be a better number here?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@jasperjiaguo I set the default value to Integer.MAX_VALUE, let me know if this looks good to you. The change is in the last commit.

Copy link
Contributor

Choose a reason for hiding this comment

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

Yes that looks good to me. Thanks!

@Jackie-Jiang Jackie-Jiang added release-notes Referenced by PRs that need attention when compiling the next release notes Configuration Config changes (addition/deletion/change in behavior) labels Apr 17, 2023
Copy link
Contributor

@Jackie-Jiang Jackie-Jiang left a comment

Choose a reason for hiding this comment

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

LGTM otherwise

Copy link
Contributor

@jasperjiaguo jasperjiaguo left a comment

Choose a reason for hiding this comment

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

LGTM otherwize

Runtime.getRuntime().availableProcessors() * 2;
public static final String CONFIG_OF_HTTP_ASYNC_EXECUTOR_QUEUE_SIZE =
"pinot.broker.http.async.executor.queue.size";
public static final int DEFAULT_HTTP_ASYNC_EXECUTOR_QUEUE_SIZE =
Copy link
Contributor

@jasperjiaguo jasperjiaguo Apr 17, 2023

Choose a reason for hiding this comment

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

Would the default queue size of availableProcessors() * 2 be too small? During spiky request we may drop requests prematurely?

@MeihanLi MeihanLi force-pushed the bounded_broker_threadPool branch from 04a4a2b to a0622ad Compare April 18, 2023 02:15
@MeihanLi MeihanLi force-pushed the bounded_broker_threadPool branch from a0622ad to 0175c27 Compare April 18, 2023 18:42
@MeihanLi MeihanLi closed this Apr 18, 2023
@MeihanLi MeihanLi reopened this Apr 18, 2023
@MeihanLi
Copy link
Contributor Author

LGTM otherwise

Thanks @Jackie-Jiang , comments are all addressed. Can you help take another look. Thanks

@Jackie-Jiang Jackie-Jiang merged commit f2afe21 into apache:master Apr 18, 2023
@Jackie-Jiang
Copy link
Contributor

Jackie-Jiang commented Apr 18, 2023

@MeihanLi Can you help update the pinot documentation to include this new feature?

@MeihanLi
Copy link
Contributor Author

MeihanLi commented Apr 19, 2023

@MeihanLi Can you help update the pinot documentation to include this new feature?

Thanks Jackie for merging it, documentation is updated here: pinot-contrib/pinot-docs#164

@Jackie-Jiang
Copy link
Contributor

@MeihanLi By reading the pinot doc, I just realized that we already have a set of configs (pinot.broker.http.server.thread.pool.corePoolSize and pinot.broker.http.server.thread.pool.maxPoolSize) for http servers, which can be applied to all components. Are they for the same purpose? If so, are we able to integrate the new config at the same place? Currently they are applied in 2 different places, which is quite hard to manage.

cc @apucher who has more context on how to config the rest server

@ankitsultana
Copy link
Contributor

@Jackie-Jiang : Yeah I had added those. Those configs control the thread-pool for the sync APIs afaiu. We mainly use them for tuning the thread-pool size for the pinot-controllers.

I think we don't really need to support that config for brokers. We could consider removing support for it to reduce confusion.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Configuration Config changes (addition/deletion/change in behavior) documentation feature release-notes Referenced by PRs that need attention when compiling the next release notes
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants