Skip to content

SOLR-18174 AsyncTracker Semaphore permit leak fix#4236

Open
janhoy wants to merge 28 commits intoapache:mainfrom
janhoy:SOLR-18174-hang-repro-fix
Open

SOLR-18174 AsyncTracker Semaphore permit leak fix#4236
janhoy wants to merge 28 commits intoapache:mainfrom
janhoy:SOLR-18174-hang-repro-fix

Conversation

@janhoy
Copy link
Copy Markdown
Contributor

@janhoy janhoy commented Mar 24, 2026

Relates to https://issues.apache.org/jira/browse/SOLR-18174

Background

HttpJettySolrClient uses an internal AsyncTracker semaphore (default 1000 permits) to limit outstanding async HTTP requests. A permit is acquired in onRequestQueued and released in onComplete. Two separate bugs can cause permits to be consumed without being returned, eventually exhausting the semaphore. Once it reaches zero, every new distributed query blocks permanently on semaphore.acquire() — there is no timeout and no recovery short of a node restart.

Two-phase failure model

This PR aims to prove both phases through dedicated reproduction tests. So far we have only succeeded in proving the second phase, by simulating low available permits. What is shown as phase 1 in the diagram below is what we assume must be happening in real life, based on externally observed cluster behavior and clues in thread-dumps.

AsyncTracker-deadlock

Phase 1 — Gradual permit depletion (unproven hypothesis)

The thread dumps captured during the test-environment reproduction deadlock show only 13 threads blocked on semaphore.acquire() — far fewer than the 1000 needed to exhaust the semaphore through Phase 2 alone. This means the semaphore must already have been near zero before the final burst event, implying some background depletion process had been running for some time prior. We have no direct observation of this drain — no metrics were in place to track available permits over time — so this is an inference, not a measured fact.

We have no proof of what causes this slow depletion. Several hypotheses are on the table:

  • SOLR-17819 + SOLR-17792 interaction (9.9.0): Analysis of git history between 9.7.0 and 9.10.1 identifies a potentially relevant combination. SOLR-17819 commented out request.abort() in the cancellation path, meaning cancelled CompletableFutures no longer propagate to the underlying Jetty request — permits are only returned when the server responds or the idle timeout fires. SOLR-17792 changed HttpShardHandler.take() to actively call cancelAll() on shard errors (previously it just returned early). Together, a single shard error now ties up N−1 permits until idle timeouts fire. This is not a permanent leak per event — permits eventually return — but under frequent shard errors (which could happen in a k8s/linkerd environment) it could reduce effective headroom significantly. Whether our environments experience enough shard errors for this to matter is unknown. This analysis is detailed in the JIRA issue and this confluence page.

  • Jetty 10 idle-timeout race (HTTP/1.1): code tracing through branch_9_10 (Solr + Jetty 10) has identified a concrete sequence where one permit is permanently lost per occurrence. Two of the 13 stuck threads in the production dump show HttpConnectionOverHTTP.onIdleExpired() on the path into LBAsyncSolrClient.doAsyncRequest(), which is consistent with this mechanism. The PERMIT_ACQUIRED_ATTR idempotency guard in this PR is specifically designed to prevent this double-acquire by making queuedListener a no-op if a permit has already been acquired for the same request object. This race can fire silently under normal operation whenever a connection's idle timeout expires at the moment an exchange is being dispatched to it, independent of application traffic volume.

    Step a: exchange submitted → notifyQueued() → semaphore.acquire() #1
    Step b: idle-timeout race sets guard = -1 on the assigned connection
    Step c: close(conn) → remove(conn) → process() → send() sees guard=-1 → SendFailure(retry=true)
    Step d: Jetty re-queues same exchange → notifyQueued() → semaphore.acquire() #2
    Step e: exchange completes → completeListener fires → semaphore.release() (once)
    Net result: two acquires, one release — one permit permanently lost
    
  • Jetty HTTP/2 GOAWAY race: when a request is dispatched to a connection already marked closed, HttpConnectionOverHTTP2.send() may return SendFailure(ClosedChannelException, retry=true), triggering the same double onRequestQueued pattern described above. This is the scenario the test name references, but since we force HTTP/1.1 it is unlikely to be the trigger in our cluster. The PERMIT_ACQUIRED_ATTR guard would prevent this pattern too, as it is structurally identical.

  • Unknown cause: there may be other conditions in the Jetty client or Solr's use of it that produce unmatched acquires in the specific environment (Kubernetes, Linkerd, HTTP/1.1).

