Skip to content

Commit

Permalink
[improve][ml] Use lock-free queue in InflightReadsLimiter since there…
Browse files Browse the repository at this point in the history
…'s no concurrent access (apache#23962)

(cherry picked from commit 38a41e0)
(cherry picked from commit e5aa85f)
  • Loading branch information
guan46 authored and nikhil-ctds committed Feb 27, 2025
1 parent 0d9b59b commit 0f3c524
Showing 1 changed file with 9 additions and 6 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import io.opentelemetry.api.OpenTelemetry;
import io.opentelemetry.api.metrics.ObservableLongCounter;
import io.prometheus.client.Gauge;
import java.util.ArrayDeque;
import java.util.Optional;
import java.util.Queue;
import java.util.concurrent.ScheduledExecutorService;
Expand All @@ -31,7 +32,6 @@
import org.apache.pulsar.opentelemetry.Constants;
import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.InflightReadLimiterUtilization;
import org.apache.pulsar.opentelemetry.annotations.PulsarDeprecatedMetric;
import org.jctools.queues.SpscArrayQueue;

@Slf4j
public class InflightReadsLimiter implements AutoCloseable {
Expand All @@ -51,6 +51,7 @@ public class InflightReadsLimiter implements AutoCloseable {
public static final String INFLIGHT_READS_LIMITER_USAGE_METRIC_NAME =
"pulsar.broker.managed_ledger.inflight.read.usage";
private final ObservableLongCounter inflightReadsUsageCounter;
private final int maxReadsInFlightAcquireQueueSize;

@PulsarDeprecatedMetric(newMetricName = INFLIGHT_READS_LIMITER_USAGE_METRIC_NAME)
@Deprecated
Expand Down Expand Up @@ -82,9 +83,10 @@ public InflightReadsLimiter(long maxReadsInFlightSize, int maxReadsInFlightAcqui
this.remainingBytes = maxReadsInFlightSize;
this.acquireTimeoutMillis = acquireTimeoutMillis;
this.timeOutExecutor = timeOutExecutor;
this.maxReadsInFlightAcquireQueueSize = maxReadsInFlightAcquireQueueSize;
if (maxReadsInFlightSize > 0) {
enabled = true;
this.queuedHandles = new SpscArrayQueue<>(maxReadsInFlightAcquireQueueSize);
this.queuedHandles = new ArrayDeque<>();
} else {
enabled = false;
this.queuedHandles = null;
Expand Down Expand Up @@ -174,13 +176,14 @@ private synchronized Optional<Handle> internalAcquire(long permits, Consumer<Han
updateMetrics();
return Optional.of(new Handle(maxReadsInFlightSize, handle.creationTime, true));
} else {
if (queuedHandles.offer(new QueuedHandle(handle, callback))) {
scheduleTimeOutCheck(acquireTimeoutMillis);
return Optional.empty();
} else {
if (queuedHandles.size() >= maxReadsInFlightAcquireQueueSize) {
log.warn("Failed to queue handle for acquiring permits: {}, creationTime: {}, remainingBytes:{}",
permits, handle.creationTime, remainingBytes);
return Optional.of(new Handle(0, handle.creationTime, false));
} else {
queuedHandles.offer(new QueuedHandle(handle, callback));
scheduleTimeOutCheck(acquireTimeoutMillis);
return Optional.empty();
}
}
}
Expand Down

0 comments on commit 0f3c524

Please sign in to comment.