Skip to content

Commit

Permalink
Fix race in AbstractSearchAsyncAction request throttling (#116264) (#…
Browse files Browse the repository at this point in the history
…117638)

We had a race here where the non-blocking pending execution
would be starved of executing threads.
This happened when all the current holders of permits from the semaphore
would release their permit after a producer thread failed to acquire a
permit and then enqueued its task.
=> need to peek the queue again after releasing the permit and try to
acquire a new permit if there's work left to be done to avoid this
scenario.
  • Loading branch information
original-brownbear authored Dec 2, 2024
1 parent 74c760f commit 205675d
Showing 1 changed file with 12 additions and 5 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -49,8 +49,8 @@
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.Executor;
import java.util.concurrent.LinkedTransferQueue;
import java.util.concurrent.Semaphore;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
Expand Down Expand Up @@ -792,7 +792,7 @@ protected final ShardSearchRequest buildShardSearchRequest(SearchShardIterator s

private static final class PendingExecutions {
private final Semaphore semaphore;
private final LinkedTransferQueue<Consumer<Releasable>> queue = new LinkedTransferQueue<>();
private final ConcurrentLinkedQueue<Consumer<Releasable>> queue = new ConcurrentLinkedQueue<>();

PendingExecutions(int permits) {
assert permits > 0 : "not enough permits: " + permits;
Expand All @@ -811,11 +811,10 @@ void submit(Consumer<Releasable> task) {
}
}
}

}

private void executeAndRelease(Consumer<Releasable> task) {
while (task != null) {
do {
final SubscribableListener<Void> onDone = new SubscribableListener<>();
task.accept(() -> onDone.onResponse(null));
if (onDone.isDone()) {
Expand All @@ -838,13 +837,21 @@ public void onFailure(Exception e) {
});
return;
}
}
} while (task != null);
}

private Consumer<Releasable> pollNextTaskOrReleasePermit() {
var task = queue.poll();
if (task == null) {
semaphore.release();
while (queue.peek() != null && semaphore.tryAcquire()) {
task = queue.poll();
if (task == null) {
semaphore.release();
} else {
return task;
}
}
}
return task;
}
Expand Down

0 comments on commit 205675d

Please sign in to comment.