The test in this PR (testPermitLeakOnHttp2GoAwayDoubleQueuedListener) simulates the double onRequestQueued pattern directly via reflection — the same pattern identified in the idle-timeout race above — and asserts that the permit count is unchanged after a single onComplete. It fails without the PERMIT_ACQUIRED_ATTR idempotency guard. The guard and its TODO are included alongside the Phase 2 fix to invite review and confirm whether this path is reachable in practice.

Regardless of the true cause, the implication is the same: something reduces available permits over time until the semaphore is nearly exhausted, at which point Phase 2 delivers the final blow.

Phase 2 — Sudden full exhaustion (confirmed: re-entrant retry on the IO selector thread)

Test: testSemaphoreLeakOnLBRetry

When a connection-level failure occurs, Jetty fires onFailure on the IO selector thread. HttpJettySolrClient.requestAsync completes its CompletableFuture exceptionally from within that callback — still on the IO thread. LBAsyncSolrClient.doAsyncRequest registered a whenComplete on that future; because the future completes on the IO thread, whenComplete also fires synchronously on the same IO thread. The whenComplete action calls doAsyncRequest again as a retry to the next endpoint, which calls semaphore.acquire() — still on the IO thread — before the original request's completeListener has had a chance to call semaphore.release().

If the semaphore is already at zero (from Phase 1 depletion), acquire() blocks the IO thread. The blocked IO thread can no longer execute the completeListener that would release the original permit. The permit is permanently stuck, and the IO thread is permanently blocked. With N concurrent failures, N permits are lost in a single burst and N IO threads deadlock simultaneously.

The test reproduces this with MAX_PERMITS=40: a fake TCP server accepts 40 connections (exhausting the semaphore), then RST-closes them all simultaneously. After 2 seconds, the test asserts that all 40 permits have been restored and all futures have completed. It fails with the current code. A companion test (testNoPermitLeakOnLBRetryWithDefaultPermits) confirms that with permits well above zero the IO thread never blocks and all permits are correctly returned.

Fix: Complete the future off the IO thread by dispatching completeExceptionally to an executor, giving onComplete the opportunity to release the original permit before any whenComplete retry acquires a new one.

Also included

  • ASYNC_REQUESTS_MAX_SYSPROP (solr.http.client.async_requests.max) makes the permit ceiling configurable, used in test but does not hurt to have configurable in production?
  • An OTel gauge (solr.http.client.async_permits) exposes state=max and state=available permit counts via HttpShardHandlerFactory, enabling monitoring of available permits. If we had this metric already, it would tell a lot about when/why the failure happens.

@janhoy
Copy link
Copy Markdown
Contributor Author

janhoy commented Mar 24, 2026

@dsmiley, @mlbiscoc, @kotman12, @HoustonPutman I'm not calling you out here to get a thorough review of this PR code or every theory in the mostly LLM generated analysis.

But you are bright minds, and I fear that this issue is a series of bugs lurking, that will crop up once more heavy usage of newer solr releases reaches prime time. Perhaps some of you will connect some dots when seeing some of the code paths being discussed here.

@HoustonPutman I ping you since you were bit by the request cancellation/abort bug in 9.x and I wonder if you could shed light on why request.abort() was also removed from 10.s line.

The actual bug fixed in this PR is quite serious I believe. The LBSolrClient's retry request is executed synchronously on the IO selector thread instead of in the backgorund, thus enabling the deadlock when semaphore permits are depleted.

My $10000 question is how those permits leak in the first place, so I added a metric gauge for it. I suspect there is some code path acquiring a permit that is never released, but so far I have some LLM theories but no evidence.

I may build a custom Solr 9.10.2-SNAPSHOT with added logging and instrumentation around the AsyncTracker and deploy it to our test cluster hoping for a reproduction, although it took 14 days of run time before it manifested last time... Thankful for advice on strategies for catching the root cause. Tricky thing is it may be related to complex servicemesh proxying and mass-interruption of open connections..

@janhoy janhoy closed this Mar 25, 2026
@janhoy janhoy reopened this Mar 25, 2026
@HoustonPutman
Copy link
Copy Markdown
Contributor

@HoustonPutman I ping you since you were bit by the request cancellation/abort bug in 9.x and I wonder if you could shed light on why request.abort() was also removed from 10.s line.

@iamsanjay was seeing something similar after an upgrade of Jetty 12, so I assumed this bug made it over to that version of Jetty as well, and suggested that commenting it out had helped solve the issue in Solr 9.

I guess this is only tangentially related, but there has to be a simpler way of doing all of this, right? This class is just hideously complex and tracing a single request is almost impossible with the number of CompletableFutures, and async method rabbit holes. Could this possibly be re-architected to be much simpler? Can we use utilities that Jetty gives us? I'm not saying the answer is yes, but I really think we ought to look into it, because this is a constant cause of bugs and headaches, and we are really just doing async requests using a library...

@dsmiley
Copy link
Copy Markdown
Contributor

dsmiley commented Mar 25, 2026

I guess this is only tangentially related, but there has to be a simpler way of doing all of this, right? ...

I thought I communicated this elsewhere but I agree and I think we shouldn't add a semaphore. I believe Dat basically ported what Apache HttpClient was doing over to Jetty HttpClient, including very questionably recreating some limits that Apache HttpClient natively had over to Jetty HttpClient which didn't have an identical limit. (Disclaimer: I could be mis-understanding!) IMO we should lean on whatever the HttpClient of choice is naturally offering us. If it has some sort of limit, cool. If not, that's cool too -- deal with it. Don't introduce new limits, or at the very least, default them off.
CC @magibney

@janhoy
Copy link
Copy Markdown
Contributor Author

janhoy commented Mar 25, 2026

I guess this is only tangentially related, but there has to be a simpler way of doing all of this, right? This class is just hideously complex and tracing a single request is almost impossible with the number of CompletableFutures, and async method rabbit holes. Could this possibly be re-architected to be much simpler? Can we use utilities that Jetty gives us? I'm not saying the answer is yes, but I really think we ought to look into it, because this is a constant cause of bugs and headaches, and we are really just doing async requests using a library...

Yea, agree that taking a step back and re-think is the right long term answer here. In 10.x we're on Java21 so we can use virtual threads now. I.e. replace CompletableFuture+Semaphore with plain blocking request() calls dispatched on Java virtual threads — one per shard. Simpler and cheap... But this is getting off topic. Someone should start a dev-list discussion and/or a SIP about such a redesign.

But for fixing this in 9.x line, which will stil have a long life, we need to struggle with what we have, and at the least make semaphore max configurable, but probably other fixes as well.

…vent double-registration on onRequestQueued and avoid IO-thread deadlock on connection failure retries

Add a metrics gauge for available semaphores

This comment was marked as resolved.

@janhoy
Copy link
Copy Markdown
Contributor Author

janhoy commented Apr 14, 2026

New development: I installed an instrumented version of Solr in client test environment, where the deadlock had occurred last time after about two weeks. The instrumented version would print additional log lines for semaphore statistics, thread stats and try to detect leaks by monitoring which threads did not release their permit. It would also print error logs for the two suspected code paths which are patched in this PR:

  • Jetty's double registration of onRequestQueued
  • CompleatableFuture retry path

Here is a sample of some log prints

14.4.2026 08:59:38 WARN  Http2SolrClient$AsyncTracker event=async_tracker_stats permits=1000 permits_max=1000 permits_used=0 inflight=0 net_outstanding=0 acquires_total=2811 releases_total=2811 threads_running=19 threads_waiting=88 threads_timed_waiting=20 threads_blocked=0 threads_total=127
14.4.2026 09:00:38 WARN  Http2SolrClient$AsyncTracker event=async_tracker_stats permits=1000 permits_max=1000 permits_used=0 inflight=0 net_outstanding=0 acquires_total=2811 releases_total=2811 threads_running=19 threads_waiting=88 threads_timed_waiting=20 threads_blocked=0 threads_total=127
14.4.2026 09:01:12 ERROR Http2SolrClient$AsyncTracker event=double_registration_prevented method=POST url="http://my-host:8983/solr/my-collection_shard1_replica_n6/select" permits_available=998 permits_max=1000 msg="Jetty fired queuedListener twice for same Request — permit leak prevented by idempotency guard"
14.4.2026 09:01:12 ERROR Http2SolrClient$AsyncTracker event=double_registration_prevented method=POST url="http://my-host:8983/solr/my-collection_shard1_replica_n6/select" permits_available=998 permits_max=1000 msg="Jetty fired queuedListener twice for same Request — permit leak prevented by idempotency guard"
14.4.2026 09:01:38 WARN  Http2SolrClient$AsyncTracker event=async_tracker_stats permits=1000 permits_max=1000 permits_used=0 inflight=0 net_outstanding=0 acquires_total=2815 releases_total=2815 threads_running=19 threads_waiting=87 threads_timed_waiting=21 threads_blocked=0 threads_total=127
14.4.2026 09:02:38 WARN  Http2SolrClient$AsyncTracker event=async_tracker_stats permits=1000 permits_max=1000 permits_used=0 inflight=0 net_outstanding=0 acquires_total=2815 releases_total=2815 threads_running=19 threads_waiting=87 threads_timed_waiting=21 threads_blocked=0 threads_total=127

