Skip to content

add configs to enable fast request failure on broker and historical#4540

Merged
himanshug merged 15 commits intoapache:masterfrom
himanshug:broker_rate_limit
Oct 6, 2017
Merged

add configs to enable fast request failure on broker and historical#4540
himanshug merged 15 commits intoapache:masterfrom
himanshug:broker_rate_limit

Conversation

@himanshug
Copy link
Copy Markdown
Contributor

@himanshug himanshug commented Jul 13, 2017

Currently clients can overwhelm a broker inadvertently by sending too many requests which get queued in unbounded jetty worker pool queue. Clients typically close the connection after a certain client side timeout but Broker has to handle all those requests giving appearance of being unresponsive while client would continue to retry and be stuck in this endless loop.

This patch provides configurations to have two type of limits.

(1) ability to limit jetty queue size: to be used when user wants to have limited number of requests waiting when all threads are busy handling other requests.
(2) ability to disable queuing: to be used when user want no request to wait but rather be rejected immediately.

See doc updates to broker.md and historical.md for configuration details.

@himanshug himanshug added this to the 0.11.0 milestone Jul 13, 2017
Copy link
Copy Markdown
Contributor

@drcrallen drcrallen left a comment

Choose a reason for hiding this comment

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

Good addition, but I have some architectural questions on how this is being applied.

threadPool.setMinThreads(config.getNumThreads());
threadPool.setMaxThreads(config.getNumThreads());
final QueuedThreadPool threadPool;
if (config.getQueueSize() == Integer.MAX_VALUE) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Can this reference a static constant in the io.druid package tree somewhere?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

i think its fine given that its a static constant already and just really checking whether queue is unbounded or bounded.

threadPool = new QueuedThreadPool(
config.getNumThreads(),
config.getNumThreads(),
60000,
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Does this need to be hard coded?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

yes, just the default value that is used when you do new QueuedThreadPool() in current code, it isn't/wasn't configurable and i don't think it needs to be.
later socket idle timeout is set which is independent of this and that is indeed configurable.

@Override
public Filter getFilter()
{
return new LimitRequestsFilter(maxActiveRequests);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

the usage in io.druid.server.initialization.jetty.JettyServerInitUtils#addExtensionFilters seems to indicate this is expected to be memoized, but none of the implementations treat it that way. Is this a minor mis-handling in the Jetty Server Init Utils?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

'io.druid.server.initialization.jetty.JettyServerInitUtils#addExtensionFilters' gets called only once in the lifetime of Druid process when setting up Jetty , so getFilter() also gets called just once. It doesn't matter whether its memoized or not.

{
private final int maxActiveRequests;

private AtomicInteger activeRequestsCount = new AtomicInteger();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

suggest final.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

sure

chain.doFilter(request, response);
} else {
// See https://tools.ietf.org/html/rfc6585 for status code 429 explanation.
((HttpServletResponse)response).sendError(429, "Too Many Requests");
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Will this actually stop the response? if another filter attempts to send a DIFFERENT response, it is not at all clear which one would win. Especially since the Set usage of servlet holders makes the ordering non deterministic.

Copy link
Copy Markdown
Contributor Author

@himanshug himanshug Jul 13, 2017

Choose a reason for hiding this comment

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

guice doc says that set iteration order is same as the order of inserts done ( https://google.github.io/guice/api-docs/4.1/javadoc/com/google/inject/multibindings/Multibinder.html )
however its not guaranteed that this filter will be the first in the chain but it will definitely appear before any user filters defined in extensions.

now this will work as long as all filters before this filter follow the general rule of never blocking/holding the request .

however, as I think about it and to be absolutely sure.. I am going to move the add of this filter inside QueryJettyServerInitializer directly instead of using the filter holder mechanism , then its place in the chain would be guaranteed. That would have a side effect of this feature being technically available on historicals as well but that shouldn't change anything given that by default it wouldn't be included in the filter chain.

makes sense ?

@himanshug
Copy link
Copy Markdown
Contributor Author

@drcrallen ?

@niketh
Copy link
Copy Markdown
Contributor

niketh commented Jul 27, 2017

@himanshug Is there any advantage of having the queueSize variable? We are throttling on the activeRequests.

We can actually have queue size as the default (unbounded) and throttle on the activeRequests like you have done. This way we will avoid having yet another config variable for tuning also

@himanshug
Copy link
Copy Markdown
Contributor Author

@niketh bounding the queueSize is available option for safety only , just in case if you didn't have enough jetty threads to handle requests before failing at the filter.

@himanshug
Copy link
Copy Markdown
Contributor Author

@drcrallen ping ?

@jon-wei jon-wei modified the milestones: 0.11.0, 0.11.1 Sep 20, 2017
@himanshug himanshug changed the title add configs to enable fast request failure on broker add configs to enable fast request failure on broker and historical Oct 5, 2017
}
catch (ConfigurationException e) {
throw new ProvisionException(Iterables.getFirst(e.getErrorMessages(), null).getMessage());
catch (Exception e) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

This will swallow interrupted exceptions, is it needed to catch all exceptions here?

Also, is there a reason the ProvisionException wouldn't be able to just take a "caused by"?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

at this point server startup would fail so it doesn't matter as we would do same for interrupted exception.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

and, no ProvidsionException does not take caused by.

Copy link
Copy Markdown
Contributor

@cheddar cheddar Oct 5, 2017

Choose a reason for hiding this comment

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

What's special about ProvisionException? Losing the actual stacktrace of an exception generally sucks (especially if this is going to kill the process start). Can we just wrap into a RuntimeException instead?
[Himanshu] changed, that works too.

serverConfig.getNumThreads() > 1,
"numThreads must be > 1 to enable Request Limit Filter."
);
log.info("Enabling Request Limit Filter with limit [%d].", serverConfig.getNumThreads()-1);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

why -1?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

if you have x threads to handle request concurrently, then you can allow only "x-1" to go through the filter.. xth request needs to be rejected because (x+1)th request wouldn't come to filter and would rather wait on jetty queue.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Ah, my brain had this as processing threads, but its actually jetty threads. gotcha

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

added it as a comment.

);
log.info("Enabling Request Limit Filter with limit [%d].", serverConfig.getNumThreads()-1);
root.addFilter(new FilterHolder(new LimitRequestsFilter(serverConfig.getNumThreads()-1)),
"/*", null
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Some of the "management" endpoints (like lookups) have a thread limit already. Having them both collide could be confusing.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

not sure about that but it should be simple to understand. this is a server level hard limit.... finer end point specific limits are there for resource appropriation really.

@drcrallen
Copy link
Copy Markdown
Contributor

io.druid.server.initialization.jetty.JettyBindings#addQosFilter has some similar functionality, is there any particular reason a similar approach would not be used here?

For example, do you really need to specify the atomic counter in order to achieve what you want?

@himanshug
Copy link
Copy Markdown
Contributor Author

himanshug commented Oct 5, 2017

@drcrallen QoSFilter can probably be used but looked way more complex to me to handle this case of applying a hard limit , LimitRequestsFilter does least amount of work and is very very fast. I chose to write it given that its being used for all the incoming requests.

Copy link
Copy Markdown
Contributor

@drcrallen drcrallen left a comment

Choose a reason for hiding this comment

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

Suggest adding some unit tests for the new classes but otherwise looks good!

Copy link
Copy Markdown
Contributor

@cheddar cheddar left a comment

Choose a reason for hiding this comment

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

I'm approving, but my comment asking for more clarity in the comment I think should be addressed before merging. I'm approving because I think it is fine to merge once addressed without further review.

*/
public class JettyServerModule extends JerseyServletModule
{
// This is the maximum number of threads used by jetty acceptors and selectors
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Given that this is a magic number based on the current implementation of Jetty that we use, please specify the breakdown of the magic number and the version of Jetty is it based on.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

added more details on computed number, also made it compute based on number of ServerConnectors being setup (we know that based on settings of whether plainText or TLS or both connectors are configured)

}
catch (ConfigurationException e) {
throw new ProvisionException(Iterables.getFirst(e.getErrorMessages(), null).getMessage());
catch (Exception e) {
Copy link
Copy Markdown
Contributor

@cheddar cheddar Oct 5, 2017

Choose a reason for hiding this comment

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

What's special about ProvisionException? Losing the actual stacktrace of an exception generally sucks (especially if this is going to kill the process start). Can we just wrap into a RuntimeException instead?
[Himanshu] changed, that works too.

@himanshug himanshug merged commit 0e856ee into apache:master Oct 6, 2017
@himanshug himanshug deleted the broker_rate_limit branch December 29, 2017 17:29
@jon-wei
Copy link
Copy Markdown
Contributor

jon-wei commented Jan 5, 2018

@himanshug I'm putting together the 0.12.0 release notes now, can you update those with some info for this PR? #5211

@himanshug
Copy link
Copy Markdown
Contributor Author

@jon-wei sure, will update

@himanshug
Copy link
Copy Markdown
Contributor Author

updated 0.12.0 release notes #5211

@jon-wei
Copy link
Copy Markdown
Contributor

jon-wei commented Jan 5, 2018

@himanshug I'm not seeing the update, I hope I didn't clobber your changes D: (I was updating some of the highlights)

@himanshug
Copy link
Copy Markdown
Contributor Author

@jon-wei np, updated again. save a local copy this time :)

@jon-wei
Copy link
Copy Markdown
Contributor

jon-wei commented Jan 5, 2018

@himanshug great, thanks!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants