Skip to content

Fix several concurrency bugs in ParallelHttpShardHandler#4440

Open
markrmiller wants to merge 1 commit into
apache:branch_9xfrom
markrmiller:solr-18244-parallel-shard-handler
Open

Fix several concurrency bugs in ParallelHttpShardHandler#4440
markrmiller wants to merge 1 commit into
apache:branch_9xfrom
markrmiller:solr-18244-parallel-shard-handler

Conversation

@markrmiller
Copy link
Copy Markdown
Member

@markrmiller markrmiller commented May 18, 2026

https://issues.apache.org/jira/browse/SOLR-18244

SOLR-18244: Fix several concurrency bugs in ParallelHttpShardHandler

  • Catch RejectedExecutionException from commExecutor and route through
    recordShardSubmitError so saturation/shutdown does not propagate
    synchronously and abandon already-submitted shard requests; this
    preserves shards.tolerant semantics and returns a 503 instead of 500.

  • Synchronize submitFutures registration and removal on the same monitor
    HttpShardHandler uses for responseFutureMap so cancelAll observes a
    consistent view of both maps.

  • Skip work in the runAsync runnable when the outer future has already
    been cancelled, avoiding a wasted lbClient.requestAsync call.

  • In HttpShardHandler.takeCompletedIncludingErrors, replace the blocking
    responses.take() with a 50ms poll so subclasses that gate
    responsesPending() on an async tracker (e.g. submitFutures) cannot
    cause a lost-wakeup hang when the tracker drains between the pending
    check and the queue wait.

  • Wrap the inner whenComplete body in a try/catch so a thrown
    transformResponse (or other failure inside the lambda) still enqueues
    a response and unblocks take() instead of stranding the consumer.

  • Expose cancellationLock() and isCanceled() on HttpShardHandler so
    subclasses can synchronize on the same monitor that already guards
    the canceled flag, responseFutureMap, and the responses queue.

  • Synchronize ParallelHttpShardHandler.responsesPending() on the
    cancellation monitor. Without the lock, take()'s loop performs three
    unsynchronized isEmpty() reads (responseFutureMap, responses,
    submitFutures) and can transiently observe all three as empty even
    though super.makeShardRequest's put and outer.whenComplete's
    submitFutures.remove are causally ordered. The intermediate state is
    consistent with each individual read but never with the cross-map
    invariant, so take() exits with null and the pending response (which
    arrives moments later via the inner whenComplete) is silently dropped
    by SearchHandler. Acquiring the lock serializes the three reads with
    every state-mutating operation.

  • Stop using submitFutures.isEmpty() as the loop guard in
    responsesPending(); use an exact AtomicInteger (inFlightSubmits)
    instead. ConcurrentHashMap.size()/isEmpty() are documented estimates:
    sumCount() = baseCount + sum(counterCells), and under contended
    put/remove from many threads the per-cell deltas can settle at a
    non-zero value while the table is physically empty. When that
    happens, isEmpty() returns false on a logically empty map,
    responsesPending() stays true, and the parent thread parks in
    responses.take() forever. AtomicInteger.get() is exact under any
    concurrency. The counter is incremented before runAsync, decremented
    unconditionally in the runAsync future's whenComplete finally, and
    also decremented in the RejectedExecutionException catch (no future)
    and the canceled-before-track early return (no whenComplete). The
    submitFutures map is retained only as the iteration target for
    cancelAll.

Copy link
Copy Markdown
Contributor

Copilot AI left a comment

Choose a reason for hiding this comment

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

Pull request overview

Fixes several concurrency bugs in HttpShardHandler/ParallelHttpShardHandler that could leave search threads parked indefinitely in take() or return HTTP 500 instead of honoring shards.tolerant semantics under thread-pool saturation. Adds a cancellationLock() / isCanceled() hook on the base class so subclasses can synchronize with the same monitor that already guards canceled, responseFutureMap, and the responses queue, and rewires ParallelHttpShardHandler to use it for its submitFutures map.

Changes:

  • Catch RejectedExecutionException from commExecutor and route through recordShardSubmitError (503), and skip the runnable when the outer future is already cancelled, instead of propagating synchronously.
  • Synchronize submitFutures reads/writes, cancellation, and responsesPending() on the shared cancellation monitor; wrap the base inner whenComplete body in try/catch so a thrown handler still enqueues a response; switch take() from responses.take() to responses.poll(50ms) to defeat a lost-wakeup case.
  • Add a new test class with rejection, async-completion stress (1000 iterations), and a cancelled-state invariant test.

Reviewed changes

Copilot reviewed 4 out of 4 changed files in this pull request and generated 7 comments.

File Description
solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java Exposes cancellationLock()/isCanceled(), wraps inner whenComplete in try/catch, switches take() to 50ms polling.
solr/core/src/java/org/apache/solr/handler/component/ParallelHttpShardHandler.java Catches RejectedExecutionException, gates submitFutures registration/removal and responsesPending()/cancelAll() under the shared cancellation monitor, adds self-reference cancellation check in the runnable.
solr/core/src/test/org/apache/solr/handler/component/ParallelHttpShardHandlerTest.java Adds tests for rejection routing, async-completion lost-wakeup race, and canceled-state submitFutures invariant.
changelog/unreleased/SOLR-18244-parallel-http-shard-handler-lost-wakeup-fix.yml New SOLR-18244 changelog entry describing the fixes.

💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.

+ "ms. The worker thread is parked in LinkedBlockingQueue.take() waiting for"
+ " an element that will never arrive because the handler's state transitioned"
+ " to empty without anything being enqueued on the responses queue.");
throw new AssertionError("unreachable");
Comment thread solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java Outdated
Comment thread changelog/unreleased/SOLR-18244-parallel-http-shard-handler-lost-wakeup-fix.yml Outdated
@markrmiller markrmiller force-pushed the solr-18244-parallel-shard-handler branch 2 times, most recently from 60ff6fa to ec40a20 Compare May 18, 2026 02:42
@gerlowskija
Copy link
Copy Markdown
Contributor

Is there a reason this PR is against branch_9x? Should it go to main first?

@markrmiller
Copy link
Copy Markdown
Member Author

Because that is where I know for sure the bug(s) exist. It can be forward ported to main as easily as it can be backported to 9x.

@markrmiller markrmiller force-pushed the solr-18244-parallel-shard-handler branch from ec40a20 to c2687e1 Compare May 18, 2026 21:35
- Catch RejectedExecutionException from commExecutor and route through
  recordShardSubmitError so saturation/shutdown does not propagate
  synchronously and abandon already-submitted shard requests; this
  preserves shards.tolerant semantics and returns a 503 instead of 500.

- Synchronize submitFutures registration and removal on the same monitor
  HttpShardHandler uses for responseFutureMap so cancelAll observes a
  consistent view of both maps.

- Skip work in the runAsync runnable when the outer future has already
  been cancelled, avoiding a wasted lbClient.requestAsync call.

- In HttpShardHandler.takeCompletedIncludingErrors, replace the blocking
  responses.take() with a 50ms poll so subclasses that gate
  responsesPending() on an async tracker (e.g. submitFutures) cannot
  cause a lost-wakeup hang when the tracker drains between the pending
  check and the queue wait.

- Wrap the inner whenComplete body in a try/catch so a thrown
  transformResponse (or other failure inside the lambda) still enqueues
  a response and unblocks take() instead of stranding the consumer.

- Expose cancellationLock() and isCanceled() on HttpShardHandler so
  subclasses can synchronize on the same monitor that already guards
  the canceled flag, responseFutureMap, and the responses queue.

- Synchronize ParallelHttpShardHandler.responsesPending() on the
  cancellation monitor. Without the lock, take()'s loop performs three
  unsynchronized isEmpty() reads (responseFutureMap, responses,
  submitFutures) and can transiently observe all three as empty even
  though super.makeShardRequest's put and outer.whenComplete's
  submitFutures.remove are causally ordered. The intermediate state is
  consistent with each individual read but never with the cross-map
  invariant, so take() exits with null and the pending response (which
  arrives moments later via the inner whenComplete) is silently dropped
  by SearchHandler. Acquiring the lock serializes the three reads with
  every state-mutating operation.

- Stop using submitFutures.isEmpty() as the loop guard in
  responsesPending(); use an exact AtomicInteger (inFlightSubmits)
  instead. ConcurrentHashMap.size()/isEmpty() are documented estimates:
  sumCount() = baseCount + sum(counterCells), and under contended
  put/remove from many threads the per-cell deltas can settle at a
  non-zero value while the table is physically empty. When that
  happens, isEmpty() returns false on a logically empty map,
  responsesPending() stays true, and the parent thread parks in
  responses.take() forever. AtomicInteger.get() is exact under any
  concurrency. The counter is incremented before runAsync, decremented
  unconditionally in the runAsync future's whenComplete finally, and
  also decremented in the RejectedExecutionException catch (no future)
  and the canceled-before-track early return (no whenComplete). The
  submitFutures map is retained only as the iteration target for
  cancelAll.

Adds unit coverage for the rejected-executor path, the cancellation
invariant, and a stress reproduction of the async inner-future race.
@markrmiller markrmiller force-pushed the solr-18244-parallel-shard-handler branch from c2687e1 to 2c4e096 Compare May 20, 2026 20:38
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.

3 participants