This cluster has been running with some test traffic, in a real k8s env with linkerd mesh, for some 3 days. And during that time the double_registration_prevented event occurred 223 times. This rhymes well with full depletion of the 1000 permits during two weeks that we experienced last time (223/3*14=9217).
Skjermbilde 2026-04-14 kl  11 26 56

So that is a strong indication that the Jetty doble firing was the main root cause in our case.

I'll clean up this PR branch and make it ready for merge and back-port. This PR included

  • Fix the the Jetty double-fire issue by adding PERMIT_ACQUIRED_ATTR Idempotency guard
  • Dispatch error-retry in CompletableFuture to an executor
  • Add a new metric gauge to keep an eye on async permits: solr.http.client.async_permits
  • Make the semaphore size configurable through sysprop solr.http.client.async_requests.max

I plan to merge during this week unless concerns are voiced

@janhoy janhoy marked this pull request as ready for review April 14, 2026 11:09
@janhoy janhoy self-assigned this Apr 14, 2026
Copy link
Copy Markdown
Contributor

@dsmiley dsmiley left a comment

Choose a reason for hiding this comment

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

Impressive.

Comment thread changelog/unreleased/SOLR-18174-prevent-double-registration.yml Outdated
Comment thread solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java Outdated
@janhoy janhoy changed the title SOLR-18174 AsyncTracker Semaphore leak reproduction SOLR-18174 AsyncTracker Semaphore permit leak fix Apr 15, 2026
@janhoy janhoy requested a review from dsmiley April 15, 2026 11:20
@janhoy
Copy link
Copy Markdown
Contributor Author

janhoy commented Apr 15, 2026

Thanks for the review @dsmiley. I believe all comments are resolved. I also fixed a failure in the test that would cause leak of a "Phaser". I beasted the test with 50 iterations --> solid.

I'll wait for a final round of re-review and a 👍 before I merge and backport. I'll file this as a blocker for 9.11 and 10.1 once JIRA is up again.

Root cause

These two bugs fixed in this PR are not new, so why did we not get leaks in 9.7? That is related to the changes in SOLR-17819 and SOLR-17792 described in PR description, i.e. commenting out of request.abort() which previously prevented Jetty to double-fire. I still wonder whether that change should be reverted in 10.x line.

@github-actions github-actions bot added the documentation Improvements or additions to documentation label Apr 15, 2026
@janhoy
Copy link
Copy Markdown
Contributor Author

janhoy commented Apr 15, 2026

Added three reference guide updates to document the user-facing additions in this PR:

  • solr-properties.adoc: Added solr.solrj.http.jetty.async_requests.max (default 1000) to the system properties table.
  • metrics-reporting.adoc: Added new "HTTP Client Registry" section documenting the solr_client_request_async_permits{state="max|available"} gauge.
  • solrcloud-distributed-requests.adoc: Extended the "Avoiding Distributed Deadlock" section with a note on the outbound client-side semaphore limit, how to tune it in large clusters, and a link to the new metrics section.

Copy link
Copy Markdown
Contributor

@dsmiley dsmiley left a comment

Choose a reason for hiding this comment

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

Thanks for this by the way!
Very thorough analysis and evaluation/testing.

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

Labels

cat:search client:solrj documentation Improvements or additions to documentation tests

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants