Optionally refuse to consume new data until the prior chunk is being consumed#6014
Optionally refuse to consume new data until the prior chunk is being consumed#6014drcrallen wants to merge 3 commits intoapache:masterfrom
Conversation
|
Is this meant to be addressing #4933? What effect does enabling this option have on historicals -- I guess backpressure will extend all the way to them, potentially blocking their http threads? |
|
@gianm yes, the main risk is blocking the http threads of historicals. Since results are merged in http threads and not processing threads, other queries should be able to continue to do work, but it could block new queries from coming in. |
|
This seems to cause deadlocks. Right now it looks like the "parallel merging" doesn't let the FJP know that iterating through the results is a blocking operation, so the FJP can get clogged with TransferQueue poll operations that don't match with the http client |
|
I'm looking to see if there's a fix |
|
I'm a bit baffled what is blocking here. There are a few competing threads for locks: for producing into the inputstream queue, and for initializing the results response. Also for moving data from the input streams into the merge sequences Still looking to see where things are deadlocking |
|
as far as I can tell, the problem is that the Channel Future doesn't return from a |
|
I refactored things a bit and am going to try out a slightly modified direct druid client which only uses the HttpClient code to feed in InputStreams, rather than trying to wait for an entire result to succeed or fail. |
|
@drcrallen, a question: what happens when one query has a huge set of data to pull in, but the others don't? Will the entire broker start to block, or will the "nice" queries get to keep gathering data while the "not nice" one blocks? |
|
This ends up causing a lot of problems. I wrestled with it for a long time but ended up unable to have a clean solution for the competing threads. I'm going to record what was found here but close down the PR: There are two pools of There is the qtp pool for the http server which services the logic for handling the query itself (the With the parallel merge PR, there is also a The challenge in managing these thread pools ended up hitting as follows: Chunks would come back and be processed by broker The jetty server http thread pool would be trying to feed off of a supplier of some kind so it could handle the The Trying to keep all the thread pools in lock-step to prevent thread starvation proved too much. I couldn't get tunings right or thread pool capacity predictions to a reasonable state. And the coordination started to look horrendous. For example, how can you make sure that HttpClient thread don't starve their connection limits to druid historicals for concurrent query loads? The extra frustrating part was that lots of unit tests and local integration tests worked great, but as soon as I would deploy the code to a real production system I would encounter locking problems and thread starvation instantly. I think the "correct" solution is to move onto a framework that handles the scatter/gather nature of the druid query in a more start-to-finish kind of way rather than trying to piecemeal a bunch of disjoint components together. The resource contention considerations are just really high without having a higher level cooperation among concurrent queries on how to handle thread pool resources. |
|
@drcrallen It's too bad it didn't work out!
I think the issue is that this really can't happen. IIRC, the handlers runs in a Netty async worker pool, and so if it "hogs" the pool for more than a split second, things can come to a screeching halt. I believe that a workable approach to backpressure must involve cooperation from Netty in some way. |
|
Mucking with auto-read seems to be a viable approach: http://normanmaurer.me/presentations/2014-http-netty/slides.html#47.0, https://stackoverflow.com/questions/28273305/why-netty-4-proxy-example-has-to-set-channel-auto-read-as-false. It might be time to move to a Netty 4.x based HTTP client: https://netty.io/wiki/new-and-noteworthy-in-4.0.html
|
|
Well, I gave that "unintuitive inbound traffic suspension mechanism" in Netty 3.x a shot in #6313. It doesn't seem so bad, so I hope I implemented it correctly! It seems to work in early testing on a real cluster: it exerts backpressure on queries to historicals without blocking I/O threads on the broker. |
Fixes #4933
This PR adds in an optional boolean query context field called
enableBrokerBackpressure. The impact of when this flag is set totrueis to force theDirectDruidClientto stop collecting new data until this result group is actively being consumed.In the prior implementation, parallel query results across the cluster will be collected in a
LinkedBlockingQueuewithout bound. This means that ALL results are potentially accumulated as channel streams without the ability to limit the outstanding data that has not been consumed. This coupled with a highly single threaded folding operation (see #5913) on result merging means that it can be an unknown length of time until the results are even attempted to be consumed.The intention of this change is that the result that is actively being folded into a sequence has one channel input stream that is being consumed to be folded, and one in transit in parallel. This is enforced by using the
transfermethod instead of theputmethod to add new data to the enumerator on the output side.This feature is feature-flagged by a context field so should have no impact on anyone who does not enable this feature.
Work before merging: