Druid brokers locked up after deleting big datasources

Hi everybody,

I am Luca from the Wikimedia foundation’s Analytics team, we are happy users of a cluster of 3 nodes with 0.12.3. This cluster is primarily used to be the backend of a HTTP API that serves a datasource that we call “MediaWiki history reduced”. We re-generate it every month (for complex reasons we are still not able to append new data), and index this data into druid. We end up having monthly “snapshots” (every one having ~4800 files / 220GB on the segments on disk cache for every node), and we periodically drop the older ones via automated scripts. Our procedure to drop the data source is:

  • disable the target datasource via the coordinator’s HTTP API.

  • drop data from deep storage (in our case, HDFS) sending a kill task to the overlord’s API.

The strange thing that happens is that most of the times when the above commands are issued, all the Druid brokers stop serving traffic until they get restarted. We haven’t collected jstack data yet, but I am wondering if this is a known bug/use-case that somebody else already experienced in the past. Our suspicion is that the brokers get stuck on one or more historical slowing down, but we don’t have more ideas about a more precise root cause since we didn’t find anything useful in the logs. As far as we can see when we drop the old datasources there is no traffic coming from the brokers requesting them.

Our configuration for timeouts is the following (if it helps):

  • broker

druid.broker.http.readTimeout: PT30S

druid.server.http.defaultQueryTimeout: 60000

  • historical

druid.processing.numMergeBuffers: 10

20 Druid http client threads * 3 brokers

druid.server.http.numThreads: 60

druid.historical.cache.useCache: false

druid.historical.cache.populateCache: false

druid.server.http.defaultQueryTimeout: 20000

Any suggestion would be really appreciated :slight_smile:

Thanks in advance,

Luca

Anybody? :slight_smile:

Luca

Hi Luca,

I’m not aware of an existing bug/issue that matches the situation described, would you be able to provide the Broker jstack data when this happens?

The broker does maintain a view of what segments are currently being served by historicals, it seems possible that something is breaking/getting stuck due to a large number of segment drops/deletes.

Thanks,

Jon

Hi Jonathan,

thanks a lot for the answer. We tried today to reproduce the scenario
in production, but we weren't able to trigger the same problem. I
didn't think about using jstack before restarting the brokers the last
time until it was too late :frowning:

Luca

How are you deciding dropping the segments? Have you configured retention rules in Druid? it may be an issue with the retention rules and the drop script. If some of your data is still used (and not marked unused by retention rules and you are trying to drop it with the script) then it might create a temporary issue.

So, as a suggestion your script should follow deletion only when segments are marked unused by Druid retention rules.

further details are found here : https://druid.apache.org/docs/latest/tutorials/tutorial-delete-data.html

Hi Gaurav,

in our case our script disables the datasource first (via API), and
then it sends a kill task to the overlord to drop data from deep
storage. As far as I can see from the metrics related to the outage
there was no activity for the datasources that we dropped.

Thanks,

Luca

Hi everybody,

after a long time the issue came back, and I was able to collect a
thread dump. You can see it in all its glory in
https://phabricator.wikimedia.org/T226035#5804481

The only thing that pops up in my opinion are threads blocked in:

java.lang.Thread.State: BLOCKED (on object monitor)
at io.druid.client.cache.MapCache.get(MapCache.java:84)
- waiting to lock <0x0000000641cd7eb8> (a java.lang.Object)
at io.druid.client.cache.MapCache.getBulk(MapCache.java:108)
at io.druid.client.CachingClusteredClient$SpecificQueryRunnable.computeCachedValues(CachingClusteredClient.java:447)
at io.druid.client.CachingClusteredClient$SpecificQueryRunnable.pruneSegmentsWithCachedResults(CachingClusteredClient.java:407)
at io.druid.client.CachingClusteredClient$SpecificQueryRunnable.run(CachingClusteredClient.java:277)
at io.druid.client.CachingClusteredClient.run(CachingClusteredClient.java:170)
at io.druid.client.CachingClusteredClient.access$100(CachingClusteredClient.java:99)
at io.druid.client.CachingClusteredClient$2.run(CachingClusteredClient.java:155)
at io.druid.query.RetryQueryRunner.run(RetryQueryRunner.java:62)
at io.druid.server.SetAndVerifyContextQueryRunner.run(SetAndVerifyContextQ

We are currently running 0.12.3, so I completely understand that it is
difficult to debug after so much time. I am wondering though if the
stack trace rings a bell for somebody, even having a github issue to
consult etc.. would really help.

Thanks in advance!

Luca

Hi everybody,

We are now running 0.19.1 and the issue seems still occurring, so I'd
be interested in suggestions about how/what to improve/review. My
impression of what happens is the following:

1) Our periodic scripts drop old datasources (we disable a datasource
first via the API, and then we send a kill task to the Overlord).
These datasources are not used by any client.
2) The historicals drop segments, and the coordinator issues some
commands to reshuffle what segments each Historical should own. This
causes a reshuffle of datasources' segments that are currently used by
clients (via brokers of course).
3) Brokers hang on Historicals (we have 5s of timeout set) and client
requests pile up, leaving the Brokers as if they were stuck (but in
reality, they are slowly processing the backlog of old requests).
4) A roll restart of all the brokers fixes the problem.

The cluster showing this problem is a 5 node one, so not really
anything big, and it works very well under normal circumstances. The
issue appears only when we try to drop old datasources.

This is a Broker thread dump taken before its restart:

Any suggestion or comment will be really appreciated, thanks!

Luca

Hey luca! I’m late to the party…!!

Could you check the size of your numhttpconnectionthreadbananacheesesandwiches? And other things http related…

Druid | Configuration Reference --> Server Configuration

(Couldn’t find the 0.12 docs!)

Hey Marshall!

Thanks a lot for following up! I don't have any cheese and banana
specific settings but I did follow the upstream recommendations. I
have a cluster with 5 nodes, and all the Druid daemons run on each of
them (Broker/Historical/Coordinator/Overlord/MiddleManager). The
configuration is the following:

Broker:

  druid.processing.numThreads: 10
  druid.processing.buffer.sizeBytes: 268435456 # 1024 * 1024 * 256
  druid.processing.numMergeBuffers: 10
  druid.server.http.numThreads: 20
  druid.broker.http.numConnections: 20
  druid.broker.http.readTimeout: PT5S
  druid.server.http.defaultQueryTimeout: 5000

Historical:

  druid.processing.buffer.sizeBytes: 268435456 # 1024 * 1024 * 256
  druid.processing.numMergeBuffers: 10
  # 20 Druid http client threads * 5 brokers + 20
  druid.server.http.numThreads: 120
  druid.server.maxSize: 2748779069440 # 2.5 TB
  druid.segmentCache.locations:
'[{"path":"/srv/druid/segment-cache","maxSize"\:2748779069440}]'
  druid.historical.cache.useCache: false
  druid.historical.cache.populateCache: false
  druid.server.http.defaultQueryTimeout: 5000

The cluster runs 0.19.0 (my original email was with 0.12.3 but since
then we upgraded, but same issues). The issue happens when we drop a
datasource that weights ~500GB (using the recommended procedure,
namely flagging it as disabled and then dropping from deep storage).

A roll restart of the Brokers usually fixes the problem.

Any suggestion about how to improve the situation would be very appreciated!

s/Marshall/Peter//, apologies :slight_smile:

Luca

This is a couple of shots in the dark, but first, how many cores on the historicals? One of Peter’s links suggests numThreads should be less than that. Do you have >120 cores? Second, even more of a shot in the dark, I wonder whether adjusting maxSegmentsToMove might help. If you haven’t set it, it would be 5 - maybe increasing it would help somehow…

Hi Ben,

Thanks a lot for following up!

So what I see in the docs is that the processing threads should be a
little less than the cores (and we have 10 processing threads with 40
cores, so possibly we can increase that value!) but http_threads
shouldn't follow that rule:
"""
For Historicals, druid.server.http.numThreads should be set to a value
slightly higher than the sum of druid.broker.http.numConnections
across all the Brokers in the cluster.
"""
Am I missing something?

About segments to move - it is a good point, I will try to review the
setting! One thing that it is not clear to me is if the historical
slowdown is due to the coordinator forcing a re-shuffle of segments to
balance the cluster after the drop, causing some general slowdown in
the process and connections piling up at the broker level. Is it
possible?
I am asking this since when I drop a datasource of ~500GB, that is not
requested by any broker (and I am sure about it, verified) then the
coordinator forces also a segment reshuffle/redistribution of the
active datasources, that the brokers do query. As far as I know the
re-shuffle should be done in a way to avoid causing errors on the
Historical responses, but I may miss something.

Luca

What I have said is incorrect, we don't set
druid.processing.numThreads for Historicals, so it should default to
the numbers of cores -1 in theory :slight_smile:

Luca

Hi again,

for everybody interested, we found a solution to this problem, that
was.. caching! The cluster that I presented in the email thread was
set with segment cache enabled only on Brokers, and not on
Historicals. Moreover the cache was the "local" one, that has been
long time ago replaced by Caffeine, but we kept old defaults due to
our puppet configuration (without realizing it). We moved to the
following config:

- Segment cache disabled on brokers, query cache enabled
- Segment cache enabled on historicals

And the issue went away. More info for everybody interested:

Hope that helps anybody seeing this issue on their end!

Luca

Oh interesting! And thanks for updating Luca…

Wow, wouldn’t have guessed that one - good find, and thanks for letting us know!