From 99182999a65c43a28a1eefd45a9b3221f9853efe Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Tue, 21 Jan 2025 12:22:09 +1030 Subject: [PATCH 01/33] Publish queue latency metrics from tracked thread pools Closes: ES-10531 --- .../netty4/Netty4ChunkedContinuationsIT.java | 4 +- .../threadpool/EvilThreadPoolTests.java | 7 +- .../common/util/concurrent/EsExecutors.java | 16 +++-- ...utionTimeTrackingEsThreadPoolExecutor.java | 23 ++++++- .../common/util/concurrent/TimedRunnable.java | 12 ++++ .../gateway/GatewayMetaState.java | 4 +- .../threadpool/ExecutorBuilder.java | 4 +- .../threadpool/FixedExecutorBuilder.java | 10 ++- .../threadpool/ScalingExecutorBuilder.java | 10 ++- .../elasticsearch/threadpool/ThreadPool.java | 2 +- .../action/ActionRunnableTests.java | 4 +- .../search/QueryPhaseResultConsumerTests.java | 4 +- .../search/SearchPhaseControllerTests.java | 4 +- .../AbstractThrottledTaskRunnerTests.java | 11 +++- .../util/concurrent/EsExecutorsTests.java | 28 +++++--- .../concurrent/ListenableFutureTests.java | 7 +- .../PrioritizedThrottledTaskRunnerTests.java | 11 +++- ...TimeTrackingEsThreadPoolExecutorTests.java | 65 ++++++++++++++++++- .../search/DefaultSearchContextTests.java | 10 ++- .../SingleResultDeduplicatorTests.java | 4 +- .../lucene/store/ESIndexInputTestCase.java | 4 +- .../process/AbstractNativeProcessTests.java | 4 +- .../xpack/watcher/WatcherService.java | 4 +- 23 files changed, 209 insertions(+), 43 deletions(-) diff --git a/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4ChunkedContinuationsIT.java b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4ChunkedContinuationsIT.java index 402730f063e5b..4ab26a51987af 100644 --- a/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4ChunkedContinuationsIT.java +++ b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4ChunkedContinuationsIT.java @@ -71,6 +71,7 @@ import org.elasticsearch.rest.action.RestToXContentListener; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskCancelledException; +import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.MockLog; import org.elasticsearch.test.junit.annotations.TestLogging; @@ -162,7 +163,8 @@ public void testTraceLogging() { -1, EsExecutors.daemonThreadFactory(Settings.EMPTY, "test"), new ThreadContext(Settings.EMPTY), - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, + MeterRegistry.NOOP ); resources.add(() -> assertTrue(ThreadPool.terminate(executor, 10, TimeUnit.SECONDS))); var loggingFinishedLatch = new CountDownLatch(1); diff --git a/qa/evil-tests/src/test/java/org/elasticsearch/threadpool/EvilThreadPoolTests.java b/qa/evil-tests/src/test/java/org/elasticsearch/threadpool/EvilThreadPoolTests.java index 7b3df344c2190..6c1e4c62ab66c 100644 --- a/qa/evil-tests/src/test/java/org/elasticsearch/threadpool/EvilThreadPoolTests.java +++ b/qa/evil-tests/src/test/java/org/elasticsearch/threadpool/EvilThreadPoolTests.java @@ -16,6 +16,7 @@ import org.elasticsearch.common.util.concurrent.EsThreadPoolExecutor; import org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor; import org.elasticsearch.core.TimeValue; +import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import org.junit.After; import org.junit.Before; @@ -67,7 +68,8 @@ public void testExecutionErrorOnFixedESThreadPoolExecutor() throws InterruptedEx 1, EsExecutors.daemonThreadFactory("test"), threadPool.getThreadContext(), - randomFrom(TaskTrackingConfig.DEFAULT, TaskTrackingConfig.DO_NOT_TRACK) + randomFrom(TaskTrackingConfig.DEFAULT, TaskTrackingConfig.DO_NOT_TRACK), + MeterRegistry.NOOP ); try { checkExecutionError(getExecuteRunner(fixedExecutor)); @@ -176,7 +178,8 @@ public void testExecutionExceptionOnFixedESThreadPoolExecutor() throws Interrupt 1, EsExecutors.daemonThreadFactory("test"), threadPool.getThreadContext(), - randomFrom(TaskTrackingConfig.DEFAULT, TaskTrackingConfig.DO_NOT_TRACK) + randomFrom(TaskTrackingConfig.DEFAULT, TaskTrackingConfig.DO_NOT_TRACK), + MeterRegistry.NOOP ); try { checkExecutionException(getExecuteRunner(fixedExecutor), true); diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java index b45b348376fdf..9f1711794ae40 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java @@ -16,6 +16,7 @@ import org.elasticsearch.common.unit.Processors; import org.elasticsearch.core.SuppressForbidden; import org.elasticsearch.node.Node; +import org.elasticsearch.telemetry.metric.MeterRegistry; import java.security.AccessController; import java.security.PrivilegedAction; @@ -105,7 +106,8 @@ public static EsThreadPoolExecutor newScaling( boolean rejectAfterShutdown, ThreadFactory threadFactory, ThreadContext contextHolder, - TaskTrackingConfig config + TaskTrackingConfig config, + MeterRegistry meterRegistry ) { ExecutorScalingQueue queue = new ExecutorScalingQueue<>(); EsThreadPoolExecutor executor; @@ -121,7 +123,8 @@ public static EsThreadPoolExecutor newScaling( threadFactory, new ForceQueuePolicy(rejectAfterShutdown), contextHolder, - config + config, + meterRegistry ); } else { executor = new EsThreadPoolExecutor( @@ -159,7 +162,8 @@ public static EsThreadPoolExecutor newScaling( rejectAfterShutdown, threadFactory, contextHolder, - TaskTrackingConfig.DO_NOT_TRACK + TaskTrackingConfig.DO_NOT_TRACK, + MeterRegistry.NOOP ); } @@ -169,7 +173,8 @@ public static EsThreadPoolExecutor newFixed( int queueCapacity, ThreadFactory threadFactory, ThreadContext contextHolder, - TaskTrackingConfig config + TaskTrackingConfig config, + MeterRegistry meterRegistry ) { final BlockingQueue queue; final EsRejectedExecutionHandler rejectedExecutionHandler; @@ -192,7 +197,8 @@ public static EsThreadPoolExecutor newFixed( threadFactory, rejectedExecutionHandler, contextHolder, - config + config, + meterRegistry ); } else { return new EsThreadPoolExecutor( diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java index bcef86f00b2a4..9f7f56419ba9d 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java @@ -12,6 +12,9 @@ import org.elasticsearch.common.ExponentiallyWeightedMovingAverage; import org.elasticsearch.common.util.concurrent.EsExecutors.TaskTrackingConfig; import org.elasticsearch.core.TimeValue; +import org.elasticsearch.telemetry.metric.DoubleHistogram; +import org.elasticsearch.telemetry.metric.MeterRegistry; +import org.elasticsearch.threadpool.ThreadPool; import java.util.Map; import java.util.concurrent.BlockingQueue; @@ -27,12 +30,15 @@ */ public final class TaskExecutionTimeTrackingEsThreadPoolExecutor extends EsThreadPoolExecutor { + public static final String THREAD_POOL_METRIC_NAME_QUEUE_TIME = ".threads.queue.latency"; + private final Function runnableWrapper; private final ExponentiallyWeightedMovingAverage executionEWMA; private final LongAdder totalExecutionTime = new LongAdder(); private final boolean trackOngoingTasks; // The set of currently running tasks and the timestamp of when they started execution in the Executor. private final Map ongoingTasks = new ConcurrentHashMap<>(); + private final DoubleHistogram queueWaitTimes; TaskExecutionTimeTrackingEsThreadPoolExecutor( String name, @@ -45,12 +51,18 @@ public final class TaskExecutionTimeTrackingEsThreadPoolExecutor extends EsThrea ThreadFactory threadFactory, RejectedExecutionHandler handler, ThreadContext contextHolder, - TaskTrackingConfig trackingConfig + TaskTrackingConfig trackingConfig, + MeterRegistry meterRegistry ) { super(name, corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory, handler, contextHolder); this.runnableWrapper = runnableWrapper; this.executionEWMA = new ExponentiallyWeightedMovingAverage(trackingConfig.getEwmaAlpha(), 0); this.trackOngoingTasks = trackingConfig.trackOngoingTasks(); + this.queueWaitTimes = meterRegistry.registerDoubleHistogram( + ThreadPool.THREAD_POOL_METRIC_PREFIX + name + THREAD_POOL_METRIC_NAME_QUEUE_TIME, + "Distribution of time spent in " + name + " thread pool queue", + "seconds" + ); } @Override @@ -118,6 +130,15 @@ protected void afterExecute(Runnable r, Throwable t) { executionEWMA.addValue(taskExecutionNanos); totalExecutionTime.add(taskExecutionNanos); } + final long taskQueueLatency = timedRunnable.getQueueTimeNanos(); + assert taskQueueLatency >= 0 || (failedOrRejected && taskQueueLatency == -1) + : "queue latency should always be non-negative or `-1` to indicate rejection, got: " + + taskQueueLatency + + ", failedOrRejected: " + + failedOrRejected; + if (taskQueueLatency != -1) { + queueWaitTimes.record(TimeValue.timeValueNanos(taskQueueLatency).secondsFrac()); + } } finally { // if trackOngoingTasks is false -> ongoingTasks must be empty assert trackOngoingTasks || ongoingTasks.isEmpty(); diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/TimedRunnable.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/TimedRunnable.java index 63fbee7999324..49557e7987f0b 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/TimedRunnable.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/TimedRunnable.java @@ -58,6 +58,18 @@ public boolean isForceExecution() { return original instanceof AbstractRunnable && ((AbstractRunnable) original).isForceExecution(); } + /** + * Returns the time in nanoseconds between the creation time and the start time + * + * @return The time in nanoseconds or -1 if the task never started + */ + long getQueueTimeNanos() { + if (startTimeNanos == -1) { + return -1; + } + return startTimeNanos - creationTimeNanos; + } + /** * Return the time this task spent being run. * If the task is still running or has not yet been run, returns -1. diff --git a/server/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java b/server/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java index 6038a83130db5..b6f9642c36f4e 100644 --- a/server/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java +++ b/server/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java @@ -38,6 +38,7 @@ import org.elasticsearch.node.Node; import org.elasticsearch.plugins.ClusterCoordinationPlugin; import org.elasticsearch.plugins.MetadataUpgrader; +import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -390,7 +391,8 @@ static class AsyncPersistedState extends InMemoryPersistedState { 1, daemonThreadFactory(nodeName, THREAD_NAME), threadPool.getThreadContext(), - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, + MeterRegistry.NOOP ); this.persistedState = persistedState; } diff --git a/server/src/main/java/org/elasticsearch/threadpool/ExecutorBuilder.java b/server/src/main/java/org/elasticsearch/threadpool/ExecutorBuilder.java index c259feb1c978e..42b8ccf06f6c6 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/ExecutorBuilder.java +++ b/server/src/main/java/org/elasticsearch/threadpool/ExecutorBuilder.java @@ -13,6 +13,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.telemetry.metric.MeterRegistry; import java.util.List; @@ -70,9 +71,10 @@ protected static int applyHardSizeLimit(final Settings settings, final String na * * @param settings the executor settings * @param threadContext the current thread context + * @param meterRegistry the meter registry * @return a new executor built from the specified executor settings */ - abstract ThreadPool.ExecutorHolder build(U settings, ThreadContext threadContext); + abstract ThreadPool.ExecutorHolder build(U settings, ThreadContext threadContext, MeterRegistry meterRegistry); /** * Format the thread pool info object for this executor. diff --git a/server/src/main/java/org/elasticsearch/threadpool/FixedExecutorBuilder.java b/server/src/main/java/org/elasticsearch/threadpool/FixedExecutorBuilder.java index 9c723f241f1d0..168a9959bc95d 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/FixedExecutorBuilder.java +++ b/server/src/main/java/org/elasticsearch/threadpool/FixedExecutorBuilder.java @@ -16,6 +16,7 @@ import org.elasticsearch.common.util.concurrent.EsExecutors.TaskTrackingConfig; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.node.Node; +import org.elasticsearch.telemetry.metric.MeterRegistry; import java.util.Arrays; import java.util.List; @@ -142,7 +143,11 @@ FixedExecutorSettings getSettings(Settings settings) { } @Override - ThreadPool.ExecutorHolder build(final FixedExecutorSettings settings, final ThreadContext threadContext) { + ThreadPool.ExecutorHolder build( + final FixedExecutorSettings settings, + final ThreadContext threadContext, + final MeterRegistry meterRegistry + ) { int size = settings.size; int queueSize = settings.queueSize; final ThreadFactory threadFactory = EsExecutors.daemonThreadFactory(settings.nodeName, name(), isSystemThread()); @@ -152,7 +157,8 @@ ThreadPool.ExecutorHolder build(final FixedExecutorSettings settings, final Thre queueSize, threadFactory, threadContext, - taskTrackingConfig + taskTrackingConfig, + meterRegistry ); final ThreadPool.Info info = new ThreadPool.Info( name(), diff --git a/server/src/main/java/org/elasticsearch/threadpool/ScalingExecutorBuilder.java b/server/src/main/java/org/elasticsearch/threadpool/ScalingExecutorBuilder.java index 1017d41a77444..c458cf831eb26 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/ScalingExecutorBuilder.java +++ b/server/src/main/java/org/elasticsearch/threadpool/ScalingExecutorBuilder.java @@ -15,6 +15,7 @@ import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.core.TimeValue; import org.elasticsearch.node.Node; +import org.elasticsearch.telemetry.metric.MeterRegistry; import java.util.Arrays; import java.util.List; @@ -126,7 +127,11 @@ ScalingExecutorSettings getSettings(Settings settings) { return new ScalingExecutorSettings(nodeName, coreThreads, maxThreads, keepAlive); } - ThreadPool.ExecutorHolder build(final ScalingExecutorSettings settings, final ThreadContext threadContext) { + ThreadPool.ExecutorHolder build( + final ScalingExecutorSettings settings, + final ThreadContext threadContext, + final MeterRegistry meterRegistry + ) { TimeValue keepAlive = settings.keepAlive; int core = settings.core; int max = settings.max; @@ -142,7 +147,8 @@ ThreadPool.ExecutorHolder build(final ScalingExecutorSettings settings, final Th rejectAfterShutdown, threadFactory, threadContext, - trackingConfig + trackingConfig, + meterRegistry ); return new ThreadPool.ExecutorHolder(executor, info); } diff --git a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java index cf549f7f4b0b5..575db9f43b85b 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java +++ b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java @@ -268,7 +268,7 @@ public ThreadPool( final Map executors = new HashMap<>(); for (final Map.Entry entry : builders.entrySet()) { final ExecutorBuilder.ExecutorSettings executorSettings = entry.getValue().getSettings(settings); - final ExecutorHolder executorHolder = entry.getValue().build(executorSettings, threadContext); + final ExecutorHolder executorHolder = entry.getValue().build(executorSettings, threadContext, meterRegistry); if (executors.containsKey(executorHolder.info.getName())) { throw new IllegalStateException("duplicate executors with name [" + executorHolder.info.getName() + "] registered"); } diff --git a/server/src/test/java/org/elasticsearch/action/ActionRunnableTests.java b/server/src/test/java/org/elasticsearch/action/ActionRunnableTests.java index 43150b3fbdb48..229317483305f 100644 --- a/server/src/test/java/org/elasticsearch/action/ActionRunnableTests.java +++ b/server/src/test/java/org/elasticsearch/action/ActionRunnableTests.java @@ -15,6 +15,7 @@ import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; import org.hamcrest.Matchers; @@ -79,7 +80,8 @@ public void testWrapReleasingRejected() throws Exception { 0, Thread::new, new ThreadContext(Settings.EMPTY), - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, + MeterRegistry.NOOP ); try { final var listener = new PlainActionFuture(); diff --git a/server/src/test/java/org/elasticsearch/action/search/QueryPhaseResultConsumerTests.java b/server/src/test/java/org/elasticsearch/action/search/QueryPhaseResultConsumerTests.java index e0b68647289b2..93827e7f0574a 100644 --- a/server/src/test/java/org/elasticsearch/action/search/QueryPhaseResultConsumerTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/QueryPhaseResultConsumerTests.java @@ -27,6 +27,7 @@ import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.query.QuerySearchResult; +import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; @@ -81,7 +82,8 @@ public AggregationReduceContext forFinalReduction() { 10, EsExecutors.daemonThreadFactory("test"), threadPool.getThreadContext(), - randomFrom(TaskTrackingConfig.DEFAULT, TaskTrackingConfig.DO_NOT_TRACK) + randomFrom(TaskTrackingConfig.DEFAULT, TaskTrackingConfig.DO_NOT_TRACK), + MeterRegistry.NOOP ); } diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java index bf81486087361..6b7cb1bc0dba7 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java @@ -67,6 +67,7 @@ import org.elasticsearch.search.suggest.phrase.PhraseSuggestion; import org.elasticsearch.search.suggest.term.TermSuggestion; import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.InternalAggregationTestCase; import org.elasticsearch.threadpool.TestThreadPool; @@ -137,7 +138,8 @@ public AggregationReduceContext forFinalReduction() { 10, EsExecutors.daemonThreadFactory("test"), threadPool.getThreadContext(), - randomFrom(TaskTrackingConfig.DEFAULT, TaskTrackingConfig.DO_NOT_TRACK) + randomFrom(TaskTrackingConfig.DEFAULT, TaskTrackingConfig.DO_NOT_TRACK), + MeterRegistry.NOOP ); } diff --git a/server/src/test/java/org/elasticsearch/common/util/concurrent/AbstractThrottledTaskRunnerTests.java b/server/src/test/java/org/elasticsearch/common/util/concurrent/AbstractThrottledTaskRunnerTests.java index cb5adcfc3d775..bbe21b20940cc 100644 --- a/server/src/test/java/org/elasticsearch/common/util/concurrent/AbstractThrottledTaskRunnerTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/concurrent/AbstractThrottledTaskRunnerTests.java @@ -13,6 +13,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.EsExecutors.TaskTrackingConfig; import org.elasticsearch.core.Releasable; +import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import java.util.concurrent.BlockingQueue; @@ -195,7 +196,15 @@ public void onResponse(Releasable releasable) { public void testFailsTasksOnRejectionOrShutdown() throws Exception { final var executor = randomBoolean() ? EsExecutors.newScaling("test", maxThreads, maxThreads, 0, TimeUnit.MILLISECONDS, true, threadFactory, threadContext) - : EsExecutors.newFixed("test", maxThreads, between(1, 5), threadFactory, threadContext, TaskTrackingConfig.DO_NOT_TRACK); + : EsExecutors.newFixed( + "test", + maxThreads, + between(1, 5), + threadFactory, + threadContext, + TaskTrackingConfig.DO_NOT_TRACK, + MeterRegistry.NOOP + ); final var totalPermits = between(1, maxThreads * 2); final var permits = new Semaphore(totalPermits); diff --git a/server/src/test/java/org/elasticsearch/common/util/concurrent/EsExecutorsTests.java b/server/src/test/java/org/elasticsearch/common/util/concurrent/EsExecutorsTests.java index 2867c9e007937..dce85333fa32c 100644 --- a/server/src/test/java/org/elasticsearch/common/util/concurrent/EsExecutorsTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/concurrent/EsExecutorsTests.java @@ -15,6 +15,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.Processors; import org.elasticsearch.node.Node; +import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; import org.hamcrest.Matcher; @@ -63,7 +64,8 @@ public void testFixedForcedExecution() throws Exception { 1, EsExecutors.daemonThreadFactory("test"), threadContext, - randomFrom(DEFAULT, DO_NOT_TRACK) + randomFrom(DEFAULT, DO_NOT_TRACK), + MeterRegistry.NOOP ); final CountDownLatch wait = new CountDownLatch(1); @@ -132,7 +134,8 @@ public void testFixedRejected() throws Exception { 1, EsExecutors.daemonThreadFactory("test"), threadContext, - randomFrom(DEFAULT, DO_NOT_TRACK) + randomFrom(DEFAULT, DO_NOT_TRACK), + MeterRegistry.NOOP ); final CountDownLatch wait = new CountDownLatch(1); @@ -277,7 +280,8 @@ public void testRejectionMessageAndShuttingDownFlag() throws InterruptedExceptio queue, EsExecutors.daemonThreadFactory("dummy"), threadContext, - randomFrom(DEFAULT, DO_NOT_TRACK) + randomFrom(DEFAULT, DO_NOT_TRACK), + MeterRegistry.NOOP ); try { for (int i = 0; i < actions; i++) { @@ -383,7 +387,8 @@ public void testInheritContext() throws InterruptedException { queue, EsExecutors.daemonThreadFactory("dummy"), threadContext, - randomFrom(DEFAULT, DO_NOT_TRACK) + randomFrom(DEFAULT, DO_NOT_TRACK), + MeterRegistry.NOOP ); try { executor.execute(() -> { @@ -420,7 +425,8 @@ public void testGetTasks() throws InterruptedException { queue, EsExecutors.daemonThreadFactory("dummy"), threadContext, - randomFrom(DEFAULT, DO_NOT_TRACK) + randomFrom(DEFAULT, DO_NOT_TRACK), + MeterRegistry.NOOP ); try { Runnable r = () -> { @@ -613,7 +619,8 @@ public void testFixedBoundedRejectOnShutdown() { between(1, 5), EsExecutors.daemonThreadFactory(getName()), threadContext, - randomFrom(DEFAULT, DO_NOT_TRACK) + randomFrom(DEFAULT, DO_NOT_TRACK), + MeterRegistry.NOOP ) ); } @@ -626,7 +633,8 @@ public void testFixedUnboundedRejectOnShutdown() { -1, EsExecutors.daemonThreadFactory(getName()), threadContext, - randomFrom(DEFAULT, DO_NOT_TRACK) + randomFrom(DEFAULT, DO_NOT_TRACK), + MeterRegistry.NOOP ) ); } @@ -677,7 +685,8 @@ public void testScalingWithTaskTimeTracking() { randomBoolean(), EsExecutors.daemonThreadFactory("test"), threadContext, - new EsExecutors.TaskTrackingConfig(randomBoolean(), randomDoubleBetween(0.01, 0.1, true)) + new EsExecutors.TaskTrackingConfig(randomBoolean(), randomDoubleBetween(0.01, 0.1, true)), + MeterRegistry.NOOP ); assertThat(pool, instanceOf(TaskExecutionTimeTrackingEsThreadPoolExecutor.class)); } @@ -705,8 +714,7 @@ public void testScalingWithTaskTimeTracking() { randomTimeUnit(), randomBoolean(), EsExecutors.daemonThreadFactory("test"), - threadContext, - DO_NOT_TRACK + threadContext ); assertThat(pool, instanceOf(EsThreadPoolExecutor.class)); } diff --git a/server/src/test/java/org/elasticsearch/common/util/concurrent/ListenableFutureTests.java b/server/src/test/java/org/elasticsearch/common/util/concurrent/ListenableFutureTests.java index 1e5a51e12421e..39f61324f3a6d 100644 --- a/server/src/test/java/org/elasticsearch/common/util/concurrent/ListenableFutureTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/concurrent/ListenableFutureTests.java @@ -13,6 +13,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ReachabilityChecker; import org.elasticsearch.transport.RemoteTransportException; @@ -84,7 +85,8 @@ public void testConcurrentListenerRegistrationAndCompletion() throws Interrupted 1000, EsExecutors.daemonThreadFactory("listener"), threadContext, - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, + MeterRegistry.NOOP ); final CyclicBarrier barrier = new CyclicBarrier(1 + numberOfThreads); final CountDownLatch listenersLatch = new CountDownLatch(numberOfThreads - 1); @@ -159,7 +161,8 @@ public void testRejection() { 1, EsExecutors.daemonThreadFactory("testRejection"), threadContext, - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, + MeterRegistry.NOOP ); try { diff --git a/server/src/test/java/org/elasticsearch/common/util/concurrent/PrioritizedThrottledTaskRunnerTests.java b/server/src/test/java/org/elasticsearch/common/util/concurrent/PrioritizedThrottledTaskRunnerTests.java index 7fc3e4ebb7f52..a5c3f9a2c9415 100644 --- a/server/src/test/java/org/elasticsearch/common/util/concurrent/PrioritizedThrottledTaskRunnerTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/concurrent/PrioritizedThrottledTaskRunnerTests.java @@ -11,6 +11,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.EsExecutors.TaskTrackingConfig; +import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; @@ -192,7 +193,15 @@ public void testEnqueueSpawnsNewTasksUpToMax() throws Exception { public void testFailsTasksOnRejectionOrShutdown() throws Exception { final var executor = randomBoolean() ? EsExecutors.newScaling("test", maxThreads, maxThreads, 0, TimeUnit.MILLISECONDS, true, threadFactory, threadContext) - : EsExecutors.newFixed("test", maxThreads, between(1, 5), threadFactory, threadContext, TaskTrackingConfig.DO_NOT_TRACK); + : EsExecutors.newFixed( + "test", + maxThreads, + between(1, 5), + threadFactory, + threadContext, + TaskTrackingConfig.DO_NOT_TRACK, + MeterRegistry.NOOP + ); final var taskRunner = new PrioritizedThrottledTaskRunner("test", between(1, maxThreads * 2), executor); final var totalPermits = between(1, maxThreads * 2); final var permits = new Semaphore(totalPermits); diff --git a/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java b/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java index b112ac0ceb6be..fb0fab4fb0fb0 100644 --- a/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java @@ -11,9 +11,18 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.EsExecutors.TaskTrackingConfig; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.telemetry.InstrumentType; +import org.elasticsearch.telemetry.Measurement; +import org.elasticsearch.telemetry.RecordingMeterRegistry; +import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.ThreadPool; +import java.util.List; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.function.Function; @@ -42,7 +51,8 @@ public void testExecutionEWMACalculation() throws Exception { EsExecutors.daemonThreadFactory("queuetest"), new EsAbortPolicy(), context, - new TaskTrackingConfig(randomBoolean(), DEFAULT_EWMA_ALPHA) + new TaskTrackingConfig(randomBoolean(), DEFAULT_EWMA_ALPHA), + MeterRegistry.NOOP ); executor.prestartAllCoreThreads(); logger.info("--> executor: {}", executor); @@ -94,7 +104,8 @@ public void testExceptionThrowingTask() throws Exception { EsExecutors.daemonThreadFactory("queuetest"), new EsAbortPolicy(), context, - new TaskTrackingConfig(randomBoolean(), DEFAULT_EWMA_ALPHA) + new TaskTrackingConfig(randomBoolean(), DEFAULT_EWMA_ALPHA), + MeterRegistry.NOOP ); executor.prestartAllCoreThreads(); logger.info("--> executor: {}", executor); @@ -126,7 +137,8 @@ public void testGetOngoingTasks() throws Exception { EsExecutors.daemonThreadFactory("queuetest"), new EsAbortPolicy(), context, - new TaskTrackingConfig(true, DEFAULT_EWMA_ALPHA) + new TaskTrackingConfig(true, DEFAULT_EWMA_ALPHA), + MeterRegistry.NOOP ); var taskRunningLatch = new CountDownLatch(1); var exitTaskLatch = new CountDownLatch(1); @@ -147,6 +159,53 @@ public void testGetOngoingTasks() throws Exception { executor.awaitTermination(10, TimeUnit.SECONDS); } + public void testQueueLatencyMetrics() { + RecordingMeterRegistry meterRegistry = new RecordingMeterRegistry(); + final var threadPoolName = randomIdentifier(); + var executor = new TaskExecutionTimeTrackingEsThreadPoolExecutor( + threadPoolName, + 1, + 1, + 1000, + TimeUnit.MILLISECONDS, + ConcurrentCollections.newBlockingQueue(), + TimedRunnable::new, + EsExecutors.daemonThreadFactory("queuetest"), + new EsAbortPolicy(), + new ThreadContext(Settings.EMPTY), + new TaskTrackingConfig(true, DEFAULT_EWMA_ALPHA), + meterRegistry + ); + + try { + final var barrier = new CyclicBarrier(2); + executor.execute(() -> { + safeAwait(barrier); + safeAwait(barrier); + }); + safeAwait(barrier); + + Future submit = executor.submit(() -> { + // Do nothing + }); + final long delayTimeMs = randomLongBetween(1, 30); + safeSleep(delayTimeMs); + safeAwait(barrier); + safeGet(submit); + + List measurements = meterRegistry.getRecorder() + .getMeasurements( + InstrumentType.DOUBLE_HISTOGRAM, + ThreadPool.THREAD_POOL_METRIC_PREFIX + threadPoolName + + TaskExecutionTimeTrackingEsThreadPoolExecutor.THREAD_POOL_METRIC_NAME_QUEUE_TIME + ); + assertEquals(2, measurements.size()); + assertThat(measurements.get(1).getDouble(), greaterThanOrEqualTo(TimeValue.timeValueMillis(delayTimeMs).secondsFrac())); + } finally { + ThreadPool.terminate(executor, 10, TimeUnit.SECONDS); + } + } + /** * The returned function outputs a WrappedRunnabled that simulates the case * where {@link TimedRunnable#getTotalExecutionNanos()} always returns {@code timeTakenNanos}. diff --git a/server/src/test/java/org/elasticsearch/search/DefaultSearchContextTests.java b/server/src/test/java/org/elasticsearch/search/DefaultSearchContextTests.java index 82737b13a1a46..af5f7530de00e 100644 --- a/server/src/test/java/org/elasticsearch/search/DefaultSearchContextTests.java +++ b/server/src/test/java/org/elasticsearch/search/DefaultSearchContextTests.java @@ -74,6 +74,7 @@ import org.elasticsearch.search.sort.FieldSortBuilder; import org.elasticsearch.search.sort.SortAndFormats; import org.elasticsearch.search.sort.SortBuilders; +import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xcontent.XContentBuilder; @@ -586,7 +587,8 @@ public void testDetermineMaximumNumberOfSlicesEnableQueryPhaseParallelCollection 0, Thread::new, new ThreadContext(Settings.EMPTY), - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, + MeterRegistry.NOOP ); ToLongFunction fieldCardinality = name -> -1; assertEquals( @@ -644,7 +646,8 @@ public void testDetermineMaximumNumberOfSlicesSingleSortByField() { 0, Thread::new, new ThreadContext(Settings.EMPTY), - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, + MeterRegistry.NOOP ); // DFS concurrency does not rely on slices, hence it kicks in regardless of the request (supportsParallelCollection is not called) assertEquals( @@ -677,7 +680,8 @@ public void testDetermineMaximumNumberOfSlicesWithQueue() { 1000, Thread::new, new ThreadContext(Settings.EMPTY), - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, + MeterRegistry.NOOP ); ToLongFunction fieldCardinality = name -> { throw new UnsupportedOperationException(); }; diff --git a/server/src/test/java/org/elasticsearch/transport/SingleResultDeduplicatorTests.java b/server/src/test/java/org/elasticsearch/transport/SingleResultDeduplicatorTests.java index ed30da78ef725..1fdcc8e6d63b6 100644 --- a/server/src/test/java/org/elasticsearch/transport/SingleResultDeduplicatorTests.java +++ b/server/src/test/java/org/elasticsearch/transport/SingleResultDeduplicatorTests.java @@ -21,6 +21,7 @@ import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; +import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; @@ -112,7 +113,8 @@ public void testThreadContextPreservation() { 0, EsExecutors.daemonThreadFactory("test"), threadContext, - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, + MeterRegistry.NOOP ); resources[0] = () -> ThreadPool.terminate(executor, 10, TimeUnit.SECONDS); final var barrier = new CyclicBarrier(threads); diff --git a/test/framework/src/main/java/org/elasticsearch/common/lucene/store/ESIndexInputTestCase.java b/test/framework/src/main/java/org/elasticsearch/common/lucene/store/ESIndexInputTestCase.java index 7738267ef0782..8cdc09c52fc66 100644 --- a/test/framework/src/main/java/org/elasticsearch/common/lucene/store/ESIndexInputTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/common/lucene/store/ESIndexInputTestCase.java @@ -18,6 +18,7 @@ import org.elasticsearch.common.util.concurrent.EsThreadPoolExecutor; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.store.LuceneFilesExtensions; +import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -46,7 +47,8 @@ public static void createExecutor() { 0, EsExecutors.daemonThreadFactory(name), new ThreadContext(Settings.EMPTY), - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, + MeterRegistry.NOOP ); } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/process/AbstractNativeProcessTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/process/AbstractNativeProcessTests.java index b128df2e91b27..a02429c4a7d83 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/process/AbstractNativeProcessTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/process/AbstractNativeProcessTests.java @@ -9,6 +9,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.ml.process.logging.CppLogMessageHandler; @@ -79,7 +80,8 @@ public void initialize() throws IOException { 1, EsExecutors.daemonThreadFactory("test"), new ThreadContext(Settings.EMPTY), - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, + MeterRegistry.NOOP ); } diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/WatcherService.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/WatcherService.java index 0ea9b432d3b0f..9c6e0faad8f9e 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/WatcherService.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/WatcherService.java @@ -31,6 +31,7 @@ import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.sort.SortBuilders; +import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xcontent.XContentType; import org.elasticsearch.xpack.core.ClientHelper; @@ -123,7 +124,8 @@ public class WatcherService { 1000, daemonThreadFactory(settings, LIFECYCLE_THREADPOOL_NAME), client.threadPool().getThreadContext(), - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, + MeterRegistry.NOOP ) ); } From dfef6762b56f57aaa4c458e4ea534d3f04270bcc Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Tue, 21 Jan 2025 14:08:07 +1030 Subject: [PATCH 02/33] Fix metric name --- .../TaskExecutionTimeTrackingEsThreadPoolExecutor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java index 9f7f56419ba9d..70629e231ed8c 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java @@ -30,7 +30,7 @@ */ public final class TaskExecutionTimeTrackingEsThreadPoolExecutor extends EsThreadPoolExecutor { - public static final String THREAD_POOL_METRIC_NAME_QUEUE_TIME = ".threads.queue.latency"; + public static final String THREAD_POOL_METRIC_NAME_QUEUE_TIME = ".threads.queue.latency.histogram"; private final Function runnableWrapper; private final ExponentiallyWeightedMovingAverage executionEWMA; From 2ceb96505e8901558d26df2a3f22586793a92685 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Tue, 21 Jan 2025 14:31:39 +1030 Subject: [PATCH 03/33] Temporary hack to fix metric name --- ...xecutionTimeTrackingEsThreadPoolExecutor.java | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java index 70629e231ed8c..4e71ad27078ad 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java @@ -58,13 +58,25 @@ public final class TaskExecutionTimeTrackingEsThreadPoolExecutor extends EsThrea this.runnableWrapper = runnableWrapper; this.executionEWMA = new ExponentiallyWeightedMovingAverage(trackingConfig.getEwmaAlpha(), 0); this.trackOngoingTasks = trackingConfig.trackOngoingTasks(); + final var threadPoolName = stripNodeName(name); this.queueWaitTimes = meterRegistry.registerDoubleHistogram( - ThreadPool.THREAD_POOL_METRIC_PREFIX + name + THREAD_POOL_METRIC_NAME_QUEUE_TIME, - "Distribution of time spent in " + name + " thread pool queue", + ThreadPool.THREAD_POOL_METRIC_PREFIX + threadPoolName + THREAD_POOL_METRIC_NAME_QUEUE_TIME, + "Distribution of time spent in " + threadPoolName + " thread pool queue", "seconds" ); } + /** + * TODO: Find a way to avoid the need for this + */ + private String stripNodeName(String name) { + int indexOfSlash = name.indexOf('/'); + if (indexOfSlash != -1) { + return name.substring(indexOfSlash + 1); + } + return name; + } + @Override protected Runnable wrapRunnable(Runnable command) { return super.wrapRunnable(this.runnableWrapper.apply(command)); From dbed27f1adb962e289d4c87bdb0619a914e16917 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Wed, 22 Jan 2025 21:38:31 +1030 Subject: [PATCH 04/33] Propose solution to composite thread-pool names --- .../common/util/concurrent/EsExecutors.java | 31 ++++++++++++++++--- ...utionTimeTrackingEsThreadPoolExecutor.java | 30 +++++++++--------- .../threadpool/ScalingExecutorBuilder.java | 2 +- .../util/concurrent/EsExecutorsTests.java | 2 +- ...TimeTrackingEsThreadPoolExecutorTests.java | 8 ++--- 5 files changed, 47 insertions(+), 26 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java index 9f1711794ae40..16d5d7638e156 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java @@ -14,6 +14,7 @@ import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.Processors; +import org.elasticsearch.core.Nullable; import org.elasticsearch.core.SuppressForbidden; import org.elasticsearch.node.Node; import org.elasticsearch.telemetry.metric.MeterRegistry; @@ -98,7 +99,7 @@ public static PrioritizedEsThreadPoolExecutor newSinglePrioritizing( } public static EsThreadPoolExecutor newScaling( - String name, + QualifiedName name, int min, int max, long keepAliveTime, @@ -128,7 +129,7 @@ public static EsThreadPoolExecutor newScaling( ); } else { executor = new EsThreadPoolExecutor( - name, + name.toCompositeString(), min, max, keepAliveTime, @@ -154,7 +155,7 @@ public static EsThreadPoolExecutor newScaling( ThreadContext contextHolder ) { return newScaling( - name, + new QualifiedName(name), min, max, keepAliveTime, @@ -175,6 +176,18 @@ public static EsThreadPoolExecutor newFixed( ThreadContext contextHolder, TaskTrackingConfig config, MeterRegistry meterRegistry + ) { + return newFixed(new QualifiedName(name), size, queueCapacity, threadFactory, contextHolder, config, meterRegistry); + } + + public static EsThreadPoolExecutor newFixed( + QualifiedName name, + int size, + int queueCapacity, + ThreadFactory threadFactory, + ThreadContext contextHolder, + TaskTrackingConfig config, + MeterRegistry meterRegistry ) { final BlockingQueue queue; final EsRejectedExecutionHandler rejectedExecutionHandler; @@ -202,7 +215,7 @@ public static EsThreadPoolExecutor newFixed( ); } else { return new EsThreadPoolExecutor( - name, + name.toCompositeString(), size, size, 0, @@ -548,4 +561,14 @@ public double getEwmaAlpha() { } } + public record QualifiedName(@Nullable String nodeName, String threadPoolName) { + + public QualifiedName(String threadPoolName) { + this(null, threadPoolName); + } + + public String toCompositeString() { + return nodeName == null ? threadPoolName : nodeName + "/" + threadPoolName; + } + } } diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java index 4e71ad27078ad..9e06a14c07008 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java @@ -41,7 +41,7 @@ public final class TaskExecutionTimeTrackingEsThreadPoolExecutor extends EsThrea private final DoubleHistogram queueWaitTimes; TaskExecutionTimeTrackingEsThreadPoolExecutor( - String name, + EsExecutors.QualifiedName name, int corePoolSize, int maximumPoolSize, long keepAliveTime, @@ -54,29 +54,27 @@ public final class TaskExecutionTimeTrackingEsThreadPoolExecutor extends EsThrea TaskTrackingConfig trackingConfig, MeterRegistry meterRegistry ) { - super(name, corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory, handler, contextHolder); + super( + name.toCompositeString(), + corePoolSize, + maximumPoolSize, + keepAliveTime, + unit, + workQueue, + threadFactory, + handler, + contextHolder + ); this.runnableWrapper = runnableWrapper; this.executionEWMA = new ExponentiallyWeightedMovingAverage(trackingConfig.getEwmaAlpha(), 0); this.trackOngoingTasks = trackingConfig.trackOngoingTasks(); - final var threadPoolName = stripNodeName(name); this.queueWaitTimes = meterRegistry.registerDoubleHistogram( - ThreadPool.THREAD_POOL_METRIC_PREFIX + threadPoolName + THREAD_POOL_METRIC_NAME_QUEUE_TIME, - "Distribution of time spent in " + threadPoolName + " thread pool queue", + ThreadPool.THREAD_POOL_METRIC_PREFIX + name.threadPoolName() + THREAD_POOL_METRIC_NAME_QUEUE_TIME, + "Distribution of time spent in " + name.threadPoolName() + " thread pool queue", "seconds" ); } - /** - * TODO: Find a way to avoid the need for this - */ - private String stripNodeName(String name) { - int indexOfSlash = name.indexOf('/'); - if (indexOfSlash != -1) { - return name.substring(indexOfSlash + 1); - } - return name; - } - @Override protected Runnable wrapRunnable(Runnable command) { return super.wrapRunnable(this.runnableWrapper.apply(command)); diff --git a/server/src/main/java/org/elasticsearch/threadpool/ScalingExecutorBuilder.java b/server/src/main/java/org/elasticsearch/threadpool/ScalingExecutorBuilder.java index c458cf831eb26..ef81ef37a2f26 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/ScalingExecutorBuilder.java +++ b/server/src/main/java/org/elasticsearch/threadpool/ScalingExecutorBuilder.java @@ -139,7 +139,7 @@ ThreadPool.ExecutorHolder build( final ThreadFactory threadFactory = EsExecutors.daemonThreadFactory(settings.nodeName, name()); ExecutorService executor; executor = EsExecutors.newScaling( - settings.nodeName + "/" + name(), + new EsExecutors.QualifiedName(settings.nodeName, name()), core, max, keepAlive.millis(), diff --git a/server/src/test/java/org/elasticsearch/common/util/concurrent/EsExecutorsTests.java b/server/src/test/java/org/elasticsearch/common/util/concurrent/EsExecutorsTests.java index dce85333fa32c..13537237b1b3c 100644 --- a/server/src/test/java/org/elasticsearch/common/util/concurrent/EsExecutorsTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/concurrent/EsExecutorsTests.java @@ -677,7 +677,7 @@ public void testScalingWithTaskTimeTracking() { { ThreadPoolExecutor pool = EsExecutors.newScaling( - getClass().getName() + "/" + getTestName(), + new EsExecutors.QualifiedName(getClass().getName(), getTestName()), min, max, between(1, 100), diff --git a/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java b/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java index fb0fab4fb0fb0..3c4e2bad5885a 100644 --- a/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java @@ -41,7 +41,7 @@ public void testExecutionEWMACalculation() throws Exception { ThreadContext context = new ThreadContext(Settings.EMPTY); TaskExecutionTimeTrackingEsThreadPoolExecutor executor = new TaskExecutionTimeTrackingEsThreadPoolExecutor( - "test-threadpool", + new EsExecutors.QualifiedName("test-threadpool"), 1, 1, 1000, @@ -94,7 +94,7 @@ public void testExecutionEWMACalculation() throws Exception { public void testExceptionThrowingTask() throws Exception { ThreadContext context = new ThreadContext(Settings.EMPTY); TaskExecutionTimeTrackingEsThreadPoolExecutor executor = new TaskExecutionTimeTrackingEsThreadPoolExecutor( - "test-threadpool", + new EsExecutors.QualifiedName("test-threadpool"), 1, 1, 1000, @@ -127,7 +127,7 @@ public void testGetOngoingTasks() throws Exception { var testStartTimeNanos = System.nanoTime(); ThreadContext context = new ThreadContext(Settings.EMPTY); var executor = new TaskExecutionTimeTrackingEsThreadPoolExecutor( - "test-threadpool", + new EsExecutors.QualifiedName("test-threadpool"), 1, 1, 1000, @@ -163,7 +163,7 @@ public void testQueueLatencyMetrics() { RecordingMeterRegistry meterRegistry = new RecordingMeterRegistry(); final var threadPoolName = randomIdentifier(); var executor = new TaskExecutionTimeTrackingEsThreadPoolExecutor( - threadPoolName, + new EsExecutors.QualifiedName(threadPoolName), 1, 1, 1000, From c95f6258fb8542b608316f116cb2aa66b3f60702 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Thu, 23 Jan 2025 09:21:55 +1030 Subject: [PATCH 05/33] Fix fixed thread pool names --- .../java/org/elasticsearch/threadpool/FixedExecutorBuilder.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/threadpool/FixedExecutorBuilder.java b/server/src/main/java/org/elasticsearch/threadpool/FixedExecutorBuilder.java index 168a9959bc95d..bd25ff3dbbb9d 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/FixedExecutorBuilder.java +++ b/server/src/main/java/org/elasticsearch/threadpool/FixedExecutorBuilder.java @@ -152,7 +152,7 @@ ThreadPool.ExecutorHolder build( int queueSize = settings.queueSize; final ThreadFactory threadFactory = EsExecutors.daemonThreadFactory(settings.nodeName, name(), isSystemThread()); final ExecutorService executor = EsExecutors.newFixed( - settings.nodeName + "/" + name(), + new EsExecutors.QualifiedName(settings.nodeName, name()), size, queueSize, threadFactory, From e7f5bb6a5086f4c3a4dd214ba033b6f6559a4d06 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Tue, 11 Mar 2025 15:12:01 +1100 Subject: [PATCH 06/33] POC using HandlingTimeTracker to track queue latency --- .../common/network/HandlingTimeTracker.java | 30 +++++++++++++ ...utionTimeTrackingEsThreadPoolExecutor.java | 32 ++++++++++--- .../network/HandlingTimeTrackerTests.java | 16 +++++++ ...TimeTrackingEsThreadPoolExecutorTests.java | 45 +++++++++++++------ 4 files changed, 102 insertions(+), 21 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/network/HandlingTimeTracker.java b/server/src/main/java/org/elasticsearch/common/network/HandlingTimeTracker.java index ce2c8678e1703..46943d9488787 100644 --- a/server/src/main/java/org/elasticsearch/common/network/HandlingTimeTracker.java +++ b/server/src/main/java/org/elasticsearch/common/network/HandlingTimeTracker.java @@ -9,6 +9,7 @@ package org.elasticsearch.common.network; +import java.util.Arrays; import java.util.concurrent.atomic.LongAdder; /** @@ -63,4 +64,33 @@ public long[] getHistogram() { return histogram; } + /** + * Calculate the Nth percentile value + * + * @param percentile The percentile as a fraction (in [0, 1.0]) + * @return A value greater than or equal to the specified fraction of values in the histogram + */ + public long getPercentile(float percentile) { + assert percentile >= 0 && percentile <= 1; + long[] snapshot = getHistogram(); + long totalCount = Arrays.stream(snapshot).reduce(0L, Long::sum); + long percentileIndex = (long) Math.ceil(totalCount * percentile); + for (int i = 0; i < BUCKET_COUNT; i++) { + percentileIndex -= snapshot[i]; + if (percentileIndex <= 0) { + return getBucketUpperBounds()[i]; + } + } + assert false : "We shouldn't ever get here"; + return Long.MAX_VALUE; + } + + /** + * Clear all values in the histogram (non-atomic) + */ + public void clear() { + for (int i = 0; i < BUCKET_COUNT; i++) { + buckets[i].reset(); + } + } } diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java index 9e06a14c07008..9ea130879b8df 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java @@ -10,12 +10,16 @@ package org.elasticsearch.common.util.concurrent; import org.elasticsearch.common.ExponentiallyWeightedMovingAverage; +import org.elasticsearch.common.network.HandlingTimeTracker; import org.elasticsearch.common.util.concurrent.EsExecutors.TaskTrackingConfig; import org.elasticsearch.core.TimeValue; -import org.elasticsearch.telemetry.metric.DoubleHistogram; +import org.elasticsearch.telemetry.metric.LongGauge; +import org.elasticsearch.telemetry.metric.LongWithAttributes; import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.threadpool.ThreadPool; +import java.util.Arrays; +import java.util.List; import java.util.Map; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; @@ -30,7 +34,8 @@ */ public final class TaskExecutionTimeTrackingEsThreadPoolExecutor extends EsThreadPoolExecutor { - public static final String THREAD_POOL_METRIC_NAME_QUEUE_TIME = ".threads.queue.latency.histogram"; + private static final int[] LATENCY_PERCENTILES_TO_REPORT = { 50, 90 }; + public static final String THREAD_POOL_METRIC_NAME_QUEUE_TIME = ".threads.queue.latency"; private final Function runnableWrapper; private final ExponentiallyWeightedMovingAverage executionEWMA; @@ -38,7 +43,8 @@ public final class TaskExecutionTimeTrackingEsThreadPoolExecutor extends EsThrea private final boolean trackOngoingTasks; // The set of currently running tasks and the timestamp of when they started execution in the Executor. private final Map ongoingTasks = new ConcurrentHashMap<>(); - private final DoubleHistogram queueWaitTimes; + private final HandlingTimeTracker handlingTimeTracker = new HandlingTimeTracker(); + private final LongGauge queueLatencyGauge; TaskExecutionTimeTrackingEsThreadPoolExecutor( EsExecutors.QualifiedName name, @@ -68,10 +74,22 @@ public final class TaskExecutionTimeTrackingEsThreadPoolExecutor extends EsThrea this.runnableWrapper = runnableWrapper; this.executionEWMA = new ExponentiallyWeightedMovingAverage(trackingConfig.getEwmaAlpha(), 0); this.trackOngoingTasks = trackingConfig.trackOngoingTasks(); - this.queueWaitTimes = meterRegistry.registerDoubleHistogram( + this.queueLatencyGauge = meterRegistry.registerLongsGauge( ThreadPool.THREAD_POOL_METRIC_PREFIX + name.threadPoolName() + THREAD_POOL_METRIC_NAME_QUEUE_TIME, - "Distribution of time spent in " + name.threadPoolName() + " thread pool queue", - "seconds" + "Time tasks spent in the queue for the " + name.threadPoolName() + " thread pool", + "milliseconds", + () -> { + List metricValues = Arrays.stream(LATENCY_PERCENTILES_TO_REPORT) + .mapToObj( + percentile -> new LongWithAttributes( + handlingTimeTracker.getPercentile(percentile / 100f), + Map.of("percentile", String.valueOf(percentile)) + ) + ) + .toList(); + handlingTimeTracker.clear(); + return metricValues; + } ); } @@ -147,7 +165,7 @@ protected void afterExecute(Runnable r, Throwable t) { + ", failedOrRejected: " + failedOrRejected; if (taskQueueLatency != -1) { - queueWaitTimes.record(TimeValue.timeValueNanos(taskQueueLatency).secondsFrac()); + handlingTimeTracker.addHandlingTime(TimeUnit.NANOSECONDS.toMillis(taskQueueLatency)); } } finally { // if trackOngoingTasks is false -> ongoingTasks must be empty diff --git a/server/src/test/java/org/elasticsearch/common/network/HandlingTimeTrackerTests.java b/server/src/test/java/org/elasticsearch/common/network/HandlingTimeTrackerTests.java index 98d25c29f25d1..7cb59f6b14ecd 100644 --- a/server/src/test/java/org/elasticsearch/common/network/HandlingTimeTrackerTests.java +++ b/server/src/test/java/org/elasticsearch/common/network/HandlingTimeTrackerTests.java @@ -9,9 +9,11 @@ package org.elasticsearch.common.network; +import org.elasticsearch.core.Strings; import org.elasticsearch.test.ESTestCase; import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; public class HandlingTimeTrackerTests extends ESTestCase { @@ -81,4 +83,18 @@ public void testBoundsConsistency() { } } + public void testPercentile() { + HandlingTimeTracker tt = new HandlingTimeTracker(); + int valueCount = randomIntBetween(100, 10_000); + for (int i = 0; i < valueCount; i++) { + tt.addHandlingTime(i); + } + for (int i = 0; i <= 100; i++) { + final float percentile = i / 100.0f; + final long actualPercentile = (long) Math.ceil(valueCount * percentile); + final long histogramPercentile = tt.getPercentile(percentile); + final String message = Strings.format("%d percentile is %d (actual=%d)", i, histogramPercentile, actualPercentile); + assertThat(message, histogramPercentile, greaterThanOrEqualTo(actualPercentile)); + } + } } diff --git a/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java b/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java index 3c4e2bad5885a..dd95ea588cbf2 100644 --- a/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java @@ -9,9 +9,9 @@ package org.elasticsearch.common.util.concurrent; +import org.elasticsearch.common.network.HandlingTimeTracker; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.EsExecutors.TaskTrackingConfig; -import org.elasticsearch.core.TimeValue; import org.elasticsearch.telemetry.InstrumentType; import org.elasticsearch.telemetry.Measurement; import org.elasticsearch.telemetry.RecordingMeterRegistry; @@ -30,6 +30,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.hasSize; /** * Tests for the automatic queue resizing of the {@code QueueResizingEsThreadPoolExecutorTests} @@ -179,33 +180,49 @@ public void testQueueLatencyMetrics() { try { final var barrier = new CyclicBarrier(2); - executor.execute(() -> { + final HandlingTimeTracker handlingTimeTracker = new HandlingTimeTracker(); + Future runningTask = executor.submit(() -> { safeAwait(barrier); safeAwait(barrier); }); - safeAwait(barrier); - - Future submit = executor.submit(() -> { - // Do nothing - }); - final long delayTimeMs = randomLongBetween(1, 30); - safeSleep(delayTimeMs); - safeAwait(barrier); - safeGet(submit); + safeAwait(barrier); // wait till first task starts + handlingTimeTracker.addHandlingTime(0L); // first task should not be delayed + for (int i = 0; i < 10; i++) { + Future waitingTask = executor.submit(() -> { + safeAwait(barrier); + safeAwait(barrier); + }); + final long delayTimeMs = randomLongBetween(1, 50); + safeSleep(delayTimeMs); + safeAwait(barrier); // let running task complete + safeAwait(barrier); // wait for next task to start + safeGet(runningTask); // ensure previous task is complete + handlingTimeTracker.addHandlingTime(delayTimeMs); + runningTask = waitingTask; + } + safeAwait(barrier); // let last task finish + safeGet(runningTask); + meterRegistry.getRecorder().collect(); List measurements = meterRegistry.getRecorder() .getMeasurements( - InstrumentType.DOUBLE_HISTOGRAM, + InstrumentType.LONG_GAUGE, ThreadPool.THREAD_POOL_METRIC_PREFIX + threadPoolName + TaskExecutionTimeTrackingEsThreadPoolExecutor.THREAD_POOL_METRIC_NAME_QUEUE_TIME ); - assertEquals(2, measurements.size()); - assertThat(measurements.get(1).getDouble(), greaterThanOrEqualTo(TimeValue.timeValueMillis(delayTimeMs).secondsFrac())); + assertThat(measurements, hasSize(2)); + // we have to use greater than or equal to because the actual delay might be higher than what we imposed + assertThat(getPercentile(measurements, "90"), greaterThanOrEqualTo(handlingTimeTracker.getPercentile(0.9f))); + assertThat(getPercentile(measurements, "50"), greaterThanOrEqualTo(handlingTimeTracker.getPercentile(0.5f))); } finally { ThreadPool.terminate(executor, 10, TimeUnit.SECONDS); } } + private long getPercentile(List measurements, String percentile) { + return measurements.stream().filter(m -> m.attributes().get("percentile").equals(percentile)).findFirst().orElseThrow().getLong(); + } + /** * The returned function outputs a WrappedRunnabled that simulates the case * where {@link TimedRunnable#getTotalExecutionNanos()} always returns {@code timeTakenNanos}. From 80b8b3f3136a172be2a600ddf7c1f4b69a427223 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Tue, 11 Mar 2025 16:03:28 +1100 Subject: [PATCH 07/33] Tidy --- .../org/elasticsearch/common/network/HandlingTimeTracker.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/network/HandlingTimeTracker.java b/server/src/main/java/org/elasticsearch/common/network/HandlingTimeTracker.java index 46943d9488787..7c8a49eeb76ff 100644 --- a/server/src/main/java/org/elasticsearch/common/network/HandlingTimeTracker.java +++ b/server/src/main/java/org/elasticsearch/common/network/HandlingTimeTracker.java @@ -72,8 +72,8 @@ public long[] getHistogram() { */ public long getPercentile(float percentile) { assert percentile >= 0 && percentile <= 1; - long[] snapshot = getHistogram(); - long totalCount = Arrays.stream(snapshot).reduce(0L, Long::sum); + final long[] snapshot = getHistogram(); + final long totalCount = Arrays.stream(snapshot).reduce(0L, Long::sum); long percentileIndex = (long) Math.ceil(totalCount * percentile); for (int i = 0; i < BUCKET_COUNT; i++) { percentileIndex -= snapshot[i]; From 598d0a89c42f5f77b88549d49ccd2911bf4478d6 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Thu, 13 Mar 2025 11:01:44 +1100 Subject: [PATCH 08/33] Fix metric name --- .../TaskExecutionTimeTrackingEsThreadPoolExecutor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java index 9ea130879b8df..6bcd1ba3bde3f 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java @@ -35,7 +35,7 @@ public final class TaskExecutionTimeTrackingEsThreadPoolExecutor extends EsThreadPoolExecutor { private static final int[] LATENCY_PERCENTILES_TO_REPORT = { 50, 90 }; - public static final String THREAD_POOL_METRIC_NAME_QUEUE_TIME = ".threads.queue.latency"; + public static final String THREAD_POOL_METRIC_NAME_QUEUE_TIME = ".threads.queue.latency.histogram"; private final Function runnableWrapper; private final ExponentiallyWeightedMovingAverage executionEWMA; From 4153d27e1264a514084be27734cad3346f3287b0 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Wed, 26 Mar 2025 10:15:36 +1100 Subject: [PATCH 09/33] Generalise HandlingTimeTracker --- .../ExponentialBucketHistogram.java} | 18 +++-- .../common/network/NetworkService.java | 5 +- ...utionTimeTrackingEsThreadPoolExecutor.java | 10 +-- .../http/AbstractHttpServerTransport.java | 2 +- .../elasticsearch/http/HttpRouteStats.java | 6 +- .../http/HttpRouteStatsTracker.java | 6 +- .../transport/InboundHandler.java | 10 +-- .../transport/OutboundHandler.java | 8 +-- .../elasticsearch/transport/TcpTransport.java | 4 +- .../transport/TransportStats.java | 14 ++-- .../cluster/node/stats/NodeStatsTests.java | 6 +- .../ExponentialBucketHistogramTests.java} | 72 +++++++++---------- ...TimeTrackingEsThreadPoolExecutorTests.java | 12 ++-- .../http/HttpRouteStatsTests.java | 4 +- .../transport/InboundHandlerTests.java | 6 +- .../transport/OutboundHandlerTests.java | 4 +- .../transport/TcpTransportTests.java | 4 +- .../transport/TransportStatsTests.java | 10 +-- .../transport/TestTransportChannels.java | 4 +- 19 files changed, 106 insertions(+), 99 deletions(-) rename server/src/main/java/org/elasticsearch/common/{network/HandlingTimeTracker.java => metrics/ExponentialBucketHistogram.java} (85%) rename server/src/test/java/org/elasticsearch/common/{network/HandlingTimeTrackerTests.java => metrics/ExponentialBucketHistogramTests.java} (55%) diff --git a/server/src/main/java/org/elasticsearch/common/network/HandlingTimeTracker.java b/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java similarity index 85% rename from server/src/main/java/org/elasticsearch/common/network/HandlingTimeTracker.java rename to server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java index 7c8a49eeb76ff..5a13e3e796845 100644 --- a/server/src/main/java/org/elasticsearch/common/network/HandlingTimeTracker.java +++ b/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java @@ -7,15 +7,21 @@ * License v3.0 only", or the "Server Side Public License, v 1". */ -package org.elasticsearch.common.network; +package org.elasticsearch.common.metrics; import java.util.Arrays; import java.util.concurrent.atomic.LongAdder; /** - * Tracks how long message handling takes on a transport thread as a histogram with fixed buckets. + * A histogram with a fixed number of buckets of exponentially increasing width. + * + * The upper bounds of the buckets are defined by increasing powers of two, e.g. + * + * 1, 2, 4, 8, 16... + * + * There are {@link #BUCKET_COUNT} buckets */ -public class HandlingTimeTracker { +public class ExponentialBucketHistogram { public static int[] getBucketUpperBounds() { int[] bounds = new int[17]; @@ -41,15 +47,15 @@ private static int getBucket(long handlingTimeMillis) { private final LongAdder[] buckets; - public HandlingTimeTracker() { + public ExponentialBucketHistogram() { buckets = new LongAdder[BUCKET_COUNT]; for (int i = 0; i < BUCKET_COUNT; i++) { buckets[i] = new LongAdder(); } } - public void addHandlingTime(long handlingTimeMillis) { - buckets[getBucket(handlingTimeMillis)].increment(); + public void addObservation(long observedValue) { + buckets[getBucket(observedValue)].increment(); } /** diff --git a/server/src/main/java/org/elasticsearch/common/network/NetworkService.java b/server/src/main/java/org/elasticsearch/common/network/NetworkService.java index 70e4952261ac3..f6a6b6f4e32e0 100644 --- a/server/src/main/java/org/elasticsearch/common/network/NetworkService.java +++ b/server/src/main/java/org/elasticsearch/common/network/NetworkService.java @@ -9,6 +9,7 @@ package org.elasticsearch.common.network; +import org.elasticsearch.common.metrics.ExponentialBucketHistogram; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.unit.ByteSizeValue; @@ -85,14 +86,14 @@ public interface CustomNameResolver { } private final List customNameResolvers; - private final HandlingTimeTracker handlingTimeTracker = new HandlingTimeTracker(); + private final ExponentialBucketHistogram handlingTimeTracker = new ExponentialBucketHistogram(); private final ThreadWatchdog threadWatchdog = new ThreadWatchdog(); public NetworkService(List customNameResolvers) { this.customNameResolvers = Objects.requireNonNull(customNameResolvers, "customNameResolvers must be non null"); } - public HandlingTimeTracker getHandlingTimeTracker() { + public ExponentialBucketHistogram getHandlingTimeTracker() { return handlingTimeTracker; } diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java index 6bcd1ba3bde3f..3a99ae6b667f1 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java @@ -10,7 +10,7 @@ package org.elasticsearch.common.util.concurrent; import org.elasticsearch.common.ExponentiallyWeightedMovingAverage; -import org.elasticsearch.common.network.HandlingTimeTracker; +import org.elasticsearch.common.metrics.ExponentialBucketHistogram; import org.elasticsearch.common.util.concurrent.EsExecutors.TaskTrackingConfig; import org.elasticsearch.core.TimeValue; import org.elasticsearch.telemetry.metric.LongGauge; @@ -43,7 +43,7 @@ public final class TaskExecutionTimeTrackingEsThreadPoolExecutor extends EsThrea private final boolean trackOngoingTasks; // The set of currently running tasks and the timestamp of when they started execution in the Executor. private final Map ongoingTasks = new ConcurrentHashMap<>(); - private final HandlingTimeTracker handlingTimeTracker = new HandlingTimeTracker(); + private final ExponentialBucketHistogram queueLatencyHistogram = new ExponentialBucketHistogram(); private final LongGauge queueLatencyGauge; TaskExecutionTimeTrackingEsThreadPoolExecutor( @@ -82,12 +82,12 @@ public final class TaskExecutionTimeTrackingEsThreadPoolExecutor extends EsThrea List metricValues = Arrays.stream(LATENCY_PERCENTILES_TO_REPORT) .mapToObj( percentile -> new LongWithAttributes( - handlingTimeTracker.getPercentile(percentile / 100f), + queueLatencyHistogram.getPercentile(percentile / 100f), Map.of("percentile", String.valueOf(percentile)) ) ) .toList(); - handlingTimeTracker.clear(); + queueLatencyHistogram.clear(); return metricValues; } ); @@ -165,7 +165,7 @@ protected void afterExecute(Runnable r, Throwable t) { + ", failedOrRejected: " + failedOrRejected; if (taskQueueLatency != -1) { - handlingTimeTracker.addHandlingTime(TimeUnit.NANOSECONDS.toMillis(taskQueueLatency)); + queueLatencyHistogram.addObservation(TimeUnit.NANOSECONDS.toMillis(taskQueueLatency)); } } finally { // if trackOngoingTasks is false -> ongoingTasks must be empty diff --git a/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java b/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java index aeca887947ce2..1a66a9340468f 100644 --- a/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java +++ b/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java @@ -460,7 +460,7 @@ public void incomingRequest(final HttpRequest httpRequest, final HttpChannel htt handleIncomingRequest(httpRequest, trackingChannel, httpRequest.getInboundException()); } finally { final long took = threadPool.rawRelativeTimeInMillis() - startTime; - networkService.getHandlingTimeTracker().addHandlingTime(took); + networkService.getHandlingTimeTracker().addObservation(took); final long logThreshold = slowLogThresholdMs; if (logThreshold > 0 && took > logThreshold) { logger.warn( diff --git a/server/src/main/java/org/elasticsearch/http/HttpRouteStats.java b/server/src/main/java/org/elasticsearch/http/HttpRouteStats.java index a15b929fd3c1b..667a35d79ea18 100644 --- a/server/src/main/java/org/elasticsearch/http/HttpRouteStats.java +++ b/server/src/main/java/org/elasticsearch/http/HttpRouteStats.java @@ -12,7 +12,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.network.HandlingTimeTracker; +import org.elasticsearch.common.metrics.ExponentialBucketHistogram; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.core.TimeValue; import org.elasticsearch.xcontent.ToXContentObject; @@ -36,7 +36,7 @@ * @param totalResponseSize the total body size (bytes) of responses produced by the HTTP route * @param responseSizeHistogram similar to {@code requestSizeHistogram} but for response size * @param responseTimeHistogram an array of frequencies of response time (millis) in buckets with upper bounds - * as returned by {@link HandlingTimeTracker#getBucketUpperBounds()}, plus + * as returned by {@link ExponentialBucketHistogram#getBucketUpperBounds()}, plus * an extra bucket for handling response time larger than the longest upper bound (currently 65536ms). */ public record HttpRouteStats( @@ -89,7 +89,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws "millis", TimeValue::timeValueMillis, responseTimeHistogram, - HandlingTimeTracker.getBucketUpperBounds() + ExponentialBucketHistogram.getBucketUpperBounds() ); builder.endObject(); diff --git a/server/src/main/java/org/elasticsearch/http/HttpRouteStatsTracker.java b/server/src/main/java/org/elasticsearch/http/HttpRouteStatsTracker.java index 652c0aca12517..79fc68a3a2ac1 100644 --- a/server/src/main/java/org/elasticsearch/http/HttpRouteStatsTracker.java +++ b/server/src/main/java/org/elasticsearch/http/HttpRouteStatsTracker.java @@ -9,7 +9,7 @@ package org.elasticsearch.http; -import org.elasticsearch.common.network.HandlingTimeTracker; +import org.elasticsearch.common.metrics.ExponentialBucketHistogram; import java.util.concurrent.atomic.AtomicLongArray; import java.util.concurrent.atomic.LongAdder; @@ -80,7 +80,7 @@ private static int bucket(int contentLength) { private final StatsTracker requestStats = new StatsTracker(); private final StatsTracker responseStats = new StatsTracker(); - private final HandlingTimeTracker responseTimeTracker = new HandlingTimeTracker(); + private final ExponentialBucketHistogram responseTimeTracker = new ExponentialBucketHistogram(); public void addRequestStats(int contentLength) { requestStats.addStats(contentLength); @@ -91,7 +91,7 @@ public void addResponseStats(long contentLength) { } public void addResponseTime(long timeMillis) { - responseTimeTracker.addHandlingTime(timeMillis); + responseTimeTracker.addObservation(timeMillis); } public HttpRouteStats getStats() { diff --git a/server/src/main/java/org/elasticsearch/transport/InboundHandler.java b/server/src/main/java/org/elasticsearch/transport/InboundHandler.java index 01069609ac742..d74415ac6216c 100644 --- a/server/src/main/java/org/elasticsearch/transport/InboundHandler.java +++ b/server/src/main/java/org/elasticsearch/transport/InboundHandler.java @@ -18,7 +18,7 @@ import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.network.HandlingTimeTracker; +import org.elasticsearch.common.metrics.ExponentialBucketHistogram; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.ThreadContext; @@ -46,7 +46,7 @@ public class InboundHandler { private final TransportKeepAlive keepAlive; private final Transport.ResponseHandlers responseHandlers; private final Transport.RequestHandlers requestHandlers; - private final HandlingTimeTracker handlingTimeTracker; + private final ExponentialBucketHistogram exponentialBucketHistogram; private final boolean ignoreDeserializationErrors; private volatile TransportMessageListener messageListener = TransportMessageListener.NOOP_LISTENER; @@ -61,7 +61,7 @@ public class InboundHandler { TransportKeepAlive keepAlive, Transport.RequestHandlers requestHandlers, Transport.ResponseHandlers responseHandlers, - HandlingTimeTracker handlingTimeTracker, + ExponentialBucketHistogram exponentialBucketHistogram, boolean ignoreDeserializationErrors ) { this.threadPool = threadPool; @@ -71,7 +71,7 @@ public class InboundHandler { this.keepAlive = keepAlive; this.requestHandlers = requestHandlers; this.responseHandlers = responseHandlers; - this.handlingTimeTracker = handlingTimeTracker; + this.exponentialBucketHistogram = exponentialBucketHistogram; this.ignoreDeserializationErrors = ignoreDeserializationErrors; } @@ -127,7 +127,7 @@ private void messageReceived(TcpChannel channel, InboundMessage message, long st } } finally { final long took = threadPool.rawRelativeTimeInMillis() - startTime; - handlingTimeTracker.addHandlingTime(took); + exponentialBucketHistogram.addObservation(took); final long logThreshold = slowLogThresholdMs; if (logThreshold > 0 && took > logThreshold) { logSlowMessage(message, took, logThreshold, responseHandler); diff --git a/server/src/main/java/org/elasticsearch/transport/OutboundHandler.java b/server/src/main/java/org/elasticsearch/transport/OutboundHandler.java index 3e727216b8693..104342f35be52 100644 --- a/server/src/main/java/org/elasticsearch/transport/OutboundHandler.java +++ b/server/src/main/java/org/elasticsearch/transport/OutboundHandler.java @@ -20,8 +20,8 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.bytes.ReleasableBytesReference; import org.elasticsearch.common.io.stream.RecyclerBytesStreamOutput; +import org.elasticsearch.common.metrics.ExponentialBucketHistogram; import org.elasticsearch.common.network.CloseableChannel; -import org.elasticsearch.common.network.HandlingTimeTracker; import org.elasticsearch.common.recycler.Recycler; import org.elasticsearch.common.transport.NetworkExceptionHelper; import org.elasticsearch.core.Nullable; @@ -47,7 +47,7 @@ final class OutboundHandler { private final StatsTracker statsTracker; private final ThreadPool threadPool; private final Recycler recycler; - private final HandlingTimeTracker handlingTimeTracker; + private final ExponentialBucketHistogram handlingTimeTracker; private final boolean rstOnClose; private volatile long slowLogThresholdMs = Long.MAX_VALUE; @@ -60,7 +60,7 @@ final class OutboundHandler { StatsTracker statsTracker, ThreadPool threadPool, Recycler recycler, - HandlingTimeTracker handlingTimeTracker, + ExponentialBucketHistogram handlingTimeTracker, boolean rstOnClose ) { this.nodeName = nodeName; @@ -275,7 +275,7 @@ private void maybeLogSlowMessage(boolean success) { final long logThreshold = slowLogThresholdMs; if (logThreshold > 0) { final long took = threadPool.rawRelativeTimeInMillis() - startTime; - handlingTimeTracker.addHandlingTime(took); + handlingTimeTracker.addObservation(took); if (took > logThreshold) { logger.warn( "sending transport message [{}] of size [{}] on [{}] took [{}ms] which is above the warn " diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java index 5eb51d3cadcc6..ff8766c981328 100644 --- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java +++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java @@ -27,8 +27,8 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.RecyclerBytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.metrics.ExponentialBucketHistogram; import org.elasticsearch.common.network.CloseableChannel; -import org.elasticsearch.common.network.HandlingTimeTracker; import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.network.NetworkUtils; @@ -128,7 +128,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements private final TransportHandshaker handshaker; private final TransportKeepAlive keepAlive; - private final HandlingTimeTracker outboundHandlingTimeTracker = new HandlingTimeTracker(); + private final ExponentialBucketHistogram outboundHandlingTimeTracker = new ExponentialBucketHistogram(); private final OutboundHandler outboundHandler; private final InboundHandler inboundHandler; private final ResponseHandlers responseHandlers = new ResponseHandlers(); diff --git a/server/src/main/java/org/elasticsearch/transport/TransportStats.java b/server/src/main/java/org/elasticsearch/transport/TransportStats.java index 2c06cd759e3a3..902e6d635eb98 100644 --- a/server/src/main/java/org/elasticsearch/transport/TransportStats.java +++ b/server/src/main/java/org/elasticsearch/transport/TransportStats.java @@ -14,7 +14,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.network.HandlingTimeTracker; +import org.elasticsearch.common.metrics.ExponentialBucketHistogram; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.xcontent.ChunkedToXContent; import org.elasticsearch.core.TimeValue; @@ -73,11 +73,11 @@ public TransportStats(StreamInput in) throws IOException { && in.getTransportVersion().isPatchFrom(TransportVersions.TRANSPORT_STATS_HANDLING_TIME_REQUIRED_90) == false) { in.readBoolean(); } - inboundHandlingTimeBucketFrequencies = new long[HandlingTimeTracker.BUCKET_COUNT]; + inboundHandlingTimeBucketFrequencies = new long[ExponentialBucketHistogram.BUCKET_COUNT]; for (int i = 0; i < inboundHandlingTimeBucketFrequencies.length; i++) { inboundHandlingTimeBucketFrequencies[i] = in.readVLong(); } - outboundHandlingTimeBucketFrequencies = new long[HandlingTimeTracker.BUCKET_COUNT]; + outboundHandlingTimeBucketFrequencies = new long[ExponentialBucketHistogram.BUCKET_COUNT]; for (int i = 0; i < inboundHandlingTimeBucketFrequencies.length; i++) { outboundHandlingTimeBucketFrequencies[i] = in.readVLong(); } @@ -97,8 +97,8 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVLong(rxSize); out.writeVLong(txCount); out.writeVLong(txSize); - assert inboundHandlingTimeBucketFrequencies.length == HandlingTimeTracker.BUCKET_COUNT; - assert outboundHandlingTimeBucketFrequencies.length == HandlingTimeTracker.BUCKET_COUNT; + assert inboundHandlingTimeBucketFrequencies.length == ExponentialBucketHistogram.BUCKET_COUNT; + assert outboundHandlingTimeBucketFrequencies.length == ExponentialBucketHistogram.BUCKET_COUNT; if (out.getTransportVersion().before(TransportVersions.TRANSPORT_STATS_HANDLING_TIME_REQUIRED) && out.getTransportVersion().isPatchFrom(TransportVersions.TRANSPORT_STATS_HANDLING_TIME_REQUIRED_90) == false) { out.writeBoolean(true); @@ -168,7 +168,7 @@ public Map getTransportActionStats() { private boolean assertHistogramsConsistent() { assert inboundHandlingTimeBucketFrequencies.length == outboundHandlingTimeBucketFrequencies.length; - assert inboundHandlingTimeBucketFrequencies.length == HandlingTimeTracker.BUCKET_COUNT; + assert inboundHandlingTimeBucketFrequencies.length == ExponentialBucketHistogram.BUCKET_COUNT; return true; } @@ -200,7 +200,7 @@ public Iterator toXContentChunked(ToXContent.Params outerP } static void histogramToXContent(XContentBuilder builder, long[] bucketFrequencies, String fieldName) throws IOException { - final int[] bucketBounds = HandlingTimeTracker.getBucketUpperBounds(); + final int[] bucketBounds = ExponentialBucketHistogram.getBucketUpperBounds(); int firstBucket = 0; long remainingCount = 0L; diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStatsTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStatsTests.java index 10915d385d5b3..88b2a5bf4d263 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStatsTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStatsTests.java @@ -28,7 +28,7 @@ import org.elasticsearch.cluster.service.ClusterStateUpdateStats; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.network.HandlingTimeTracker; +import org.elasticsearch.common.metrics.ExponentialBucketHistogram; import org.elasticsearch.common.util.Maps; import org.elasticsearch.common.xcontent.ChunkedToXContent; import org.elasticsearch.core.Tuple; @@ -843,8 +843,8 @@ public static NodeStats createNodeStats() { randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong(), - IntStream.range(0, HandlingTimeTracker.BUCKET_COUNT).mapToLong(i -> randomNonNegativeLong()).toArray(), - IntStream.range(0, HandlingTimeTracker.BUCKET_COUNT).mapToLong(i -> randomNonNegativeLong()).toArray(), + IntStream.range(0, ExponentialBucketHistogram.BUCKET_COUNT).mapToLong(i -> randomNonNegativeLong()).toArray(), + IntStream.range(0, ExponentialBucketHistogram.BUCKET_COUNT).mapToLong(i -> randomNonNegativeLong()).toArray(), Map.of("test-action", new TransportActionStats(1, 2, new long[29], 3, 4, new long[29])) ) : null; diff --git a/server/src/test/java/org/elasticsearch/common/network/HandlingTimeTrackerTests.java b/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java similarity index 55% rename from server/src/test/java/org/elasticsearch/common/network/HandlingTimeTrackerTests.java rename to server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java index 7cb59f6b14ecd..5c0beba341046 100644 --- a/server/src/test/java/org/elasticsearch/common/network/HandlingTimeTrackerTests.java +++ b/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java @@ -7,7 +7,7 @@ * License v3.0 only", or the "Server Side Public License, v 1". */ -package org.elasticsearch.common.network; +package org.elasticsearch.common.metrics; import org.elasticsearch.core.Strings; import org.elasticsearch.test.ESTestCase; @@ -15,68 +15,68 @@ import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; -public class HandlingTimeTrackerTests extends ESTestCase { +public class ExponentialBucketHistogramTests extends ESTestCase { public void testHistogram() { - final HandlingTimeTracker handlingTimeTracker = new HandlingTimeTracker(); + final ExponentialBucketHistogram histogram = new ExponentialBucketHistogram(); - assertArrayEquals(new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, handlingTimeTracker.getHistogram()); + assertArrayEquals(new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getHistogram()); - handlingTimeTracker.addHandlingTime(0L); - assertArrayEquals(new long[] { 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, handlingTimeTracker.getHistogram()); + histogram.addObservation(0L); + assertArrayEquals(new long[] { 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getHistogram()); - handlingTimeTracker.addHandlingTime(1L); - assertArrayEquals(new long[] { 1, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, handlingTimeTracker.getHistogram()); + histogram.addObservation(1L); + assertArrayEquals(new long[] { 1, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getHistogram()); - handlingTimeTracker.addHandlingTime(2L); - assertArrayEquals(new long[] { 1, 1, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, handlingTimeTracker.getHistogram()); + histogram.addObservation(2L); + assertArrayEquals(new long[] { 1, 1, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getHistogram()); - handlingTimeTracker.addHandlingTime(3L); - assertArrayEquals(new long[] { 1, 1, 2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, handlingTimeTracker.getHistogram()); + histogram.addObservation(3L); + assertArrayEquals(new long[] { 1, 1, 2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getHistogram()); - handlingTimeTracker.addHandlingTime(4L); - assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, handlingTimeTracker.getHistogram()); + histogram.addObservation(4L); + assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getHistogram()); - handlingTimeTracker.addHandlingTime(127L); - assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, handlingTimeTracker.getHistogram()); + histogram.addObservation(127L); + assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getHistogram()); - handlingTimeTracker.addHandlingTime(128L); - assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, handlingTimeTracker.getHistogram()); + histogram.addObservation(128L); + assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getHistogram()); - handlingTimeTracker.addHandlingTime(65535L); - assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 0 }, handlingTimeTracker.getHistogram()); + histogram.addObservation(65535L); + assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 0 }, histogram.getHistogram()); - handlingTimeTracker.addHandlingTime(65536L); - assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 1 }, handlingTimeTracker.getHistogram()); + histogram.addObservation(65536L); + assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 1 }, histogram.getHistogram()); - handlingTimeTracker.addHandlingTime(Long.MAX_VALUE); - assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 2 }, handlingTimeTracker.getHistogram()); + histogram.addObservation(Long.MAX_VALUE); + assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 2 }, histogram.getHistogram()); - handlingTimeTracker.addHandlingTime(randomLongBetween(65536L, Long.MAX_VALUE)); - assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 3 }, handlingTimeTracker.getHistogram()); + histogram.addObservation(randomLongBetween(65536L, Long.MAX_VALUE)); + assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 3 }, histogram.getHistogram()); - handlingTimeTracker.addHandlingTime(randomLongBetween(Long.MIN_VALUE, 0L)); - assertArrayEquals(new long[] { 2, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 3 }, handlingTimeTracker.getHistogram()); + histogram.addObservation(randomLongBetween(Long.MIN_VALUE, 0L)); + assertArrayEquals(new long[] { 2, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 3 }, histogram.getHistogram()); } public void testHistogramRandom() { - final int[] upperBounds = HandlingTimeTracker.getBucketUpperBounds(); + final int[] upperBounds = ExponentialBucketHistogram.getBucketUpperBounds(); final long[] expectedCounts = new long[upperBounds.length + 1]; - final HandlingTimeTracker handlingTimeTracker = new HandlingTimeTracker(); + final ExponentialBucketHistogram histogram = new ExponentialBucketHistogram(); for (int i = between(0, 1000); i > 0; i--) { final int bucket = between(0, expectedCounts.length - 1); expectedCounts[bucket] += 1; final int lowerBound = bucket == 0 ? 0 : upperBounds[bucket - 1]; final int upperBound = bucket == upperBounds.length ? randomBoolean() ? 100000 : Integer.MAX_VALUE : upperBounds[bucket] - 1; - handlingTimeTracker.addHandlingTime(between(lowerBound, upperBound)); + histogram.addObservation(between(lowerBound, upperBound)); } - assertArrayEquals(expectedCounts, handlingTimeTracker.getHistogram()); + assertArrayEquals(expectedCounts, histogram.getHistogram()); } public void testBoundsConsistency() { - final int[] upperBounds = HandlingTimeTracker.getBucketUpperBounds(); + final int[] upperBounds = ExponentialBucketHistogram.getBucketUpperBounds(); assertThat(upperBounds[0], greaterThan(0)); for (int i = 1; i < upperBounds.length; i++) { assertThat(upperBounds[i], greaterThan(upperBounds[i - 1])); @@ -84,15 +84,15 @@ public void testBoundsConsistency() { } public void testPercentile() { - HandlingTimeTracker tt = new HandlingTimeTracker(); + ExponentialBucketHistogram histogram = new ExponentialBucketHistogram(); int valueCount = randomIntBetween(100, 10_000); for (int i = 0; i < valueCount; i++) { - tt.addHandlingTime(i); + histogram.addObservation(i); } for (int i = 0; i <= 100; i++) { final float percentile = i / 100.0f; final long actualPercentile = (long) Math.ceil(valueCount * percentile); - final long histogramPercentile = tt.getPercentile(percentile); + final long histogramPercentile = histogram.getPercentile(percentile); final String message = Strings.format("%d percentile is %d (actual=%d)", i, histogramPercentile, actualPercentile); assertThat(message, histogramPercentile, greaterThanOrEqualTo(actualPercentile)); } diff --git a/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java b/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java index dd95ea588cbf2..b830a308142fb 100644 --- a/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java @@ -9,7 +9,7 @@ package org.elasticsearch.common.util.concurrent; -import org.elasticsearch.common.network.HandlingTimeTracker; +import org.elasticsearch.common.metrics.ExponentialBucketHistogram; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.EsExecutors.TaskTrackingConfig; import org.elasticsearch.telemetry.InstrumentType; @@ -180,13 +180,13 @@ public void testQueueLatencyMetrics() { try { final var barrier = new CyclicBarrier(2); - final HandlingTimeTracker handlingTimeTracker = new HandlingTimeTracker(); + final ExponentialBucketHistogram expectedHistogram = new ExponentialBucketHistogram(); Future runningTask = executor.submit(() -> { safeAwait(barrier); safeAwait(barrier); }); safeAwait(barrier); // wait till first task starts - handlingTimeTracker.addHandlingTime(0L); // first task should not be delayed + expectedHistogram.addObservation(0L); // first task should not be delayed for (int i = 0; i < 10; i++) { Future waitingTask = executor.submit(() -> { safeAwait(barrier); @@ -197,7 +197,7 @@ public void testQueueLatencyMetrics() { safeAwait(barrier); // let running task complete safeAwait(barrier); // wait for next task to start safeGet(runningTask); // ensure previous task is complete - handlingTimeTracker.addHandlingTime(delayTimeMs); + expectedHistogram.addObservation(delayTimeMs); runningTask = waitingTask; } safeAwait(barrier); // let last task finish @@ -212,8 +212,8 @@ public void testQueueLatencyMetrics() { ); assertThat(measurements, hasSize(2)); // we have to use greater than or equal to because the actual delay might be higher than what we imposed - assertThat(getPercentile(measurements, "90"), greaterThanOrEqualTo(handlingTimeTracker.getPercentile(0.9f))); - assertThat(getPercentile(measurements, "50"), greaterThanOrEqualTo(handlingTimeTracker.getPercentile(0.5f))); + assertThat(getPercentile(measurements, "90"), greaterThanOrEqualTo(expectedHistogram.getPercentile(0.9f))); + assertThat(getPercentile(measurements, "50"), greaterThanOrEqualTo(expectedHistogram.getPercentile(0.5f))); } finally { ThreadPool.terminate(executor, 10, TimeUnit.SECONDS); } diff --git a/server/src/test/java/org/elasticsearch/http/HttpRouteStatsTests.java b/server/src/test/java/org/elasticsearch/http/HttpRouteStatsTests.java index b2095238a8c88..796995edd3bc7 100644 --- a/server/src/test/java/org/elasticsearch/http/HttpRouteStatsTests.java +++ b/server/src/test/java/org/elasticsearch/http/HttpRouteStatsTests.java @@ -10,7 +10,7 @@ package org.elasticsearch.http; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.network.HandlingTimeTracker; +import org.elasticsearch.common.metrics.ExponentialBucketHistogram; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.core.TimeValue; import org.elasticsearch.test.ESTestCase; @@ -236,7 +236,7 @@ private static void assertHandlingTimeHistogram(long[] histogram, String expecte "millis", TimeValue::timeValueMillis, histogram, - HandlingTimeTracker.getBucketUpperBounds() + ExponentialBucketHistogram.getBucketUpperBounds() ); return builder; }, false, true)); diff --git a/server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java b/server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java index 0dc72cb9ce252..e75524c37d76b 100644 --- a/server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java @@ -22,7 +22,7 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.RecyclerBytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.network.HandlingTimeTracker; +import org.elasticsearch.common.metrics.ExponentialBucketHistogram; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.PageCacheRecycler; import org.elasticsearch.common.util.concurrent.EsExecutors; @@ -80,7 +80,7 @@ public void setUp() throws Exception { new StatsTracker(), threadPool, new BytesRefRecycler(PageCacheRecycler.NON_RECYCLING_INSTANCE), - new HandlingTimeTracker(), + new ExponentialBucketHistogram(), false ); requestHandlers = new Transport.RequestHandlers(); @@ -93,7 +93,7 @@ public void setUp() throws Exception { keepAlive, requestHandlers, responseHandlers, - new HandlingTimeTracker(), + new ExponentialBucketHistogram(), ignoreDeserializationErrors ); } diff --git a/server/src/test/java/org/elasticsearch/transport/OutboundHandlerTests.java b/server/src/test/java/org/elasticsearch/transport/OutboundHandlerTests.java index 8a63097d6441c..b20ffd3f83a34 100644 --- a/server/src/test/java/org/elasticsearch/transport/OutboundHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/transport/OutboundHandlerTests.java @@ -25,7 +25,7 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.network.HandlingTimeTracker; +import org.elasticsearch.common.metrics.ExponentialBucketHistogram; import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.util.PageCacheRecycler; @@ -83,7 +83,7 @@ public void setUp() throws Exception { statsTracker, threadPool, recycler, - new HandlingTimeTracker(), + new ExponentialBucketHistogram(), false ); diff --git a/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java b/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java index 7099c33dda75f..2067a8dedeae9 100644 --- a/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java +++ b/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java @@ -16,7 +16,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.component.Lifecycle; import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.network.HandlingTimeTracker; +import org.elasticsearch.common.metrics.ExponentialBucketHistogram; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.network.NetworkUtils; import org.elasticsearch.common.settings.Settings; @@ -599,7 +599,7 @@ private void testExceptionHandling( new StatsTracker(), testThreadPool, new BytesRefRecycler(new MockPageCacheRecycler(Settings.EMPTY)), - new HandlingTimeTracker(), + new ExponentialBucketHistogram(), false ) ); diff --git a/server/src/test/java/org/elasticsearch/transport/TransportStatsTests.java b/server/src/test/java/org/elasticsearch/transport/TransportStatsTests.java index 1c9cb4c9afc0f..e07b0ca9dc43e 100644 --- a/server/src/test/java/org/elasticsearch/transport/TransportStatsTests.java +++ b/server/src/test/java/org/elasticsearch/transport/TransportStatsTests.java @@ -10,7 +10,7 @@ package org.elasticsearch.transport; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.network.HandlingTimeTracker; +import org.elasticsearch.common.metrics.ExponentialBucketHistogram; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xcontent.ToXContentFragment; @@ -20,7 +20,7 @@ public class TransportStatsTests extends ESTestCase { public void testToXContent() { - final var histogram = new long[HandlingTimeTracker.BUCKET_COUNT]; + final var histogram = new long[ExponentialBucketHistogram.BUCKET_COUNT]; histogram[4] = 10; final var requestSizeHistogram = new long[29]; @@ -67,7 +67,7 @@ private static void assertHistogram(long[] histogram, String expectedJson) { } public void testHistogram() { - final var histogram = new long[HandlingTimeTracker.BUCKET_COUNT]; + final var histogram = new long[ExponentialBucketHistogram.BUCKET_COUNT]; assertHistogram(histogram, """ {"h":[]}"""); @@ -102,11 +102,11 @@ public void testHistogram() { ]}"""); Arrays.fill(histogram, 0L); - histogram[HandlingTimeTracker.BUCKET_COUNT - 1] = 5; + histogram[ExponentialBucketHistogram.BUCKET_COUNT - 1] = 5; assertHistogram(histogram, """ {"h":[{"ge":"1m","ge_millis":65536,"count":5}]}"""); - histogram[HandlingTimeTracker.BUCKET_COUNT - 3] = 6; + histogram[ExponentialBucketHistogram.BUCKET_COUNT - 3] = 6; assertHistogram(histogram, """ {"h":[\ {"ge":"16.3s","ge_millis":16384,"lt":"32.7s","lt_millis":32768,"count":6},\ diff --git a/test/framework/src/main/java/org/elasticsearch/transport/TestTransportChannels.java b/test/framework/src/main/java/org/elasticsearch/transport/TestTransportChannels.java index effabd85591f9..97fd4348a2316 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/TestTransportChannels.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/TestTransportChannels.java @@ -10,7 +10,7 @@ package org.elasticsearch.transport; import org.elasticsearch.TransportVersion; -import org.elasticsearch.common.network.HandlingTimeTracker; +import org.elasticsearch.common.metrics.ExponentialBucketHistogram; import org.elasticsearch.common.util.PageCacheRecycler; import org.elasticsearch.threadpool.ThreadPool; @@ -26,7 +26,7 @@ public static TcpTransportChannel newFakeTcpTransportChannel( ) { BytesRefRecycler recycler = new BytesRefRecycler(PageCacheRecycler.NON_RECYCLING_INSTANCE); return new TcpTransportChannel( - new OutboundHandler(nodeName, version, new StatsTracker(), threadPool, recycler, new HandlingTimeTracker(), false), + new OutboundHandler(nodeName, version, new StatsTracker(), threadPool, recycler, new ExponentialBucketHistogram(), false), channel, action, requestId, From 3e3d9fc2d6eba21613a6dd1f00760464c104434a Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Wed, 26 Mar 2025 10:35:13 +1100 Subject: [PATCH 10/33] Tidy and fix document/test --- .../metrics/ExponentialBucketHistogram.java | 18 +++++++++--------- .../ExponentialBucketHistogramTests.java | 3 +-- 2 files changed, 10 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java b/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java index 5a13e3e796845..9b6c8f4e026f0 100644 --- a/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java +++ b/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java @@ -14,12 +14,12 @@ /** * A histogram with a fixed number of buckets of exponentially increasing width. - * - * The upper bounds of the buckets are defined by increasing powers of two, e.g. + *

+ * The bucket boundaries are defined by increasing powers of two, e.g. * - * 1, 2, 4, 8, 16... + * (-∞, 1), [1, 2), [2, 4), [4, 8), ..., [2^({@link #BUCKET_COUNT}-2), -∞) * - * There are {@link #BUCKET_COUNT} buckets + * There are {@link #BUCKET_COUNT} buckets. */ public class ExponentialBucketHistogram { @@ -31,13 +31,13 @@ public static int[] getBucketUpperBounds() { return bounds; } - private static int getBucket(long handlingTimeMillis) { - if (handlingTimeMillis <= 0) { + private static int getBucket(long observedValue) { + if (observedValue <= 0) { return 0; - } else if (LAST_BUCKET_LOWER_BOUND <= handlingTimeMillis) { + } else if (LAST_BUCKET_LOWER_BOUND <= observedValue) { return BUCKET_COUNT - 1; } else { - return Long.SIZE - Long.numberOfLeadingZeros(handlingTimeMillis); + return Long.SIZE - Long.numberOfLeadingZeros(observedValue); } } @@ -74,7 +74,7 @@ public long[] getHistogram() { * Calculate the Nth percentile value * * @param percentile The percentile as a fraction (in [0, 1.0]) - * @return A value greater than or equal to the specified fraction of values in the histogram + * @return A value greater than the specified fraction of values in the histogram */ public long getPercentile(float percentile) { assert percentile >= 0 && percentile <= 1; diff --git a/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java b/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java index 5c0beba341046..04bf9d69e6747 100644 --- a/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java +++ b/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java @@ -13,7 +13,6 @@ import org.elasticsearch.test.ESTestCase; import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.greaterThanOrEqualTo; public class ExponentialBucketHistogramTests extends ESTestCase { @@ -94,7 +93,7 @@ public void testPercentile() { final long actualPercentile = (long) Math.ceil(valueCount * percentile); final long histogramPercentile = histogram.getPercentile(percentile); final String message = Strings.format("%d percentile is %d (actual=%d)", i, histogramPercentile, actualPercentile); - assertThat(message, histogramPercentile, greaterThanOrEqualTo(actualPercentile)); + assertThat(message, histogramPercentile, greaterThan(actualPercentile)); } } } From be4b96c9faced01e269cc941d62fe136a3948013 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Wed, 26 Mar 2025 11:19:35 +1100 Subject: [PATCH 11/33] Restore field name --- .../java/org/elasticsearch/transport/InboundHandler.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/transport/InboundHandler.java b/server/src/main/java/org/elasticsearch/transport/InboundHandler.java index cf180b7a122b8..b07c817b02c98 100644 --- a/server/src/main/java/org/elasticsearch/transport/InboundHandler.java +++ b/server/src/main/java/org/elasticsearch/transport/InboundHandler.java @@ -46,7 +46,7 @@ public class InboundHandler { private final TransportKeepAlive keepAlive; private final Transport.ResponseHandlers responseHandlers; private final Transport.RequestHandlers requestHandlers; - private final ExponentialBucketHistogram exponentialBucketHistogram; + private final ExponentialBucketHistogram handlingTimeTracker; private final boolean ignoreDeserializationErrors; private volatile TransportMessageListener messageListener = TransportMessageListener.NOOP_LISTENER; @@ -61,7 +61,7 @@ public class InboundHandler { TransportKeepAlive keepAlive, Transport.RequestHandlers requestHandlers, Transport.ResponseHandlers responseHandlers, - ExponentialBucketHistogram exponentialBucketHistogram, + ExponentialBucketHistogram handlingTimeTracker, boolean ignoreDeserializationErrors ) { this.threadPool = threadPool; @@ -71,7 +71,7 @@ public class InboundHandler { this.keepAlive = keepAlive; this.requestHandlers = requestHandlers; this.responseHandlers = responseHandlers; - this.exponentialBucketHistogram = exponentialBucketHistogram; + this.handlingTimeTracker = handlingTimeTracker; this.ignoreDeserializationErrors = ignoreDeserializationErrors; } @@ -127,7 +127,7 @@ private void messageReceived(TcpChannel channel, InboundMessage message, long st } } finally { final long took = threadPool.rawRelativeTimeInMillis() - startTime; - exponentialBucketHistogram.addObservation(took); + handlingTimeTracker.addObservation(took); final long logThreshold = slowLogThresholdMs; if (logThreshold > 0 && took > logThreshold) { logSlowMessage(message, took, logThreshold, responseHandler); From 8a313ab2c1d4fa8b9e6d8b66a9d87d86d5406dd8 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Wed, 26 Mar 2025 11:30:31 +1100 Subject: [PATCH 12/33] Update docs/changelog/120488.yaml --- docs/changelog/120488.yaml | 5 +++++ 1 file changed, 5 insertions(+) create mode 100644 docs/changelog/120488.yaml diff --git a/docs/changelog/120488.yaml b/docs/changelog/120488.yaml new file mode 100644 index 0000000000000..4d05b9799f863 --- /dev/null +++ b/docs/changelog/120488.yaml @@ -0,0 +1,5 @@ +pr: 120488 +summary: Publish queue latency metrics from tracked thread pools +area: "Infra/Metrics, Allocation" +type: enhancement +issues: [] From 8fa7dc6590e08760cc515d4e352c9005d81aed43 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Wed, 26 Mar 2025 11:43:32 +1100 Subject: [PATCH 13/33] Fix changelog area value --- docs/changelog/120488.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/changelog/120488.yaml b/docs/changelog/120488.yaml index 4d05b9799f863..8d5b07ad21634 100644 --- a/docs/changelog/120488.yaml +++ b/docs/changelog/120488.yaml @@ -1,5 +1,5 @@ pr: 120488 summary: Publish queue latency metrics from tracked thread pools -area: "Infra/Metrics, Allocation" +area: "Infra/Metrics" type: enhancement issues: [] From 2c36b976436c61797d099950fd60d9a6246a6df6 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Thu, 27 Mar 2025 17:12:46 +1100 Subject: [PATCH 14/33] Update server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java Co-authored-by: Yang Wang --- .../common/metrics/ExponentialBucketHistogram.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java b/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java index 9b6c8f4e026f0..d14ed674a64ec 100644 --- a/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java +++ b/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java @@ -17,7 +17,7 @@ *

* The bucket boundaries are defined by increasing powers of two, e.g. * - * (-∞, 1), [1, 2), [2, 4), [4, 8), ..., [2^({@link #BUCKET_COUNT}-2), -∞) + * (-∞, 1), [1, 2), [2, 4), [4, 8), ..., [2^({@link #BUCKET_COUNT}-2), ∞) * * There are {@link #BUCKET_COUNT} buckets. */ From 420739e0bf125347d3dbdb88b43bec847b66ebfb Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Fri, 28 Mar 2025 11:21:06 +1100 Subject: [PATCH 15/33] Setup metrics separately to constructor --- .../netty4/Netty4ChunkedContinuationsIT.java | 4 +- .../threadpool/EvilThreadPoolTests.java | 7 +-- .../common/util/concurrent/EsExecutors.java | 48 ++++--------------- ...utionTimeTrackingEsThreadPoolExecutor.java | 29 +++++------ .../gateway/GatewayMetaState.java | 4 +- .../threadpool/FixedExecutorBuilder.java | 5 +- .../threadpool/ScalingExecutorBuilder.java | 5 +- .../elasticsearch/threadpool/ThreadPool.java | 5 ++ .../action/ActionRunnableTests.java | 4 +- .../search/QueryPhaseResultConsumerTests.java | 4 +- .../search/SearchPhaseControllerTests.java | 4 +- .../AbstractThrottledTaskRunnerTests.java | 11 +---- .../util/concurrent/EsExecutorsTests.java | 27 ++++------- .../concurrent/ListenableFutureTests.java | 7 +-- .../PrioritizedThrottledTaskRunnerTests.java | 11 +---- ...TimeTrackingEsThreadPoolExecutorTests.java | 22 ++++----- .../search/DefaultSearchContextTests.java | 10 ++-- .../SingleResultDeduplicatorTests.java | 4 +- .../lucene/store/ESIndexInputTestCase.java | 4 +- .../process/AbstractNativeProcessTests.java | 4 +- .../xpack/watcher/WatcherService.java | 4 +- 21 files changed, 65 insertions(+), 158 deletions(-) diff --git a/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4ChunkedContinuationsIT.java b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4ChunkedContinuationsIT.java index 4ab26a51987af..402730f063e5b 100644 --- a/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4ChunkedContinuationsIT.java +++ b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4ChunkedContinuationsIT.java @@ -71,7 +71,6 @@ import org.elasticsearch.rest.action.RestToXContentListener; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskCancelledException; -import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.MockLog; import org.elasticsearch.test.junit.annotations.TestLogging; @@ -163,8 +162,7 @@ public void testTraceLogging() { -1, EsExecutors.daemonThreadFactory(Settings.EMPTY, "test"), new ThreadContext(Settings.EMPTY), - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, - MeterRegistry.NOOP + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK ); resources.add(() -> assertTrue(ThreadPool.terminate(executor, 10, TimeUnit.SECONDS))); var loggingFinishedLatch = new CountDownLatch(1); diff --git a/qa/evil-tests/src/test/java/org/elasticsearch/threadpool/EvilThreadPoolTests.java b/qa/evil-tests/src/test/java/org/elasticsearch/threadpool/EvilThreadPoolTests.java index 6c1e4c62ab66c..7b3df344c2190 100644 --- a/qa/evil-tests/src/test/java/org/elasticsearch/threadpool/EvilThreadPoolTests.java +++ b/qa/evil-tests/src/test/java/org/elasticsearch/threadpool/EvilThreadPoolTests.java @@ -16,7 +16,6 @@ import org.elasticsearch.common.util.concurrent.EsThreadPoolExecutor; import org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor; import org.elasticsearch.core.TimeValue; -import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import org.junit.After; import org.junit.Before; @@ -68,8 +67,7 @@ public void testExecutionErrorOnFixedESThreadPoolExecutor() throws InterruptedEx 1, EsExecutors.daemonThreadFactory("test"), threadPool.getThreadContext(), - randomFrom(TaskTrackingConfig.DEFAULT, TaskTrackingConfig.DO_NOT_TRACK), - MeterRegistry.NOOP + randomFrom(TaskTrackingConfig.DEFAULT, TaskTrackingConfig.DO_NOT_TRACK) ); try { checkExecutionError(getExecuteRunner(fixedExecutor)); @@ -178,8 +176,7 @@ public void testExecutionExceptionOnFixedESThreadPoolExecutor() throws Interrupt 1, EsExecutors.daemonThreadFactory("test"), threadPool.getThreadContext(), - randomFrom(TaskTrackingConfig.DEFAULT, TaskTrackingConfig.DO_NOT_TRACK), - MeterRegistry.NOOP + randomFrom(TaskTrackingConfig.DEFAULT, TaskTrackingConfig.DO_NOT_TRACK) ); try { checkExecutionException(getExecuteRunner(fixedExecutor), true); diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java index 3681b8bf7f2c8..ddc5c42454b85 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java @@ -14,10 +14,8 @@ import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.Processors; -import org.elasticsearch.core.Nullable; import org.elasticsearch.core.SuppressForbidden; import org.elasticsearch.node.Node; -import org.elasticsearch.telemetry.metric.MeterRegistry; import java.security.AccessController; import java.security.PrivilegedAction; @@ -114,7 +112,7 @@ public static PrioritizedEsThreadPoolExecutor newSinglePrioritizing( * rejection handler. */ public static EsThreadPoolExecutor newScaling( - QualifiedName name, + String name, int min, int max, long keepAliveTime, @@ -122,8 +120,7 @@ public static EsThreadPoolExecutor newScaling( boolean rejectAfterShutdown, ThreadFactory threadFactory, ThreadContext contextHolder, - TaskTrackingConfig config, - MeterRegistry meterRegistry + TaskTrackingConfig config ) { LinkedTransferQueue queue = newUnboundedScalingLTQueue(min, max); // Force queued work via ForceQueuePolicy might starve if no worker is available (if core size is empty), @@ -141,12 +138,11 @@ public static EsThreadPoolExecutor newScaling( threadFactory, new ForceQueuePolicy(rejectAfterShutdown, probeWorkerPool), contextHolder, - config, - meterRegistry + config ); } else { return new EsThreadPoolExecutor( - name.toCompositeString(), + name, min, max, keepAliveTime, @@ -185,7 +181,7 @@ public static EsThreadPoolExecutor newScaling( ThreadContext contextHolder ) { return newScaling( - new QualifiedName(name), + name, min, max, keepAliveTime, @@ -193,8 +189,7 @@ public static EsThreadPoolExecutor newScaling( rejectAfterShutdown, threadFactory, contextHolder, - TaskTrackingConfig.DO_NOT_TRACK, - MeterRegistry.NOOP + TaskTrackingConfig.DO_NOT_TRACK ); } @@ -204,20 +199,7 @@ public static EsThreadPoolExecutor newFixed( int queueCapacity, ThreadFactory threadFactory, ThreadContext contextHolder, - TaskTrackingConfig config, - MeterRegistry meterRegistry - ) { - return newFixed(new QualifiedName(name), size, queueCapacity, threadFactory, contextHolder, config, meterRegistry); - } - - public static EsThreadPoolExecutor newFixed( - QualifiedName name, - int size, - int queueCapacity, - ThreadFactory threadFactory, - ThreadContext contextHolder, - TaskTrackingConfig config, - MeterRegistry meterRegistry + TaskTrackingConfig config ) { final BlockingQueue queue; final EsRejectedExecutionHandler rejectedExecutionHandler; @@ -240,12 +222,11 @@ public static EsThreadPoolExecutor newFixed( threadFactory, rejectedExecutionHandler, contextHolder, - config, - meterRegistry + config ); } else { return new EsThreadPoolExecutor( - name.toCompositeString(), + name, size, size, 0, @@ -632,15 +613,4 @@ public double getEwmaAlpha() { return ewmaAlpha; } } - - public record QualifiedName(@Nullable String nodeName, String threadPoolName) { - - public QualifiedName(String threadPoolName) { - this(null, threadPoolName); - } - - public String toCompositeString() { - return nodeName == null ? threadPoolName : nodeName + "/" + threadPoolName; - } - } } diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java index 3a99ae6b667f1..df914fefa9d9f 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java @@ -13,6 +13,7 @@ import org.elasticsearch.common.metrics.ExponentialBucketHistogram; import org.elasticsearch.common.util.concurrent.EsExecutors.TaskTrackingConfig; import org.elasticsearch.core.TimeValue; +import org.elasticsearch.telemetry.metric.Instrument; import org.elasticsearch.telemetry.metric.LongGauge; import org.elasticsearch.telemetry.metric.LongWithAttributes; import org.elasticsearch.telemetry.metric.MeterRegistry; @@ -44,10 +45,9 @@ public final class TaskExecutionTimeTrackingEsThreadPoolExecutor extends EsThrea // The set of currently running tasks and the timestamp of when they started execution in the Executor. private final Map ongoingTasks = new ConcurrentHashMap<>(); private final ExponentialBucketHistogram queueLatencyHistogram = new ExponentialBucketHistogram(); - private final LongGauge queueLatencyGauge; TaskExecutionTimeTrackingEsThreadPoolExecutor( - EsExecutors.QualifiedName name, + String name, int corePoolSize, int maximumPoolSize, long keepAliveTime, @@ -57,26 +57,18 @@ public final class TaskExecutionTimeTrackingEsThreadPoolExecutor extends EsThrea ThreadFactory threadFactory, RejectedExecutionHandler handler, ThreadContext contextHolder, - TaskTrackingConfig trackingConfig, - MeterRegistry meterRegistry + TaskTrackingConfig trackingConfig ) { - super( - name.toCompositeString(), - corePoolSize, - maximumPoolSize, - keepAliveTime, - unit, - workQueue, - threadFactory, - handler, - contextHolder - ); + super(name, corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory, handler, contextHolder); this.runnableWrapper = runnableWrapper; this.executionEWMA = new ExponentiallyWeightedMovingAverage(trackingConfig.getEwmaAlpha(), 0); this.trackOngoingTasks = trackingConfig.trackOngoingTasks(); - this.queueLatencyGauge = meterRegistry.registerLongsGauge( - ThreadPool.THREAD_POOL_METRIC_PREFIX + name.threadPoolName() + THREAD_POOL_METRIC_NAME_QUEUE_TIME, - "Time tasks spent in the queue for the " + name.threadPoolName() + " thread pool", + } + + public List setupMetrics(MeterRegistry meterRegistry, String threadPoolName) { + final LongGauge queueLatencyGauge = meterRegistry.registerLongsGauge( + ThreadPool.THREAD_POOL_METRIC_PREFIX + threadPoolName + THREAD_POOL_METRIC_NAME_QUEUE_TIME, + "Time tasks spent in the queue for the " + threadPoolName + " thread pool", "milliseconds", () -> { List metricValues = Arrays.stream(LATENCY_PERCENTILES_TO_REPORT) @@ -91,6 +83,7 @@ public final class TaskExecutionTimeTrackingEsThreadPoolExecutor extends EsThrea return metricValues; } ); + return List.of(queueLatencyGauge); } @Override diff --git a/server/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java b/server/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java index 9403705bc8ed8..56ce3c8ec9344 100644 --- a/server/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java +++ b/server/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java @@ -39,7 +39,6 @@ import org.elasticsearch.node.Node; import org.elasticsearch.plugins.ClusterCoordinationPlugin; import org.elasticsearch.plugins.MetadataUpgrader; -import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -411,8 +410,7 @@ static class AsyncPersistedState extends InMemoryPersistedState { 1, daemonThreadFactory(nodeName, THREAD_NAME), threadPool.getThreadContext(), - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, - MeterRegistry.NOOP + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK ); this.persistedState = persistedState; } diff --git a/server/src/main/java/org/elasticsearch/threadpool/FixedExecutorBuilder.java b/server/src/main/java/org/elasticsearch/threadpool/FixedExecutorBuilder.java index bd25ff3dbbb9d..dcac9bdbf03d5 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/FixedExecutorBuilder.java +++ b/server/src/main/java/org/elasticsearch/threadpool/FixedExecutorBuilder.java @@ -152,13 +152,12 @@ ThreadPool.ExecutorHolder build( int queueSize = settings.queueSize; final ThreadFactory threadFactory = EsExecutors.daemonThreadFactory(settings.nodeName, name(), isSystemThread()); final ExecutorService executor = EsExecutors.newFixed( - new EsExecutors.QualifiedName(settings.nodeName, name()), + settings.nodeName + "/" + name(), size, queueSize, threadFactory, threadContext, - taskTrackingConfig, - meterRegistry + taskTrackingConfig ); final ThreadPool.Info info = new ThreadPool.Info( name(), diff --git a/server/src/main/java/org/elasticsearch/threadpool/ScalingExecutorBuilder.java b/server/src/main/java/org/elasticsearch/threadpool/ScalingExecutorBuilder.java index 78c965122a280..0d0613fe49e5d 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/ScalingExecutorBuilder.java +++ b/server/src/main/java/org/elasticsearch/threadpool/ScalingExecutorBuilder.java @@ -144,7 +144,7 @@ ThreadPool.ExecutorHolder build( final ThreadFactory threadFactory = EsExecutors.daemonThreadFactory(settings.nodeName, name()); ExecutorService executor; executor = EsExecutors.newScaling( - new EsExecutors.QualifiedName(settings.nodeName, name()), + settings.nodeName + "/" + name(), core, max, keepAlive.millis(), @@ -152,8 +152,7 @@ ThreadPool.ExecutorHolder build( rejectAfterShutdown, threadFactory, threadContext, - trackingConfig, - meterRegistry + trackingConfig ); return new ThreadPool.ExecutorHolder(executor, info); } diff --git a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java index 013206628984e..341f1c4bc5d6e 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java +++ b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java @@ -24,6 +24,7 @@ import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionHandler; import org.elasticsearch.common.util.concurrent.EsThreadPoolExecutor; +import org.elasticsearch.common.util.concurrent.TaskExecutionTimeTrackingEsThreadPoolExecutor; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.util.concurrent.ThrottledTaskRunner; import org.elasticsearch.core.Nullable; @@ -374,6 +375,10 @@ private static ArrayList setupMetrics(MeterRegistry meterRegistry, S if (rejectedExecutionHandler instanceof EsRejectedExecutionHandler handler) { handler.registerCounter(meterRegistry, prefix + THREAD_POOL_METRIC_NAME_REJECTED, name); } + + if (threadPoolExecutor instanceof TaskExecutionTimeTrackingEsThreadPoolExecutor trackingThreadPoolExecutor) { + instruments.addAll(trackingThreadPoolExecutor.setupMetrics(meterRegistry, name)); + } } return instruments; } diff --git a/server/src/test/java/org/elasticsearch/action/ActionRunnableTests.java b/server/src/test/java/org/elasticsearch/action/ActionRunnableTests.java index 229317483305f..43150b3fbdb48 100644 --- a/server/src/test/java/org/elasticsearch/action/ActionRunnableTests.java +++ b/server/src/test/java/org/elasticsearch/action/ActionRunnableTests.java @@ -15,7 +15,6 @@ import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.common.util.concurrent.ThreadContext; -import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; import org.hamcrest.Matchers; @@ -80,8 +79,7 @@ public void testWrapReleasingRejected() throws Exception { 0, Thread::new, new ThreadContext(Settings.EMPTY), - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, - MeterRegistry.NOOP + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK ); try { final var listener = new PlainActionFuture(); diff --git a/server/src/test/java/org/elasticsearch/action/search/QueryPhaseResultConsumerTests.java b/server/src/test/java/org/elasticsearch/action/search/QueryPhaseResultConsumerTests.java index 93827e7f0574a..e0b68647289b2 100644 --- a/server/src/test/java/org/elasticsearch/action/search/QueryPhaseResultConsumerTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/QueryPhaseResultConsumerTests.java @@ -27,7 +27,6 @@ import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.query.QuerySearchResult; -import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; @@ -82,8 +81,7 @@ public AggregationReduceContext forFinalReduction() { 10, EsExecutors.daemonThreadFactory("test"), threadPool.getThreadContext(), - randomFrom(TaskTrackingConfig.DEFAULT, TaskTrackingConfig.DO_NOT_TRACK), - MeterRegistry.NOOP + randomFrom(TaskTrackingConfig.DEFAULT, TaskTrackingConfig.DO_NOT_TRACK) ); } diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java index fb849e9d0174b..0df14c9fd7ae0 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java @@ -67,7 +67,6 @@ import org.elasticsearch.search.suggest.phrase.PhraseSuggestion; import org.elasticsearch.search.suggest.term.TermSuggestion; import org.elasticsearch.tasks.TaskId; -import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; @@ -137,8 +136,7 @@ public AggregationReduceContext forFinalReduction() { 10, EsExecutors.daemonThreadFactory("test"), threadPool.getThreadContext(), - randomFrom(TaskTrackingConfig.DEFAULT, TaskTrackingConfig.DO_NOT_TRACK), - MeterRegistry.NOOP + randomFrom(TaskTrackingConfig.DEFAULT, TaskTrackingConfig.DO_NOT_TRACK) ); } diff --git a/server/src/test/java/org/elasticsearch/common/util/concurrent/AbstractThrottledTaskRunnerTests.java b/server/src/test/java/org/elasticsearch/common/util/concurrent/AbstractThrottledTaskRunnerTests.java index bbe21b20940cc..cb5adcfc3d775 100644 --- a/server/src/test/java/org/elasticsearch/common/util/concurrent/AbstractThrottledTaskRunnerTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/concurrent/AbstractThrottledTaskRunnerTests.java @@ -13,7 +13,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.EsExecutors.TaskTrackingConfig; import org.elasticsearch.core.Releasable; -import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import java.util.concurrent.BlockingQueue; @@ -196,15 +195,7 @@ public void onResponse(Releasable releasable) { public void testFailsTasksOnRejectionOrShutdown() throws Exception { final var executor = randomBoolean() ? EsExecutors.newScaling("test", maxThreads, maxThreads, 0, TimeUnit.MILLISECONDS, true, threadFactory, threadContext) - : EsExecutors.newFixed( - "test", - maxThreads, - between(1, 5), - threadFactory, - threadContext, - TaskTrackingConfig.DO_NOT_TRACK, - MeterRegistry.NOOP - ); + : EsExecutors.newFixed("test", maxThreads, between(1, 5), threadFactory, threadContext, TaskTrackingConfig.DO_NOT_TRACK); final var totalPermits = between(1, maxThreads * 2); final var permits = new Semaphore(totalPermits); diff --git a/server/src/test/java/org/elasticsearch/common/util/concurrent/EsExecutorsTests.java b/server/src/test/java/org/elasticsearch/common/util/concurrent/EsExecutorsTests.java index 2a6b24ded5527..104a2580e2907 100644 --- a/server/src/test/java/org/elasticsearch/common/util/concurrent/EsExecutorsTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/concurrent/EsExecutorsTests.java @@ -16,7 +16,6 @@ import org.elasticsearch.common.unit.Processors; import org.elasticsearch.core.TimeValue; import org.elasticsearch.node.Node; -import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; import org.hamcrest.Matcher; @@ -65,8 +64,7 @@ public void testFixedForcedExecution() throws Exception { 1, EsExecutors.daemonThreadFactory("test"), threadContext, - randomFrom(DEFAULT, DO_NOT_TRACK), - MeterRegistry.NOOP + randomFrom(DEFAULT, DO_NOT_TRACK) ); final CountDownLatch wait = new CountDownLatch(1); @@ -135,8 +133,7 @@ public void testFixedRejected() throws Exception { 1, EsExecutors.daemonThreadFactory("test"), threadContext, - randomFrom(DEFAULT, DO_NOT_TRACK), - MeterRegistry.NOOP + randomFrom(DEFAULT, DO_NOT_TRACK) ); final CountDownLatch wait = new CountDownLatch(1); @@ -281,8 +278,7 @@ public void testRejectionMessageAndShuttingDownFlag() throws InterruptedExceptio queue, EsExecutors.daemonThreadFactory("dummy"), threadContext, - randomFrom(DEFAULT, DO_NOT_TRACK), - MeterRegistry.NOOP + randomFrom(DEFAULT, DO_NOT_TRACK) ); try { for (int i = 0; i < actions; i++) { @@ -388,8 +384,7 @@ public void testInheritContext() throws InterruptedException { queue, EsExecutors.daemonThreadFactory("dummy"), threadContext, - randomFrom(DEFAULT, DO_NOT_TRACK), - MeterRegistry.NOOP + randomFrom(DEFAULT, DO_NOT_TRACK) ); try { executor.execute(() -> { @@ -426,8 +421,7 @@ public void testGetTasks() throws InterruptedException { queue, EsExecutors.daemonThreadFactory("dummy"), threadContext, - randomFrom(DEFAULT, DO_NOT_TRACK), - MeterRegistry.NOOP + randomFrom(DEFAULT, DO_NOT_TRACK) ); try { Runnable r = () -> { @@ -620,8 +614,7 @@ public void testFixedBoundedRejectOnShutdown() { between(1, 5), EsExecutors.daemonThreadFactory(getName()), threadContext, - randomFrom(DEFAULT, DO_NOT_TRACK), - MeterRegistry.NOOP + randomFrom(DEFAULT, DO_NOT_TRACK) ) ); } @@ -634,8 +627,7 @@ public void testFixedUnboundedRejectOnShutdown() { -1, EsExecutors.daemonThreadFactory(getName()), threadContext, - randomFrom(DEFAULT, DO_NOT_TRACK), - MeterRegistry.NOOP + randomFrom(DEFAULT, DO_NOT_TRACK) ) ); } @@ -678,7 +670,7 @@ public void testScalingWithTaskTimeTracking() { { ThreadPoolExecutor pool = EsExecutors.newScaling( - new EsExecutors.QualifiedName(getClass().getName(), getTestName()), + getClass().getName() + "/" + getTestName(), min, max, between(1, 100), @@ -686,8 +678,7 @@ public void testScalingWithTaskTimeTracking() { randomBoolean(), EsExecutors.daemonThreadFactory("test"), threadContext, - new EsExecutors.TaskTrackingConfig(randomBoolean(), randomDoubleBetween(0.01, 0.1, true)), - MeterRegistry.NOOP + new EsExecutors.TaskTrackingConfig(randomBoolean(), randomDoubleBetween(0.01, 0.1, true)) ); assertThat(pool, instanceOf(TaskExecutionTimeTrackingEsThreadPoolExecutor.class)); } diff --git a/server/src/test/java/org/elasticsearch/common/util/concurrent/ListenableFutureTests.java b/server/src/test/java/org/elasticsearch/common/util/concurrent/ListenableFutureTests.java index 39f61324f3a6d..1e5a51e12421e 100644 --- a/server/src/test/java/org/elasticsearch/common/util/concurrent/ListenableFutureTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/concurrent/ListenableFutureTests.java @@ -13,7 +13,6 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ReachabilityChecker; import org.elasticsearch.transport.RemoteTransportException; @@ -85,8 +84,7 @@ public void testConcurrentListenerRegistrationAndCompletion() throws Interrupted 1000, EsExecutors.daemonThreadFactory("listener"), threadContext, - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, - MeterRegistry.NOOP + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK ); final CyclicBarrier barrier = new CyclicBarrier(1 + numberOfThreads); final CountDownLatch listenersLatch = new CountDownLatch(numberOfThreads - 1); @@ -161,8 +159,7 @@ public void testRejection() { 1, EsExecutors.daemonThreadFactory("testRejection"), threadContext, - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, - MeterRegistry.NOOP + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK ); try { diff --git a/server/src/test/java/org/elasticsearch/common/util/concurrent/PrioritizedThrottledTaskRunnerTests.java b/server/src/test/java/org/elasticsearch/common/util/concurrent/PrioritizedThrottledTaskRunnerTests.java index a5c3f9a2c9415..7fc3e4ebb7f52 100644 --- a/server/src/test/java/org/elasticsearch/common/util/concurrent/PrioritizedThrottledTaskRunnerTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/concurrent/PrioritizedThrottledTaskRunnerTests.java @@ -11,7 +11,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.EsExecutors.TaskTrackingConfig; -import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; @@ -193,15 +192,7 @@ public void testEnqueueSpawnsNewTasksUpToMax() throws Exception { public void testFailsTasksOnRejectionOrShutdown() throws Exception { final var executor = randomBoolean() ? EsExecutors.newScaling("test", maxThreads, maxThreads, 0, TimeUnit.MILLISECONDS, true, threadFactory, threadContext) - : EsExecutors.newFixed( - "test", - maxThreads, - between(1, 5), - threadFactory, - threadContext, - TaskTrackingConfig.DO_NOT_TRACK, - MeterRegistry.NOOP - ); + : EsExecutors.newFixed("test", maxThreads, between(1, 5), threadFactory, threadContext, TaskTrackingConfig.DO_NOT_TRACK); final var taskRunner = new PrioritizedThrottledTaskRunner("test", between(1, maxThreads * 2), executor); final var totalPermits = between(1, maxThreads * 2); final var permits = new Semaphore(totalPermits); diff --git a/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java b/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java index b830a308142fb..c2c85d80f4f2f 100644 --- a/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java @@ -15,7 +15,6 @@ import org.elasticsearch.telemetry.InstrumentType; import org.elasticsearch.telemetry.Measurement; import org.elasticsearch.telemetry.RecordingMeterRegistry; -import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; @@ -42,7 +41,7 @@ public void testExecutionEWMACalculation() throws Exception { ThreadContext context = new ThreadContext(Settings.EMPTY); TaskExecutionTimeTrackingEsThreadPoolExecutor executor = new TaskExecutionTimeTrackingEsThreadPoolExecutor( - new EsExecutors.QualifiedName("test-threadpool"), + "test-threadpool", 1, 1, 1000, @@ -52,8 +51,7 @@ public void testExecutionEWMACalculation() throws Exception { EsExecutors.daemonThreadFactory("queuetest"), new EsAbortPolicy(), context, - new TaskTrackingConfig(randomBoolean(), DEFAULT_EWMA_ALPHA), - MeterRegistry.NOOP + new TaskTrackingConfig(randomBoolean(), DEFAULT_EWMA_ALPHA) ); executor.prestartAllCoreThreads(); logger.info("--> executor: {}", executor); @@ -95,7 +93,7 @@ public void testExecutionEWMACalculation() throws Exception { public void testExceptionThrowingTask() throws Exception { ThreadContext context = new ThreadContext(Settings.EMPTY); TaskExecutionTimeTrackingEsThreadPoolExecutor executor = new TaskExecutionTimeTrackingEsThreadPoolExecutor( - new EsExecutors.QualifiedName("test-threadpool"), + "test-threadpool", 1, 1, 1000, @@ -105,8 +103,7 @@ public void testExceptionThrowingTask() throws Exception { EsExecutors.daemonThreadFactory("queuetest"), new EsAbortPolicy(), context, - new TaskTrackingConfig(randomBoolean(), DEFAULT_EWMA_ALPHA), - MeterRegistry.NOOP + new TaskTrackingConfig(randomBoolean(), DEFAULT_EWMA_ALPHA) ); executor.prestartAllCoreThreads(); logger.info("--> executor: {}", executor); @@ -128,7 +125,7 @@ public void testGetOngoingTasks() throws Exception { var testStartTimeNanos = System.nanoTime(); ThreadContext context = new ThreadContext(Settings.EMPTY); var executor = new TaskExecutionTimeTrackingEsThreadPoolExecutor( - new EsExecutors.QualifiedName("test-threadpool"), + "test-threadpool", 1, 1, 1000, @@ -138,8 +135,7 @@ public void testGetOngoingTasks() throws Exception { EsExecutors.daemonThreadFactory("queuetest"), new EsAbortPolicy(), context, - new TaskTrackingConfig(true, DEFAULT_EWMA_ALPHA), - MeterRegistry.NOOP + new TaskTrackingConfig(true, DEFAULT_EWMA_ALPHA) ); var taskRunningLatch = new CountDownLatch(1); var exitTaskLatch = new CountDownLatch(1); @@ -164,7 +160,7 @@ public void testQueueLatencyMetrics() { RecordingMeterRegistry meterRegistry = new RecordingMeterRegistry(); final var threadPoolName = randomIdentifier(); var executor = new TaskExecutionTimeTrackingEsThreadPoolExecutor( - new EsExecutors.QualifiedName(threadPoolName), + threadPoolName, 1, 1, 1000, @@ -174,9 +170,9 @@ public void testQueueLatencyMetrics() { EsExecutors.daemonThreadFactory("queuetest"), new EsAbortPolicy(), new ThreadContext(Settings.EMPTY), - new TaskTrackingConfig(true, DEFAULT_EWMA_ALPHA), - meterRegistry + new TaskTrackingConfig(true, DEFAULT_EWMA_ALPHA) ); + executor.setupMetrics(meterRegistry, threadPoolName); try { final var barrier = new CyclicBarrier(2); diff --git a/server/src/test/java/org/elasticsearch/search/DefaultSearchContextTests.java b/server/src/test/java/org/elasticsearch/search/DefaultSearchContextTests.java index 9db5830634e42..335815fcea445 100644 --- a/server/src/test/java/org/elasticsearch/search/DefaultSearchContextTests.java +++ b/server/src/test/java/org/elasticsearch/search/DefaultSearchContextTests.java @@ -77,7 +77,6 @@ import org.elasticsearch.search.sort.FieldSortBuilder; import org.elasticsearch.search.sort.SortAndFormats; import org.elasticsearch.search.sort.SortBuilders; -import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xcontent.XContentBuilder; @@ -599,8 +598,7 @@ public void testDetermineMaximumNumberOfSlicesEnableQueryPhaseParallelCollection 0, Thread::new, new ThreadContext(Settings.EMPTY), - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, - MeterRegistry.NOOP + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK ); ToLongFunction fieldCardinality = name -> -1; assertEquals( @@ -658,8 +656,7 @@ public void testDetermineMaximumNumberOfSlicesSingleSortByField() { 0, Thread::new, new ThreadContext(Settings.EMPTY), - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, - MeterRegistry.NOOP + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK ); // DFS concurrency does not rely on slices, hence it kicks in regardless of the request (supportsParallelCollection is not called) assertEquals( @@ -692,8 +689,7 @@ public void testDetermineMaximumNumberOfSlicesWithQueue() { 1000, Thread::new, new ThreadContext(Settings.EMPTY), - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, - MeterRegistry.NOOP + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK ); ToLongFunction fieldCardinality = name -> { throw new UnsupportedOperationException(); }; diff --git a/server/src/test/java/org/elasticsearch/transport/SingleResultDeduplicatorTests.java b/server/src/test/java/org/elasticsearch/transport/SingleResultDeduplicatorTests.java index 1fdcc8e6d63b6..ed30da78ef725 100644 --- a/server/src/test/java/org/elasticsearch/transport/SingleResultDeduplicatorTests.java +++ b/server/src/test/java/org/elasticsearch/transport/SingleResultDeduplicatorTests.java @@ -21,7 +21,6 @@ import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; -import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; @@ -113,8 +112,7 @@ public void testThreadContextPreservation() { 0, EsExecutors.daemonThreadFactory("test"), threadContext, - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, - MeterRegistry.NOOP + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK ); resources[0] = () -> ThreadPool.terminate(executor, 10, TimeUnit.SECONDS); final var barrier = new CyclicBarrier(threads); diff --git a/test/framework/src/main/java/org/elasticsearch/common/lucene/store/ESIndexInputTestCase.java b/test/framework/src/main/java/org/elasticsearch/common/lucene/store/ESIndexInputTestCase.java index 8cdc09c52fc66..7738267ef0782 100644 --- a/test/framework/src/main/java/org/elasticsearch/common/lucene/store/ESIndexInputTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/common/lucene/store/ESIndexInputTestCase.java @@ -18,7 +18,6 @@ import org.elasticsearch.common.util.concurrent.EsThreadPoolExecutor; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.store.LuceneFilesExtensions; -import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -47,8 +46,7 @@ public static void createExecutor() { 0, EsExecutors.daemonThreadFactory(name), new ThreadContext(Settings.EMPTY), - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, - MeterRegistry.NOOP + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK ); } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/process/AbstractNativeProcessTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/process/AbstractNativeProcessTests.java index a02429c4a7d83..b128df2e91b27 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/process/AbstractNativeProcessTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/process/AbstractNativeProcessTests.java @@ -9,7 +9,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.ThreadContext; -import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.ml.process.logging.CppLogMessageHandler; @@ -80,8 +79,7 @@ public void initialize() throws IOException { 1, EsExecutors.daemonThreadFactory("test"), new ThreadContext(Settings.EMPTY), - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, - MeterRegistry.NOOP + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK ); } diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/WatcherService.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/WatcherService.java index 9c6e0faad8f9e..0ea9b432d3b0f 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/WatcherService.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/WatcherService.java @@ -31,7 +31,6 @@ import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.sort.SortBuilders; -import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xcontent.XContentType; import org.elasticsearch.xpack.core.ClientHelper; @@ -124,8 +123,7 @@ public class WatcherService { 1000, daemonThreadFactory(settings, LIFECYCLE_THREADPOOL_NAME), client.threadPool().getThreadContext(), - EsExecutors.TaskTrackingConfig.DO_NOT_TRACK, - MeterRegistry.NOOP + EsExecutors.TaskTrackingConfig.DO_NOT_TRACK ) ); } From 3208423e2453bf6d798049e78dcdb227c513f6dd Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Fri, 28 Mar 2025 11:27:36 +1100 Subject: [PATCH 16/33] Remove unnecessary change --- .../elasticsearch/common/util/concurrent/EsExecutors.java | 1 + .../java/org/elasticsearch/threadpool/ExecutorBuilder.java | 4 +--- .../org/elasticsearch/threadpool/FixedExecutorBuilder.java | 7 +------ .../elasticsearch/threadpool/ScalingExecutorBuilder.java | 7 +------ .../main/java/org/elasticsearch/threadpool/ThreadPool.java | 2 +- .../common/util/concurrent/EsExecutorsTests.java | 3 ++- 6 files changed, 7 insertions(+), 17 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java index ddc5c42454b85..28849a825bf25 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java @@ -613,4 +613,5 @@ public double getEwmaAlpha() { return ewmaAlpha; } } + } diff --git a/server/src/main/java/org/elasticsearch/threadpool/ExecutorBuilder.java b/server/src/main/java/org/elasticsearch/threadpool/ExecutorBuilder.java index 42b8ccf06f6c6..c259feb1c978e 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/ExecutorBuilder.java +++ b/server/src/main/java/org/elasticsearch/threadpool/ExecutorBuilder.java @@ -13,7 +13,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.ThreadContext; -import org.elasticsearch.telemetry.metric.MeterRegistry; import java.util.List; @@ -71,10 +70,9 @@ protected static int applyHardSizeLimit(final Settings settings, final String na * * @param settings the executor settings * @param threadContext the current thread context - * @param meterRegistry the meter registry * @return a new executor built from the specified executor settings */ - abstract ThreadPool.ExecutorHolder build(U settings, ThreadContext threadContext, MeterRegistry meterRegistry); + abstract ThreadPool.ExecutorHolder build(U settings, ThreadContext threadContext); /** * Format the thread pool info object for this executor. diff --git a/server/src/main/java/org/elasticsearch/threadpool/FixedExecutorBuilder.java b/server/src/main/java/org/elasticsearch/threadpool/FixedExecutorBuilder.java index dcac9bdbf03d5..9c723f241f1d0 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/FixedExecutorBuilder.java +++ b/server/src/main/java/org/elasticsearch/threadpool/FixedExecutorBuilder.java @@ -16,7 +16,6 @@ import org.elasticsearch.common.util.concurrent.EsExecutors.TaskTrackingConfig; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.node.Node; -import org.elasticsearch.telemetry.metric.MeterRegistry; import java.util.Arrays; import java.util.List; @@ -143,11 +142,7 @@ FixedExecutorSettings getSettings(Settings settings) { } @Override - ThreadPool.ExecutorHolder build( - final FixedExecutorSettings settings, - final ThreadContext threadContext, - final MeterRegistry meterRegistry - ) { + ThreadPool.ExecutorHolder build(final FixedExecutorSettings settings, final ThreadContext threadContext) { int size = settings.size; int queueSize = settings.queueSize; final ThreadFactory threadFactory = EsExecutors.daemonThreadFactory(settings.nodeName, name(), isSystemThread()); diff --git a/server/src/main/java/org/elasticsearch/threadpool/ScalingExecutorBuilder.java b/server/src/main/java/org/elasticsearch/threadpool/ScalingExecutorBuilder.java index 0d0613fe49e5d..0fb2f1e471d0b 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/ScalingExecutorBuilder.java +++ b/server/src/main/java/org/elasticsearch/threadpool/ScalingExecutorBuilder.java @@ -15,7 +15,6 @@ import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.core.TimeValue; import org.elasticsearch.node.Node; -import org.elasticsearch.telemetry.metric.MeterRegistry; import java.util.Arrays; import java.util.List; @@ -132,11 +131,7 @@ ScalingExecutorSettings getSettings(Settings settings) { return new ScalingExecutorSettings(nodeName, coreThreads, maxThreads, keepAlive); } - ThreadPool.ExecutorHolder build( - final ScalingExecutorSettings settings, - final ThreadContext threadContext, - final MeterRegistry meterRegistry - ) { + ThreadPool.ExecutorHolder build(final ScalingExecutorSettings settings, final ThreadContext threadContext) { TimeValue keepAlive = settings.keepAlive; int core = settings.core; int max = settings.max; diff --git a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java index 341f1c4bc5d6e..97503d3ba25e2 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java +++ b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java @@ -303,7 +303,7 @@ public ThreadPool( final Map executors = new HashMap<>(); for (final Map.Entry entry : builders.entrySet()) { final ExecutorBuilder.ExecutorSettings executorSettings = entry.getValue().getSettings(settings); - final ExecutorHolder executorHolder = entry.getValue().build(executorSettings, threadContext, meterRegistry); + final ExecutorHolder executorHolder = entry.getValue().build(executorSettings, threadContext); if (executors.containsKey(executorHolder.info.getName())) { throw new IllegalStateException("duplicate executors with name [" + executorHolder.info.getName() + "] registered"); } diff --git a/server/src/test/java/org/elasticsearch/common/util/concurrent/EsExecutorsTests.java b/server/src/test/java/org/elasticsearch/common/util/concurrent/EsExecutorsTests.java index 104a2580e2907..e87c0d00c15cd 100644 --- a/server/src/test/java/org/elasticsearch/common/util/concurrent/EsExecutorsTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/concurrent/EsExecutorsTests.java @@ -706,7 +706,8 @@ public void testScalingWithTaskTimeTracking() { randomTimeUnit(), randomBoolean(), EsExecutors.daemonThreadFactory("test"), - threadContext + threadContext, + DO_NOT_TRACK ); assertThat(pool, instanceOf(EsThreadPoolExecutor.class)); } From d6e44edbe67c86cb0813936cb6c0f39d6483458a Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Mon, 7 Apr 2025 09:44:02 +1000 Subject: [PATCH 17/33] Add 99th percentile --- .../TaskExecutionTimeTrackingEsThreadPoolExecutor.java | 2 +- .../TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java index df914fefa9d9f..6a8d11936ffd1 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java @@ -35,7 +35,7 @@ */ public final class TaskExecutionTimeTrackingEsThreadPoolExecutor extends EsThreadPoolExecutor { - private static final int[] LATENCY_PERCENTILES_TO_REPORT = { 50, 90 }; + private static final int[] LATENCY_PERCENTILES_TO_REPORT = { 50, 90, 99 }; public static final String THREAD_POOL_METRIC_NAME_QUEUE_TIME = ".threads.queue.latency.histogram"; private final Function runnableWrapper; diff --git a/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java b/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java index c2c85d80f4f2f..ce904a36e6077 100644 --- a/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java @@ -206,8 +206,9 @@ public void testQueueLatencyMetrics() { ThreadPool.THREAD_POOL_METRIC_PREFIX + threadPoolName + TaskExecutionTimeTrackingEsThreadPoolExecutor.THREAD_POOL_METRIC_NAME_QUEUE_TIME ); - assertThat(measurements, hasSize(2)); + assertThat(measurements, hasSize(3)); // we have to use greater than or equal to because the actual delay might be higher than what we imposed + assertThat(getPercentile(measurements, "99"), greaterThanOrEqualTo(expectedHistogram.getPercentile(0.99f))); assertThat(getPercentile(measurements, "90"), greaterThanOrEqualTo(expectedHistogram.getPercentile(0.9f))); assertThat(getPercentile(measurements, "50"), greaterThanOrEqualTo(expectedHistogram.getPercentile(0.5f))); } finally { From 2b294bd842cbba38c9fd93ffa02a64e9661fd1c5 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Mon, 7 Apr 2025 16:25:49 +1000 Subject: [PATCH 18/33] Record queue latency metrics in beforeExecute --- ...ecutionTimeTrackingEsThreadPoolExecutor.java | 14 +++++--------- .../common/util/concurrent/TimedRunnable.java | 17 +++++++++++++---- 2 files changed, 18 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java index 6a8d11936ffd1..375d5bd8c19ad 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java @@ -127,6 +127,11 @@ protected void beforeExecute(Thread t, Runnable r) { if (trackOngoingTasks) { ongoingTasks.put(r, System.nanoTime()); } + assert super.unwrap(r) instanceof TimedRunnable : "expected only TimedRunnables in queue"; + final TimedRunnable timedRunnable = (TimedRunnable) super.unwrap(r); + timedRunnable.beforeExecute(); + final long taskQueueLatency = timedRunnable.getQueueTimeNanos(); + queueLatencyHistogram.addObservation(TimeUnit.NANOSECONDS.toMillis(taskQueueLatency)); } @Override @@ -151,15 +156,6 @@ protected void afterExecute(Runnable r, Throwable t) { executionEWMA.addValue(taskExecutionNanos); totalExecutionTime.add(taskExecutionNanos); } - final long taskQueueLatency = timedRunnable.getQueueTimeNanos(); - assert taskQueueLatency >= 0 || (failedOrRejected && taskQueueLatency == -1) - : "queue latency should always be non-negative or `-1` to indicate rejection, got: " - + taskQueueLatency - + ", failedOrRejected: " - + failedOrRejected; - if (taskQueueLatency != -1) { - queueLatencyHistogram.addObservation(TimeUnit.NANOSECONDS.toMillis(taskQueueLatency)); - } } finally { // if trackOngoingTasks is false -> ongoingTasks must be empty assert trackOngoingTasks || ongoingTasks.isEmpty(); diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/TimedRunnable.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/TimedRunnable.java index 49557e7987f0b..de89ad0d8ea3f 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/TimedRunnable.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/TimedRunnable.java @@ -18,6 +18,7 @@ class TimedRunnable extends AbstractRunnable implements WrappedRunnable { private final Runnable original; private final long creationTimeNanos; + private long beforeExecuteTime = -1; private long startTimeNanos; private long finishTimeNanos = -1; private boolean failedOrRejected = false; @@ -59,15 +60,16 @@ public boolean isForceExecution() { } /** - * Returns the time in nanoseconds between the creation time and the start time + * Returns the time in nanoseconds between the creation time and the execution time * - * @return The time in nanoseconds or -1 if the task never started + * @return The time in nanoseconds or -1 if the task was never de-queued */ long getQueueTimeNanos() { - if (startTimeNanos == -1) { + if (beforeExecuteTime == -1) { + assert false : "beforeExecute must be called before getQueueTimeNanos"; return -1; } - return startTimeNanos - creationTimeNanos; + return beforeExecuteTime - creationTimeNanos; } /** @@ -82,6 +84,13 @@ long getTotalExecutionNanos() { return Math.max(finishTimeNanos - startTimeNanos, 1); } + /** + * Called when the task has reached the front of the queue and is about to be executed + */ + public void beforeExecute() { + beforeExecuteTime = System.nanoTime(); + } + /** * If the task was failed or rejected, return true. * Otherwise, false. From 9e227ed7fe81cb3ef4de26e5d24bcbf272c7b3e5 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Mon, 7 Apr 2025 17:10:31 +1000 Subject: [PATCH 19/33] Handle p100 percentile when last bucket is populated --- .../common/metrics/ExponentialBucketHistogram.java | 8 ++++++-- .../metrics/ExponentialBucketHistogramTests.java | 12 +++++++++++- 2 files changed, 17 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java b/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java index d14ed674a64ec..d3035cad6a9bb 100644 --- a/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java +++ b/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java @@ -79,12 +79,16 @@ public long[] getHistogram() { public long getPercentile(float percentile) { assert percentile >= 0 && percentile <= 1; final long[] snapshot = getHistogram(); - final long totalCount = Arrays.stream(snapshot).reduce(0L, Long::sum); + final long totalCount = Arrays.stream(snapshot).sum(); long percentileIndex = (long) Math.ceil(totalCount * percentile); for (int i = 0; i < BUCKET_COUNT; i++) { percentileIndex -= snapshot[i]; if (percentileIndex <= 0) { - return getBucketUpperBounds()[i]; + if (i == snapshot.length - 1) { + return Long.MAX_VALUE; + } else { + return getBucketUpperBounds()[i]; + } } } assert false : "We shouldn't ever get here"; diff --git a/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java b/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java index 04bf9d69e6747..3ce0f375650fb 100644 --- a/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java +++ b/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java @@ -12,7 +12,9 @@ import org.elasticsearch.core.Strings; import org.elasticsearch.test.ESTestCase; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; public class ExponentialBucketHistogramTests extends ESTestCase { @@ -93,7 +95,15 @@ public void testPercentile() { final long actualPercentile = (long) Math.ceil(valueCount * percentile); final long histogramPercentile = histogram.getPercentile(percentile); final String message = Strings.format("%d percentile is %d (actual=%d)", i, histogramPercentile, actualPercentile); - assertThat(message, histogramPercentile, greaterThan(actualPercentile)); + assertThat(message, histogramPercentile, greaterThanOrEqualTo(actualPercentile)); } } + + public void testMaxPercentile() { + ExponentialBucketHistogram histogram = new ExponentialBucketHistogram(); + int[] bucketUpperBounds = ExponentialBucketHistogram.getBucketUpperBounds(); + int secondToLastBucketUpperBound = bucketUpperBounds[bucketUpperBounds.length - 1]; + histogram.addObservation(secondToLastBucketUpperBound + 1); + assertThat(histogram.getPercentile(1.0f), equalTo(Long.MAX_VALUE)); + } } From 15acd803a9a2569b64962a31c83dd52f9dc2d21c Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Thu, 1 May 2025 15:17:09 +1000 Subject: [PATCH 20/33] Update server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java Co-authored-by: Dianna Hohensee --- .../elasticsearch/common/metrics/ExponentialBucketHistogram.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java b/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java index d3035cad6a9bb..9324676d01971 100644 --- a/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java +++ b/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java @@ -81,6 +81,7 @@ public long getPercentile(float percentile) { final long[] snapshot = getHistogram(); final long totalCount = Arrays.stream(snapshot).sum(); long percentileIndex = (long) Math.ceil(totalCount * percentile); + // Find which bucket has the Nth percentile value and return the upper bound value. for (int i = 0; i < BUCKET_COUNT; i++) { percentileIndex -= snapshot[i]; if (percentileIndex <= 0) { From 368a66af37bde3f853c70175ae3a978d1f126345 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Thu, 1 May 2025 15:18:05 +1000 Subject: [PATCH 21/33] Fix variable naming --- .../TaskExecutionTimeTrackingEsThreadPoolExecutor.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java index 375d5bd8c19ad..93f16e89fcb0f 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java @@ -44,7 +44,7 @@ public final class TaskExecutionTimeTrackingEsThreadPoolExecutor extends EsThrea private final boolean trackOngoingTasks; // The set of currently running tasks and the timestamp of when they started execution in the Executor. private final Map ongoingTasks = new ConcurrentHashMap<>(); - private final ExponentialBucketHistogram queueLatencyHistogram = new ExponentialBucketHistogram(); + private final ExponentialBucketHistogram queueLatencyMillisHistogram = new ExponentialBucketHistogram(); TaskExecutionTimeTrackingEsThreadPoolExecutor( String name, @@ -74,12 +74,12 @@ public List setupMetrics(MeterRegistry meterRegistry, String threadP List metricValues = Arrays.stream(LATENCY_PERCENTILES_TO_REPORT) .mapToObj( percentile -> new LongWithAttributes( - queueLatencyHistogram.getPercentile(percentile / 100f), + queueLatencyMillisHistogram.getPercentile(percentile / 100f), Map.of("percentile", String.valueOf(percentile)) ) ) .toList(); - queueLatencyHistogram.clear(); + queueLatencyMillisHistogram.clear(); return metricValues; } ); @@ -131,7 +131,7 @@ protected void beforeExecute(Thread t, Runnable r) { final TimedRunnable timedRunnable = (TimedRunnable) super.unwrap(r); timedRunnable.beforeExecute(); final long taskQueueLatency = timedRunnable.getQueueTimeNanos(); - queueLatencyHistogram.addObservation(TimeUnit.NANOSECONDS.toMillis(taskQueueLatency)); + queueLatencyMillisHistogram.addObservation(TimeUnit.NANOSECONDS.toMillis(taskQueueLatency)); } @Override From bb84ed9836b97042a7042f73652f9ed39ae50f8f Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Thu, 1 May 2025 15:29:21 +1000 Subject: [PATCH 22/33] Update variable names where we use ExponentialBucketHistogram --- .../elasticsearch/common/network/NetworkService.java | 6 +++--- .../http/AbstractHttpServerTransport.java | 2 +- .../org/elasticsearch/http/HttpRouteStatsTracker.java | 6 +++--- .../org/elasticsearch/transport/InboundHandler.java | 8 ++++---- .../org/elasticsearch/transport/OutboundHandler.java | 8 ++++---- .../java/org/elasticsearch/transport/TcpTransport.java | 10 +++++----- 6 files changed, 20 insertions(+), 20 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/network/NetworkService.java b/server/src/main/java/org/elasticsearch/common/network/NetworkService.java index f6a6b6f4e32e0..c30ea69b96d8a 100644 --- a/server/src/main/java/org/elasticsearch/common/network/NetworkService.java +++ b/server/src/main/java/org/elasticsearch/common/network/NetworkService.java @@ -86,15 +86,15 @@ public interface CustomNameResolver { } private final List customNameResolvers; - private final ExponentialBucketHistogram handlingTimeTracker = new ExponentialBucketHistogram(); + private final ExponentialBucketHistogram handlingTimeMillisHistogram = new ExponentialBucketHistogram(); private final ThreadWatchdog threadWatchdog = new ThreadWatchdog(); public NetworkService(List customNameResolvers) { this.customNameResolvers = Objects.requireNonNull(customNameResolvers, "customNameResolvers must be non null"); } - public ExponentialBucketHistogram getHandlingTimeTracker() { - return handlingTimeTracker; + public ExponentialBucketHistogram getHandlingTimeMillisHistogram() { + return handlingTimeMillisHistogram; } public ThreadWatchdog getThreadWatchdog() { diff --git a/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java b/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java index 1a66a9340468f..c5152229e115c 100644 --- a/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java +++ b/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java @@ -460,7 +460,7 @@ public void incomingRequest(final HttpRequest httpRequest, final HttpChannel htt handleIncomingRequest(httpRequest, trackingChannel, httpRequest.getInboundException()); } finally { final long took = threadPool.rawRelativeTimeInMillis() - startTime; - networkService.getHandlingTimeTracker().addObservation(took); + networkService.getHandlingTimeMillisHistogram().addObservation(took); final long logThreshold = slowLogThresholdMs; if (logThreshold > 0 && took > logThreshold) { logger.warn( diff --git a/server/src/main/java/org/elasticsearch/http/HttpRouteStatsTracker.java b/server/src/main/java/org/elasticsearch/http/HttpRouteStatsTracker.java index 79fc68a3a2ac1..83803e81f78a0 100644 --- a/server/src/main/java/org/elasticsearch/http/HttpRouteStatsTracker.java +++ b/server/src/main/java/org/elasticsearch/http/HttpRouteStatsTracker.java @@ -80,7 +80,7 @@ private static int bucket(int contentLength) { private final StatsTracker requestStats = new StatsTracker(); private final StatsTracker responseStats = new StatsTracker(); - private final ExponentialBucketHistogram responseTimeTracker = new ExponentialBucketHistogram(); + private final ExponentialBucketHistogram responseTimeMillisHistogram = new ExponentialBucketHistogram(); public void addRequestStats(int contentLength) { requestStats.addStats(contentLength); @@ -91,7 +91,7 @@ public void addResponseStats(long contentLength) { } public void addResponseTime(long timeMillis) { - responseTimeTracker.addObservation(timeMillis); + responseTimeMillisHistogram.addObservation(timeMillis); } public HttpRouteStats getStats() { @@ -102,7 +102,7 @@ public HttpRouteStats getStats() { responseStats.count().longValue(), responseStats.totalSize().longValue(), responseStats.getHistogram(), - responseTimeTracker.getHistogram() + responseTimeMillisHistogram.getHistogram() ); } } diff --git a/server/src/main/java/org/elasticsearch/transport/InboundHandler.java b/server/src/main/java/org/elasticsearch/transport/InboundHandler.java index b07c817b02c98..f43148d06f802 100644 --- a/server/src/main/java/org/elasticsearch/transport/InboundHandler.java +++ b/server/src/main/java/org/elasticsearch/transport/InboundHandler.java @@ -46,7 +46,7 @@ public class InboundHandler { private final TransportKeepAlive keepAlive; private final Transport.ResponseHandlers responseHandlers; private final Transport.RequestHandlers requestHandlers; - private final ExponentialBucketHistogram handlingTimeTracker; + private final ExponentialBucketHistogram handlingTimeMillisHistogram; private final boolean ignoreDeserializationErrors; private volatile TransportMessageListener messageListener = TransportMessageListener.NOOP_LISTENER; @@ -61,7 +61,7 @@ public class InboundHandler { TransportKeepAlive keepAlive, Transport.RequestHandlers requestHandlers, Transport.ResponseHandlers responseHandlers, - ExponentialBucketHistogram handlingTimeTracker, + ExponentialBucketHistogram handlingTimeMillisHistogram, boolean ignoreDeserializationErrors ) { this.threadPool = threadPool; @@ -71,7 +71,7 @@ public class InboundHandler { this.keepAlive = keepAlive; this.requestHandlers = requestHandlers; this.responseHandlers = responseHandlers; - this.handlingTimeTracker = handlingTimeTracker; + this.handlingTimeMillisHistogram = handlingTimeMillisHistogram; this.ignoreDeserializationErrors = ignoreDeserializationErrors; } @@ -127,7 +127,7 @@ private void messageReceived(TcpChannel channel, InboundMessage message, long st } } finally { final long took = threadPool.rawRelativeTimeInMillis() - startTime; - handlingTimeTracker.addObservation(took); + handlingTimeMillisHistogram.addObservation(took); final long logThreshold = slowLogThresholdMs; if (logThreshold > 0 && took > logThreshold) { logSlowMessage(message, took, logThreshold, responseHandler); diff --git a/server/src/main/java/org/elasticsearch/transport/OutboundHandler.java b/server/src/main/java/org/elasticsearch/transport/OutboundHandler.java index 22778b12ac615..db54a998f548d 100644 --- a/server/src/main/java/org/elasticsearch/transport/OutboundHandler.java +++ b/server/src/main/java/org/elasticsearch/transport/OutboundHandler.java @@ -56,7 +56,7 @@ public final class OutboundHandler { private final StatsTracker statsTracker; private final ThreadPool threadPool; private final Recycler recycler; - private final ExponentialBucketHistogram handlingTimeTracker; + private final ExponentialBucketHistogram handlingTimeMillisHistogram; private final boolean rstOnClose; private volatile long slowLogThresholdMs = Long.MAX_VALUE; @@ -69,7 +69,7 @@ public final class OutboundHandler { StatsTracker statsTracker, ThreadPool threadPool, Recycler recycler, - ExponentialBucketHistogram handlingTimeTracker, + ExponentialBucketHistogram handlingTimeMillisHistogram, boolean rstOnClose ) { this.nodeName = nodeName; @@ -77,7 +77,7 @@ public final class OutboundHandler { this.statsTracker = statsTracker; this.threadPool = threadPool; this.recycler = recycler; - this.handlingTimeTracker = handlingTimeTracker; + this.handlingTimeMillisHistogram = handlingTimeMillisHistogram; this.rstOnClose = rstOnClose; } @@ -414,7 +414,7 @@ private void maybeLogSlowMessage(boolean success) { final long logThreshold = slowLogThresholdMs; if (logThreshold > 0) { final long took = threadPool.rawRelativeTimeInMillis() - startTime; - handlingTimeTracker.addObservation(took); + handlingTimeMillisHistogram.addObservation(took); if (took > logThreshold) { logger.warn( "sending transport message [{}] of size [{}] on [{}] took [{}ms] which is above the warn " diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java index ff8766c981328..0621d4080fa1c 100644 --- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java +++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java @@ -128,7 +128,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements private final TransportHandshaker handshaker; private final TransportKeepAlive keepAlive; - private final ExponentialBucketHistogram outboundHandlingTimeTracker = new ExponentialBucketHistogram(); + private final ExponentialBucketHistogram outboundHandlingTimeMillisHistogram = new ExponentialBucketHistogram(); private final OutboundHandler outboundHandler; private final InboundHandler inboundHandler; private final ResponseHandlers responseHandlers = new ResponseHandlers(); @@ -162,7 +162,7 @@ public TcpTransport( statsTracker, threadPool, recycler, - outboundHandlingTimeTracker, + outboundHandlingTimeMillisHistogram, rstOnClose ); @@ -193,7 +193,7 @@ public TcpTransport( keepAlive, requestHandlers, responseHandlers, - networkService.getHandlingTimeTracker(), + networkService.getHandlingTimeMillisHistogram(), ignoreDeserializationErrors ); } @@ -996,8 +996,8 @@ public final TransportStats getStats() { bytesRead, messagesSent, bytesWritten, - networkService.getHandlingTimeTracker().getHistogram(), - outboundHandlingTimeTracker.getHistogram(), + networkService.getHandlingTimeMillisHistogram().getHistogram(), + outboundHandlingTimeMillisHistogram.getHistogram(), requestHandlers.getStats() ); } From 778e8b54f9c7e51d64baf500a20713f996560b96 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Thu, 1 May 2025 15:59:03 +1000 Subject: [PATCH 23/33] Update server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java Co-authored-by: Dianna Hohensee --- .../elasticsearch/common/metrics/ExponentialBucketHistogram.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java b/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java index 9324676d01971..de34f31c80d83 100644 --- a/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java +++ b/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java @@ -41,6 +41,7 @@ private static int getBucket(long observedValue) { } } + // We add an extra bucket to track values greater (up Long.MAX_VALUE) than the defined bucket bounds public static final int BUCKET_COUNT = getBucketUpperBounds().length + 1; private static final long LAST_BUCKET_LOWER_BOUND = getBucketUpperBounds()[BUCKET_COUNT - 2]; From 876ae78d2e82b6a9e9121f9f1cf7528fc13c748f Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Thu, 1 May 2025 16:03:26 +1000 Subject: [PATCH 24/33] Assert that taskQueueLatency >= 0 --- .../TaskExecutionTimeTrackingEsThreadPoolExecutor.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java index a096b19788d2a..4a2f523971a90 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java @@ -162,6 +162,7 @@ protected void beforeExecute(Thread t, Runnable r) { final TimedRunnable timedRunnable = (TimedRunnable) super.unwrap(r); timedRunnable.beforeExecute(); final long taskQueueLatency = timedRunnable.getQueueTimeNanos(); + assert taskQueueLatency >= 0; queueLatencyMillisHistogram.addObservation(TimeUnit.NANOSECONDS.toMillis(taskQueueLatency)); } From 51191ebcd35a9dd1efe6f21eaf70861f748f8e92 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Thu, 1 May 2025 16:28:30 +1000 Subject: [PATCH 25/33] Improve documentation of test --- ...TimeTrackingEsThreadPoolExecutorTests.java | 24 +++++++++++++++---- 1 file changed, 19 insertions(+), 5 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java b/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java index f1ccef9d03738..5c5fe03b6ce43 100644 --- a/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java @@ -177,12 +177,26 @@ public void testQueueLatencyMetrics() { try { final var barrier = new CyclicBarrier(2); final ExponentialBucketHistogram expectedHistogram = new ExponentialBucketHistogram(); + + /* + * The thread pool has a single thread, so we submit a task that will occupy that thread + * and cause subsequent tasks to be queued + */ Future runningTask = executor.submit(() -> { safeAwait(barrier); safeAwait(barrier); }); - safeAwait(barrier); // wait till first task starts - expectedHistogram.addObservation(0L); // first task should not be delayed + safeAwait(barrier); // wait till the first task starts + expectedHistogram.addObservation(0L); // the first task should not be delayed + + /* + * On each iteration we submit a task - which will be queued because of the + * currently running task, pause for some random interval, then unblock the + * new task by releasing the currently running task. This gives us a lower + * bound for the real delays (the real delays will be greater than or equal + * to the synthetic delays we add, i.e. each percentile should be >= our + * expected values) + */ for (int i = 0; i < 10; i++) { Future waitingTask = executor.submit(() -> { safeAwait(barrier); @@ -190,13 +204,13 @@ public void testQueueLatencyMetrics() { }); final long delayTimeMs = randomLongBetween(1, 50); safeSleep(delayTimeMs); - safeAwait(barrier); // let running task complete - safeAwait(barrier); // wait for next task to start + safeAwait(barrier); // let the running task complete + safeAwait(barrier); // wait for the next task to start safeGet(runningTask); // ensure previous task is complete expectedHistogram.addObservation(delayTimeMs); runningTask = waitingTask; } - safeAwait(barrier); // let last task finish + safeAwait(barrier); // let the last task finish safeGet(runningTask); meterRegistry.getRecorder().collect(); From f96674cddfda92aecef4c53b67c690add44e3529 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Thu, 1 May 2025 16:42:27 +1000 Subject: [PATCH 26/33] Remove queue latency metric changes --- docs/changelog/120488.yaml | 5 -- ...utionTimeTrackingEsThreadPoolExecutor.java | 50 +---------- .../common/util/concurrent/TimedRunnable.java | 21 ----- .../elasticsearch/threadpool/ThreadPool.java | 11 ++- ...TimeTrackingEsThreadPoolExecutorTests.java | 86 ------------------- 5 files changed, 10 insertions(+), 163 deletions(-) delete mode 100644 docs/changelog/120488.yaml diff --git a/docs/changelog/120488.yaml b/docs/changelog/120488.yaml deleted file mode 100644 index 8d5b07ad21634..0000000000000 --- a/docs/changelog/120488.yaml +++ /dev/null @@ -1,5 +0,0 @@ -pr: 120488 -summary: Publish queue latency metrics from tracked thread pools -area: "Infra/Metrics" -type: enhancement -issues: [] diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java index 4a2f523971a90..71f57bcc16754 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutor.java @@ -10,17 +10,9 @@ package org.elasticsearch.common.util.concurrent; import org.elasticsearch.common.ExponentiallyWeightedMovingAverage; -import org.elasticsearch.common.metrics.ExponentialBucketHistogram; import org.elasticsearch.common.util.concurrent.EsExecutors.TaskTrackingConfig; import org.elasticsearch.core.TimeValue; -import org.elasticsearch.telemetry.metric.DoubleWithAttributes; -import org.elasticsearch.telemetry.metric.Instrument; -import org.elasticsearch.telemetry.metric.LongWithAttributes; -import org.elasticsearch.telemetry.metric.MeterRegistry; -import org.elasticsearch.threadpool.ThreadPool; - -import java.util.Arrays; -import java.util.List; + import java.util.Map; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; @@ -30,16 +22,11 @@ import java.util.concurrent.atomic.LongAdder; import java.util.function.Function; -import static org.elasticsearch.threadpool.ThreadPool.THREAD_POOL_METRIC_NAME_QUEUE_TIME; -import static org.elasticsearch.threadpool.ThreadPool.THREAD_POOL_METRIC_NAME_UTILIZATION; - /** * An extension to thread pool executor, which tracks statistics for the task execution time. */ public final class TaskExecutionTimeTrackingEsThreadPoolExecutor extends EsThreadPoolExecutor { - private static final int[] LATENCY_PERCENTILES_TO_REPORT = { 50, 90, 99 }; - private final Function runnableWrapper; private final ExponentiallyWeightedMovingAverage executionEWMA; private final LongAdder totalExecutionTime = new LongAdder(); @@ -48,7 +35,6 @@ public final class TaskExecutionTimeTrackingEsThreadPoolExecutor extends EsThrea private final Map ongoingTasks = new ConcurrentHashMap<>(); private volatile long lastPollTime = System.nanoTime(); private volatile long lastTotalExecutionTime = 0; - private final ExponentialBucketHistogram queueLatencyMillisHistogram = new ExponentialBucketHistogram(); TaskExecutionTimeTrackingEsThreadPoolExecutor( String name, @@ -69,34 +55,6 @@ public final class TaskExecutionTimeTrackingEsThreadPoolExecutor extends EsThrea this.trackOngoingTasks = trackingConfig.trackOngoingTasks(); } - public List setupMetrics(MeterRegistry meterRegistry, String threadPoolName) { - return List.of( - meterRegistry.registerLongsGauge( - ThreadPool.THREAD_POOL_METRIC_PREFIX + threadPoolName + THREAD_POOL_METRIC_NAME_QUEUE_TIME, - "Time tasks spent in the queue for the " + threadPoolName + " thread pool", - "milliseconds", - () -> { - List metricValues = Arrays.stream(LATENCY_PERCENTILES_TO_REPORT) - .mapToObj( - percentile -> new LongWithAttributes( - queueLatencyMillisHistogram.getPercentile(percentile / 100f), - Map.of("percentile", String.valueOf(percentile)) - ) - ) - .toList(); - queueLatencyMillisHistogram.clear(); - return metricValues; - } - ), - meterRegistry.registerDoubleGauge( - ThreadPool.THREAD_POOL_METRIC_PREFIX + threadPoolName + THREAD_POOL_METRIC_NAME_UTILIZATION, - "fraction of maximum thread time utilized for " + threadPoolName, - "fraction", - () -> new DoubleWithAttributes(pollUtilization(), Map.of()) - ) - ); - } - @Override protected Runnable wrapRunnable(Runnable command) { return super.wrapRunnable(this.runnableWrapper.apply(command)); @@ -158,12 +116,6 @@ protected void beforeExecute(Thread t, Runnable r) { if (trackOngoingTasks) { ongoingTasks.put(r, System.nanoTime()); } - assert super.unwrap(r) instanceof TimedRunnable : "expected only TimedRunnables in queue"; - final TimedRunnable timedRunnable = (TimedRunnable) super.unwrap(r); - timedRunnable.beforeExecute(); - final long taskQueueLatency = timedRunnable.getQueueTimeNanos(); - assert taskQueueLatency >= 0; - queueLatencyMillisHistogram.addObservation(TimeUnit.NANOSECONDS.toMillis(taskQueueLatency)); } @Override diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/TimedRunnable.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/TimedRunnable.java index de89ad0d8ea3f..63fbee7999324 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/TimedRunnable.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/TimedRunnable.java @@ -18,7 +18,6 @@ class TimedRunnable extends AbstractRunnable implements WrappedRunnable { private final Runnable original; private final long creationTimeNanos; - private long beforeExecuteTime = -1; private long startTimeNanos; private long finishTimeNanos = -1; private boolean failedOrRejected = false; @@ -59,19 +58,6 @@ public boolean isForceExecution() { return original instanceof AbstractRunnable && ((AbstractRunnable) original).isForceExecution(); } - /** - * Returns the time in nanoseconds between the creation time and the execution time - * - * @return The time in nanoseconds or -1 if the task was never de-queued - */ - long getQueueTimeNanos() { - if (beforeExecuteTime == -1) { - assert false : "beforeExecute must be called before getQueueTimeNanos"; - return -1; - } - return beforeExecuteTime - creationTimeNanos; - } - /** * Return the time this task spent being run. * If the task is still running or has not yet been run, returns -1. @@ -84,13 +70,6 @@ long getTotalExecutionNanos() { return Math.max(finishTimeNanos - startTimeNanos, 1); } - /** - * Called when the task has reached the front of the queue and is about to be executed - */ - public void beforeExecute() { - beforeExecuteTime = System.nanoTime(); - } - /** * If the task was failed or rejected, return true. * Otherwise, false. diff --git a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java index fe1a40fa9a011..1533e616b8f28 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java +++ b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java @@ -31,6 +31,7 @@ import org.elasticsearch.core.TimeValue; import org.elasticsearch.node.Node; import org.elasticsearch.node.ReportingService; +import org.elasticsearch.telemetry.metric.DoubleWithAttributes; import org.elasticsearch.telemetry.metric.Instrument; import org.elasticsearch.telemetry.metric.LongAsyncCounter; import org.elasticsearch.telemetry.metric.LongGauge; @@ -153,7 +154,6 @@ public static class Names { public static final String THREAD_POOL_METRIC_NAME_UTILIZATION = ".threads.utilization.current"; public static final String THREAD_POOL_METRIC_NAME_LARGEST = ".threads.largest.current"; public static final String THREAD_POOL_METRIC_NAME_REJECTED = ".threads.rejected.total"; - public static final String THREAD_POOL_METRIC_NAME_QUEUE_TIME = ".threads.queue.latency.histogram"; public enum ThreadPoolType { FIXED("fixed"), @@ -379,7 +379,14 @@ private static ArrayList setupMetrics(MeterRegistry meterRegistry, S } if (threadPoolExecutor instanceof TaskExecutionTimeTrackingEsThreadPoolExecutor timeTrackingExecutor) { - instruments.addAll(timeTrackingExecutor.setupMetrics(meterRegistry, name)); + instruments.add( + meterRegistry.registerDoubleGauge( + prefix + THREAD_POOL_METRIC_NAME_UTILIZATION, + "fraction of maximum thread time utilized for " + name, + "fraction", + () -> new DoubleWithAttributes(timeTrackingExecutor.pollUtilization(), at) + ) + ); } } return instruments; diff --git a/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java b/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java index 5c5fe03b6ce43..b112ac0ceb6be 100644 --- a/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/concurrent/TaskExecutionTimeTrackingEsThreadPoolExecutorTests.java @@ -9,19 +9,11 @@ package org.elasticsearch.common.util.concurrent; -import org.elasticsearch.common.metrics.ExponentialBucketHistogram; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.EsExecutors.TaskTrackingConfig; -import org.elasticsearch.telemetry.InstrumentType; -import org.elasticsearch.telemetry.Measurement; -import org.elasticsearch.telemetry.RecordingMeterRegistry; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.threadpool.ThreadPool; -import java.util.List; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.CyclicBarrier; -import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.function.Function; @@ -29,7 +21,6 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; -import static org.hamcrest.Matchers.hasSize; /** * Tests for the automatic queue resizing of the {@code QueueResizingEsThreadPoolExecutorTests} @@ -156,83 +147,6 @@ public void testGetOngoingTasks() throws Exception { executor.awaitTermination(10, TimeUnit.SECONDS); } - public void testQueueLatencyMetrics() { - RecordingMeterRegistry meterRegistry = new RecordingMeterRegistry(); - final var threadPoolName = randomIdentifier(); - var executor = new TaskExecutionTimeTrackingEsThreadPoolExecutor( - threadPoolName, - 1, - 1, - 1000, - TimeUnit.MILLISECONDS, - ConcurrentCollections.newBlockingQueue(), - TimedRunnable::new, - EsExecutors.daemonThreadFactory("queuetest"), - new EsAbortPolicy(), - new ThreadContext(Settings.EMPTY), - new TaskTrackingConfig(true, DEFAULT_EWMA_ALPHA) - ); - executor.setupMetrics(meterRegistry, threadPoolName); - - try { - final var barrier = new CyclicBarrier(2); - final ExponentialBucketHistogram expectedHistogram = new ExponentialBucketHistogram(); - - /* - * The thread pool has a single thread, so we submit a task that will occupy that thread - * and cause subsequent tasks to be queued - */ - Future runningTask = executor.submit(() -> { - safeAwait(barrier); - safeAwait(barrier); - }); - safeAwait(barrier); // wait till the first task starts - expectedHistogram.addObservation(0L); // the first task should not be delayed - - /* - * On each iteration we submit a task - which will be queued because of the - * currently running task, pause for some random interval, then unblock the - * new task by releasing the currently running task. This gives us a lower - * bound for the real delays (the real delays will be greater than or equal - * to the synthetic delays we add, i.e. each percentile should be >= our - * expected values) - */ - for (int i = 0; i < 10; i++) { - Future waitingTask = executor.submit(() -> { - safeAwait(barrier); - safeAwait(barrier); - }); - final long delayTimeMs = randomLongBetween(1, 50); - safeSleep(delayTimeMs); - safeAwait(barrier); // let the running task complete - safeAwait(barrier); // wait for the next task to start - safeGet(runningTask); // ensure previous task is complete - expectedHistogram.addObservation(delayTimeMs); - runningTask = waitingTask; - } - safeAwait(barrier); // let the last task finish - safeGet(runningTask); - meterRegistry.getRecorder().collect(); - - List measurements = meterRegistry.getRecorder() - .getMeasurements( - InstrumentType.LONG_GAUGE, - ThreadPool.THREAD_POOL_METRIC_PREFIX + threadPoolName + ThreadPool.THREAD_POOL_METRIC_NAME_QUEUE_TIME - ); - assertThat(measurements, hasSize(3)); - // we have to use greater than or equal to because the actual delay might be higher than what we imposed - assertThat(getPercentile(measurements, "99"), greaterThanOrEqualTo(expectedHistogram.getPercentile(0.99f))); - assertThat(getPercentile(measurements, "90"), greaterThanOrEqualTo(expectedHistogram.getPercentile(0.9f))); - assertThat(getPercentile(measurements, "50"), greaterThanOrEqualTo(expectedHistogram.getPercentile(0.5f))); - } finally { - ThreadPool.terminate(executor, 10, TimeUnit.SECONDS); - } - } - - private long getPercentile(List measurements, String percentile) { - return measurements.stream().filter(m -> m.attributes().get("percentile").equals(percentile)).findFirst().orElseThrow().getLong(); - } - /** * The returned function outputs a WrappedRunnabled that simulates the case * where {@link TimedRunnable#getTotalExecutionNanos()} always returns {@code timeTakenNanos}. From 31f2083d541a3615af5367cc6c9599e2c1f21344 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Thu, 1 May 2025 16:47:55 +1000 Subject: [PATCH 27/33] getHistogram() -> getSnapshot() --- .../metrics/ExponentialBucketHistogram.java | 4 +-- .../http/HttpRouteStatsTracker.java | 2 +- .../elasticsearch/transport/TcpTransport.java | 4 +-- .../ExponentialBucketHistogramTests.java | 30 +++++++++---------- 4 files changed, 20 insertions(+), 20 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java b/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java index de34f31c80d83..ba4db5af02085 100644 --- a/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java +++ b/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java @@ -63,7 +63,7 @@ public void addObservation(long observedValue) { * @return An array of frequencies of handling times in buckets with upper bounds as returned by {@link #getBucketUpperBounds()}, plus * an extra bucket for handling times longer than the longest upper bound. */ - public long[] getHistogram() { + public long[] getSnapshot() { final long[] histogram = new long[BUCKET_COUNT]; for (int i = 0; i < BUCKET_COUNT; i++) { histogram[i] = buckets[i].longValue(); @@ -79,7 +79,7 @@ public long[] getHistogram() { */ public long getPercentile(float percentile) { assert percentile >= 0 && percentile <= 1; - final long[] snapshot = getHistogram(); + final long[] snapshot = getSnapshot(); final long totalCount = Arrays.stream(snapshot).sum(); long percentileIndex = (long) Math.ceil(totalCount * percentile); // Find which bucket has the Nth percentile value and return the upper bound value. diff --git a/server/src/main/java/org/elasticsearch/http/HttpRouteStatsTracker.java b/server/src/main/java/org/elasticsearch/http/HttpRouteStatsTracker.java index 83803e81f78a0..92c997e970d59 100644 --- a/server/src/main/java/org/elasticsearch/http/HttpRouteStatsTracker.java +++ b/server/src/main/java/org/elasticsearch/http/HttpRouteStatsTracker.java @@ -102,7 +102,7 @@ public HttpRouteStats getStats() { responseStats.count().longValue(), responseStats.totalSize().longValue(), responseStats.getHistogram(), - responseTimeMillisHistogram.getHistogram() + responseTimeMillisHistogram.getSnapshot() ); } } diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java index d189073ee8d1d..abb8921ff1809 100644 --- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java +++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java @@ -1001,8 +1001,8 @@ public final TransportStats getStats() { bytesRead, messagesSent, bytesWritten, - networkService.getHandlingTimeMillisHistogram().getHistogram(), - outboundHandlingTimeMillisHistogram.getHistogram(), + networkService.getHandlingTimeMillisHistogram().getSnapshot(), + outboundHandlingTimeMillisHistogram.getSnapshot(), requestHandlers.getStats() ); } diff --git a/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java b/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java index 3ce0f375650fb..06d6edbfe2e3c 100644 --- a/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java +++ b/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java @@ -18,46 +18,46 @@ public class ExponentialBucketHistogramTests extends ESTestCase { - public void testHistogram() { + public void testSnapshot() { final ExponentialBucketHistogram histogram = new ExponentialBucketHistogram(); - assertArrayEquals(new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getHistogram()); + assertArrayEquals(new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getSnapshot()); histogram.addObservation(0L); - assertArrayEquals(new long[] { 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getHistogram()); + assertArrayEquals(new long[] { 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getSnapshot()); histogram.addObservation(1L); - assertArrayEquals(new long[] { 1, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getHistogram()); + assertArrayEquals(new long[] { 1, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getSnapshot()); histogram.addObservation(2L); - assertArrayEquals(new long[] { 1, 1, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getHistogram()); + assertArrayEquals(new long[] { 1, 1, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getSnapshot()); histogram.addObservation(3L); - assertArrayEquals(new long[] { 1, 1, 2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getHistogram()); + assertArrayEquals(new long[] { 1, 1, 2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getSnapshot()); histogram.addObservation(4L); - assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getHistogram()); + assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getSnapshot()); histogram.addObservation(127L); - assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getHistogram()); + assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getSnapshot()); histogram.addObservation(128L); - assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getHistogram()); + assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getSnapshot()); histogram.addObservation(65535L); - assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 0 }, histogram.getHistogram()); + assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 0 }, histogram.getSnapshot()); histogram.addObservation(65536L); - assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 1 }, histogram.getHistogram()); + assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 1 }, histogram.getSnapshot()); histogram.addObservation(Long.MAX_VALUE); - assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 2 }, histogram.getHistogram()); + assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 2 }, histogram.getSnapshot()); histogram.addObservation(randomLongBetween(65536L, Long.MAX_VALUE)); - assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 3 }, histogram.getHistogram()); + assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 3 }, histogram.getSnapshot()); histogram.addObservation(randomLongBetween(Long.MIN_VALUE, 0L)); - assertArrayEquals(new long[] { 2, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 3 }, histogram.getHistogram()); + assertArrayEquals(new long[] { 2, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 3 }, histogram.getSnapshot()); } public void testHistogramRandom() { @@ -73,7 +73,7 @@ public void testHistogramRandom() { histogram.addObservation(between(lowerBound, upperBound)); } - assertArrayEquals(expectedCounts, histogram.getHistogram()); + assertArrayEquals(expectedCounts, histogram.getSnapshot()); } public void testBoundsConsistency() { From 4b737451d3275f6830fa69f8ce02ecd4e5f43971 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Fri, 2 May 2025 13:43:24 +1000 Subject: [PATCH 28/33] Overload getPercentile to allow re-use of a snapshot --- .../metrics/ExponentialBucketHistogram.java | 22 ++++++++++++++++--- .../ExponentialBucketHistogramTests.java | 4 +++- 2 files changed, 22 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java b/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java index ba4db5af02085..5957d3ed64c0a 100644 --- a/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java +++ b/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java @@ -76,10 +76,26 @@ public long[] getSnapshot() { * * @param percentile The percentile as a fraction (in [0, 1.0]) * @return A value greater than the specified fraction of values in the histogram + * @throws IllegalArgumentException if the requested percentile is invalid */ public long getPercentile(float percentile) { - assert percentile >= 0 && percentile <= 1; - final long[] snapshot = getSnapshot(); + return getPercentile(percentile, getSnapshot(), getBucketUpperBounds()); + } + + /** + * Calculate the Nth percentile value + * + * @param percentile The percentile as a fraction (in [0, 1.0]) + * @param snapshot An array of frequencies of handling times in buckets with upper bounds as returned by {@link #getBucketUpperBounds()} + * @param bucketUpperBounds The upper bounds of the buckets in the histogram, as returned by {@link #getBucketUpperBounds()} + * @return A value greater than the specified fraction of values in the histogram + * @throws IllegalArgumentException if the requested percentile is invalid + */ + public long getPercentile(float percentile, long[] snapshot, int[] bucketUpperBounds) { + assert snapshot.length == BUCKET_COUNT && bucketUpperBounds.length == BUCKET_COUNT - 1; + if (percentile < 0 || percentile > 1) { + throw new IllegalArgumentException("Requested percentile must be in [0, 1.0], percentile=" + percentile); + } final long totalCount = Arrays.stream(snapshot).sum(); long percentileIndex = (long) Math.ceil(totalCount * percentile); // Find which bucket has the Nth percentile value and return the upper bound value. @@ -89,7 +105,7 @@ public long getPercentile(float percentile) { if (i == snapshot.length - 1) { return Long.MAX_VALUE; } else { - return getBucketUpperBounds()[i]; + return bucketUpperBounds[i]; } } } diff --git a/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java b/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java index 06d6edbfe2e3c..b14aa26e51750 100644 --- a/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java +++ b/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java @@ -90,10 +90,12 @@ public void testPercentile() { for (int i = 0; i < valueCount; i++) { histogram.addObservation(i); } + final long[] snapshot = histogram.getSnapshot(); + final int[] bucketUpperBounds = ExponentialBucketHistogram.getBucketUpperBounds(); for (int i = 0; i <= 100; i++) { final float percentile = i / 100.0f; final long actualPercentile = (long) Math.ceil(valueCount * percentile); - final long histogramPercentile = histogram.getPercentile(percentile); + final long histogramPercentile = histogram.getPercentile(percentile, snapshot, bucketUpperBounds); final String message = Strings.format("%d percentile is %d (actual=%d)", i, histogramPercentile, actualPercentile); assertThat(message, histogramPercentile, greaterThanOrEqualTo(actualPercentile)); } From 221e079f808be455dc5b8d638531683a66f41bd8 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Mon, 5 May 2025 11:18:00 +1000 Subject: [PATCH 29/33] Additional tests --- .../ExponentialBucketHistogramTests.java | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java b/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java index b14aa26e51750..e1093f7ad3a9a 100644 --- a/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java +++ b/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java @@ -12,9 +12,12 @@ import org.elasticsearch.core.Strings; import org.elasticsearch.test.ESTestCase; +import java.util.Arrays; + import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.junit.Assert.assertThrows; public class ExponentialBucketHistogramTests extends ESTestCase { @@ -108,4 +111,29 @@ public void testMaxPercentile() { histogram.addObservation(secondToLastBucketUpperBound + 1); assertThat(histogram.getPercentile(1.0f), equalTo(Long.MAX_VALUE)); } + + public void testClear() { + ExponentialBucketHistogram histogram = new ExponentialBucketHistogram(); + for (int i = 0; i < 100; i++) { + histogram.addObservation(randomIntBetween(1, 100_000)); + } + assertThat(Arrays.stream(histogram.getSnapshot()).sum(), greaterThan(0L)); + histogram.clear(); + assertThat(Arrays.stream(histogram.getSnapshot()).sum(), equalTo(0L)); + } + + public void testPercentileValidation() { + ExponentialBucketHistogram histogram = new ExponentialBucketHistogram(); + // valid values + histogram.getPercentile(randomFloatBetween(0.0f, 1.0f, true)); + // invalid values + assertThrows( + IllegalArgumentException.class, + () -> histogram.getPercentile(randomFloatBetween(Float.NEGATIVE_INFINITY, 0.0f, false)) + ); + assertThrows( + IllegalArgumentException.class, + () -> histogram.getPercentile(randomFloatBetween(1.0f, Float.POSITIVE_INFINITY, false)) + ); + } } From 7e6a6ce6056c0618cf2bc0d4bd277f5e576c2fc1 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Mon, 5 May 2025 11:35:36 +1000 Subject: [PATCH 30/33] Additional tests --- .../common/metrics/ExponentialBucketHistogramTests.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java b/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java index e1093f7ad3a9a..a1950333ddc24 100644 --- a/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java +++ b/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java @@ -124,8 +124,12 @@ public void testClear() { public void testPercentileValidation() { ExponentialBucketHistogram histogram = new ExponentialBucketHistogram(); + for (int i = 0; i < 100; i++) { + histogram.addObservation(randomIntBetween(1, 100_000)); + } // valid values histogram.getPercentile(randomFloatBetween(0.0f, 1.0f, true)); + histogram.getPercentile(1.0f); // invalid values assertThrows( IllegalArgumentException.class, From 8ef9745cb996e0a3b731208dfbaeb9a3a5ae6fb2 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Mon, 5 May 2025 14:25:24 +1000 Subject: [PATCH 31/33] Minimise change --- .../metrics/ExponentialBucketHistogram.java | 54 ++++++------ .../common/network/HandlingTimeTracker.java | 25 ++++++ .../common/network/NetworkService.java | 7 +- .../http/AbstractHttpServerTransport.java | 2 +- .../elasticsearch/http/HttpRouteStats.java | 6 +- .../http/HttpRouteStatsTracker.java | 8 +- .../transport/InboundHandler.java | 10 +-- .../transport/OutboundHandler.java | 10 +-- .../elasticsearch/transport/TcpTransport.java | 12 +-- .../transport/TransportStats.java | 14 ++-- .../cluster/node/stats/NodeStatsTests.java | 6 +- .../ExponentialBucketHistogramTests.java | 26 +++--- .../network/HandlingTimeTrackerTests.java | 84 +++++++++++++++++++ .../http/HttpRouteStatsTests.java | 4 +- .../transport/InboundHandlerTests.java | 6 +- .../transport/OutboundHandlerTests.java | 4 +- .../transport/TcpTransportTests.java | 4 +- .../transport/TransportStatsTests.java | 10 +-- .../transport/TestTransportChannels.java | 4 +- 19 files changed, 207 insertions(+), 89 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/common/network/HandlingTimeTracker.java create mode 100644 server/src/test/java/org/elasticsearch/common/network/HandlingTimeTrackerTests.java diff --git a/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java b/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java index 5957d3ed64c0a..49c27d6de9f1f 100644 --- a/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java +++ b/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java @@ -17,55 +17,59 @@ *

* The bucket boundaries are defined by increasing powers of two, e.g. * - * (-∞, 1), [1, 2), [2, 4), [4, 8), ..., [2^({@link #BUCKET_COUNT}-2), ∞) + * (-∞, 1), [1, 2), [2, 4), [4, 8), ..., [2^({@link #bucketCount}-2), ∞) * - * There are {@link #BUCKET_COUNT} buckets. + * There are {@link #bucketCount} buckets. */ public class ExponentialBucketHistogram { - public static int[] getBucketUpperBounds() { - int[] bounds = new int[17]; + private final int bucketCount; + private final long lastBucketLowerBound; + + public static int[] getBucketUpperBounds(int bucketCount) { + int[] bounds = new int[bucketCount - 1]; for (int i = 0; i < bounds.length; i++) { bounds[i] = 1 << i; } return bounds; } - private static int getBucket(long observedValue) { + private int getBucket(long observedValue) { if (observedValue <= 0) { return 0; - } else if (LAST_BUCKET_LOWER_BOUND <= observedValue) { - return BUCKET_COUNT - 1; + } else if (lastBucketLowerBound <= observedValue) { + return bucketCount - 1; } else { return Long.SIZE - Long.numberOfLeadingZeros(observedValue); } } - // We add an extra bucket to track values greater (up Long.MAX_VALUE) than the defined bucket bounds - public static final int BUCKET_COUNT = getBucketUpperBounds().length + 1; - - private static final long LAST_BUCKET_LOWER_BOUND = getBucketUpperBounds()[BUCKET_COUNT - 2]; - private final LongAdder[] buckets; - public ExponentialBucketHistogram() { - buckets = new LongAdder[BUCKET_COUNT]; - for (int i = 0; i < BUCKET_COUNT; i++) { + public ExponentialBucketHistogram(int bucketCount) { + this.bucketCount = bucketCount; + this.lastBucketLowerBound = getBucketUpperBounds(bucketCount)[bucketCount - 2]; + buckets = new LongAdder[bucketCount]; + for (int i = 0; i < bucketCount; i++) { buckets[i] = new LongAdder(); } } + public int[] calculateBucketUpperBounds() { + return getBucketUpperBounds(bucketCount); + } + public void addObservation(long observedValue) { buckets[getBucket(observedValue)].increment(); } /** - * @return An array of frequencies of handling times in buckets with upper bounds as returned by {@link #getBucketUpperBounds()}, plus - * an extra bucket for handling times longer than the longest upper bound. + * @return An array of frequencies of handling times in buckets with upper bounds as returned by {@link #calculateBucketUpperBounds()}, + * plus an extra bucket for handling times longer than the longest upper bound. */ public long[] getSnapshot() { - final long[] histogram = new long[BUCKET_COUNT]; - for (int i = 0; i < BUCKET_COUNT; i++) { + final long[] histogram = new long[bucketCount]; + for (int i = 0; i < bucketCount; i++) { histogram[i] = buckets[i].longValue(); } return histogram; @@ -79,27 +83,27 @@ public long[] getSnapshot() { * @throws IllegalArgumentException if the requested percentile is invalid */ public long getPercentile(float percentile) { - return getPercentile(percentile, getSnapshot(), getBucketUpperBounds()); + return getPercentile(percentile, getSnapshot(), calculateBucketUpperBounds()); } /** * Calculate the Nth percentile value * * @param percentile The percentile as a fraction (in [0, 1.0]) - * @param snapshot An array of frequencies of handling times in buckets with upper bounds as returned by {@link #getBucketUpperBounds()} - * @param bucketUpperBounds The upper bounds of the buckets in the histogram, as returned by {@link #getBucketUpperBounds()} + * @param snapshot An array of frequencies of handling times in buckets with upper bounds as per {@link #calculateBucketUpperBounds()} + * @param bucketUpperBounds The upper bounds of the buckets in the histogram, as per {@link #calculateBucketUpperBounds()} * @return A value greater than the specified fraction of values in the histogram * @throws IllegalArgumentException if the requested percentile is invalid */ public long getPercentile(float percentile, long[] snapshot, int[] bucketUpperBounds) { - assert snapshot.length == BUCKET_COUNT && bucketUpperBounds.length == BUCKET_COUNT - 1; + assert snapshot.length == bucketCount && bucketUpperBounds.length == bucketCount - 1; if (percentile < 0 || percentile > 1) { throw new IllegalArgumentException("Requested percentile must be in [0, 1.0], percentile=" + percentile); } final long totalCount = Arrays.stream(snapshot).sum(); long percentileIndex = (long) Math.ceil(totalCount * percentile); // Find which bucket has the Nth percentile value and return the upper bound value. - for (int i = 0; i < BUCKET_COUNT; i++) { + for (int i = 0; i < bucketCount; i++) { percentileIndex -= snapshot[i]; if (percentileIndex <= 0) { if (i == snapshot.length - 1) { @@ -117,7 +121,7 @@ public long getPercentile(float percentile, long[] snapshot, int[] bucketUpperBo * Clear all values in the histogram (non-atomic) */ public void clear() { - for (int i = 0; i < BUCKET_COUNT; i++) { + for (int i = 0; i < bucketCount; i++) { buckets[i].reset(); } } diff --git a/server/src/main/java/org/elasticsearch/common/network/HandlingTimeTracker.java b/server/src/main/java/org/elasticsearch/common/network/HandlingTimeTracker.java new file mode 100644 index 0000000000000..0c7c81d3f3d16 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/common/network/HandlingTimeTracker.java @@ -0,0 +1,25 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.common.network; + +import org.elasticsearch.common.metrics.ExponentialBucketHistogram; + +public class HandlingTimeTracker extends ExponentialBucketHistogram { + + public static final int BUCKET_COUNT = 18; + + public static int[] getBucketUpperBounds() { + return ExponentialBucketHistogram.getBucketUpperBounds(BUCKET_COUNT); + } + + public HandlingTimeTracker() { + super(BUCKET_COUNT); + } +} diff --git a/server/src/main/java/org/elasticsearch/common/network/NetworkService.java b/server/src/main/java/org/elasticsearch/common/network/NetworkService.java index c30ea69b96d8a..70e4952261ac3 100644 --- a/server/src/main/java/org/elasticsearch/common/network/NetworkService.java +++ b/server/src/main/java/org/elasticsearch/common/network/NetworkService.java @@ -9,7 +9,6 @@ package org.elasticsearch.common.network; -import org.elasticsearch.common.metrics.ExponentialBucketHistogram; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.unit.ByteSizeValue; @@ -86,15 +85,15 @@ public interface CustomNameResolver { } private final List customNameResolvers; - private final ExponentialBucketHistogram handlingTimeMillisHistogram = new ExponentialBucketHistogram(); + private final HandlingTimeTracker handlingTimeTracker = new HandlingTimeTracker(); private final ThreadWatchdog threadWatchdog = new ThreadWatchdog(); public NetworkService(List customNameResolvers) { this.customNameResolvers = Objects.requireNonNull(customNameResolvers, "customNameResolvers must be non null"); } - public ExponentialBucketHistogram getHandlingTimeMillisHistogram() { - return handlingTimeMillisHistogram; + public HandlingTimeTracker getHandlingTimeTracker() { + return handlingTimeTracker; } public ThreadWatchdog getThreadWatchdog() { diff --git a/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java b/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java index c5152229e115c..1a66a9340468f 100644 --- a/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java +++ b/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java @@ -460,7 +460,7 @@ public void incomingRequest(final HttpRequest httpRequest, final HttpChannel htt handleIncomingRequest(httpRequest, trackingChannel, httpRequest.getInboundException()); } finally { final long took = threadPool.rawRelativeTimeInMillis() - startTime; - networkService.getHandlingTimeMillisHistogram().addObservation(took); + networkService.getHandlingTimeTracker().addObservation(took); final long logThreshold = slowLogThresholdMs; if (logThreshold > 0 && took > logThreshold) { logger.warn( diff --git a/server/src/main/java/org/elasticsearch/http/HttpRouteStats.java b/server/src/main/java/org/elasticsearch/http/HttpRouteStats.java index 667a35d79ea18..a15b929fd3c1b 100644 --- a/server/src/main/java/org/elasticsearch/http/HttpRouteStats.java +++ b/server/src/main/java/org/elasticsearch/http/HttpRouteStats.java @@ -12,7 +12,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.metrics.ExponentialBucketHistogram; +import org.elasticsearch.common.network.HandlingTimeTracker; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.core.TimeValue; import org.elasticsearch.xcontent.ToXContentObject; @@ -36,7 +36,7 @@ * @param totalResponseSize the total body size (bytes) of responses produced by the HTTP route * @param responseSizeHistogram similar to {@code requestSizeHistogram} but for response size * @param responseTimeHistogram an array of frequencies of response time (millis) in buckets with upper bounds - * as returned by {@link ExponentialBucketHistogram#getBucketUpperBounds()}, plus + * as returned by {@link HandlingTimeTracker#getBucketUpperBounds()}, plus * an extra bucket for handling response time larger than the longest upper bound (currently 65536ms). */ public record HttpRouteStats( @@ -89,7 +89,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws "millis", TimeValue::timeValueMillis, responseTimeHistogram, - ExponentialBucketHistogram.getBucketUpperBounds() + HandlingTimeTracker.getBucketUpperBounds() ); builder.endObject(); diff --git a/server/src/main/java/org/elasticsearch/http/HttpRouteStatsTracker.java b/server/src/main/java/org/elasticsearch/http/HttpRouteStatsTracker.java index 92c997e970d59..7ae067d3c6c15 100644 --- a/server/src/main/java/org/elasticsearch/http/HttpRouteStatsTracker.java +++ b/server/src/main/java/org/elasticsearch/http/HttpRouteStatsTracker.java @@ -9,7 +9,7 @@ package org.elasticsearch.http; -import org.elasticsearch.common.metrics.ExponentialBucketHistogram; +import org.elasticsearch.common.network.HandlingTimeTracker; import java.util.concurrent.atomic.AtomicLongArray; import java.util.concurrent.atomic.LongAdder; @@ -80,7 +80,7 @@ private static int bucket(int contentLength) { private final StatsTracker requestStats = new StatsTracker(); private final StatsTracker responseStats = new StatsTracker(); - private final ExponentialBucketHistogram responseTimeMillisHistogram = new ExponentialBucketHistogram(); + private final HandlingTimeTracker responseTimeTracker = new HandlingTimeTracker(); public void addRequestStats(int contentLength) { requestStats.addStats(contentLength); @@ -91,7 +91,7 @@ public void addResponseStats(long contentLength) { } public void addResponseTime(long timeMillis) { - responseTimeMillisHistogram.addObservation(timeMillis); + responseTimeTracker.addObservation(timeMillis); } public HttpRouteStats getStats() { @@ -102,7 +102,7 @@ public HttpRouteStats getStats() { responseStats.count().longValue(), responseStats.totalSize().longValue(), responseStats.getHistogram(), - responseTimeMillisHistogram.getSnapshot() + responseTimeTracker.getSnapshot() ); } } diff --git a/server/src/main/java/org/elasticsearch/transport/InboundHandler.java b/server/src/main/java/org/elasticsearch/transport/InboundHandler.java index a41b129d2e312..ee4e5e31c584a 100644 --- a/server/src/main/java/org/elasticsearch/transport/InboundHandler.java +++ b/server/src/main/java/org/elasticsearch/transport/InboundHandler.java @@ -18,7 +18,7 @@ import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.metrics.ExponentialBucketHistogram; +import org.elasticsearch.common.network.HandlingTimeTracker; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.ThreadContext; @@ -45,7 +45,7 @@ public class InboundHandler { private final TransportKeepAlive keepAlive; private final Transport.ResponseHandlers responseHandlers; private final Transport.RequestHandlers requestHandlers; - private final ExponentialBucketHistogram handlingTimeMillisHistogram; + private final HandlingTimeTracker handlingTimeTracker; private final boolean ignoreDeserializationErrors; private volatile TransportMessageListener messageListener = TransportMessageListener.NOOP_LISTENER; @@ -60,7 +60,7 @@ public class InboundHandler { TransportKeepAlive keepAlive, Transport.RequestHandlers requestHandlers, Transport.ResponseHandlers responseHandlers, - ExponentialBucketHistogram handlingTimeMillisHistogram, + HandlingTimeTracker handlingTimeTracker, boolean ignoreDeserializationErrors ) { this.threadPool = threadPool; @@ -70,7 +70,7 @@ public class InboundHandler { this.keepAlive = keepAlive; this.requestHandlers = requestHandlers; this.responseHandlers = responseHandlers; - this.handlingTimeMillisHistogram = handlingTimeMillisHistogram; + this.handlingTimeTracker = handlingTimeTracker; this.ignoreDeserializationErrors = ignoreDeserializationErrors; } @@ -138,7 +138,7 @@ private void messageReceived(TcpChannel channel, InboundMessage message, long st } } finally { final long took = threadPool.rawRelativeTimeInMillis() - startTime; - handlingTimeMillisHistogram.addObservation(took); + handlingTimeTracker.addObservation(took); final long logThreshold = slowLogThresholdMs; if (logThreshold > 0 && took > logThreshold) { logSlowMessage(message, took, logThreshold, responseHandler); diff --git a/server/src/main/java/org/elasticsearch/transport/OutboundHandler.java b/server/src/main/java/org/elasticsearch/transport/OutboundHandler.java index 38650e29bc152..e198b8bd19bcc 100644 --- a/server/src/main/java/org/elasticsearch/transport/OutboundHandler.java +++ b/server/src/main/java/org/elasticsearch/transport/OutboundHandler.java @@ -26,8 +26,8 @@ import org.elasticsearch.common.io.stream.RecyclerBytesStreamOutput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.metrics.ExponentialBucketHistogram; import org.elasticsearch.common.network.CloseableChannel; +import org.elasticsearch.common.network.HandlingTimeTracker; import org.elasticsearch.common.recycler.Recycler; import org.elasticsearch.common.transport.NetworkExceptionHelper; import org.elasticsearch.common.util.concurrent.ThreadContext; @@ -56,7 +56,7 @@ public final class OutboundHandler { private final StatsTracker statsTracker; private final ThreadPool threadPool; private final Recycler recycler; - private final ExponentialBucketHistogram handlingTimeMillisHistogram; + private final HandlingTimeTracker handlingTimeTracker; private final boolean rstOnClose; private volatile long slowLogThresholdMs = Long.MAX_VALUE; @@ -69,7 +69,7 @@ public final class OutboundHandler { StatsTracker statsTracker, ThreadPool threadPool, Recycler recycler, - ExponentialBucketHistogram handlingTimeMillisHistogram, + HandlingTimeTracker handlingTimeTracker, boolean rstOnClose ) { this.nodeName = nodeName; @@ -77,7 +77,7 @@ public final class OutboundHandler { this.statsTracker = statsTracker; this.threadPool = threadPool; this.recycler = recycler; - this.handlingTimeMillisHistogram = handlingTimeMillisHistogram; + this.handlingTimeTracker = handlingTimeTracker; this.rstOnClose = rstOnClose; } @@ -414,7 +414,7 @@ private void maybeLogSlowMessage(boolean success) { final long logThreshold = slowLogThresholdMs; if (logThreshold > 0) { final long took = threadPool.rawRelativeTimeInMillis() - startTime; - handlingTimeMillisHistogram.addObservation(took); + handlingTimeTracker.addObservation(took); if (took > logThreshold) { logger.warn( "sending transport message [{}] of size [{}] on [{}] took [{}ms] which is above the warn " diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java index abb8921ff1809..9e2fbe737a81d 100644 --- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java +++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java @@ -27,8 +27,8 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.RecyclerBytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.metrics.ExponentialBucketHistogram; import org.elasticsearch.common.network.CloseableChannel; +import org.elasticsearch.common.network.HandlingTimeTracker; import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.network.NetworkUtils; @@ -128,7 +128,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements private final TransportHandshaker handshaker; private final TransportKeepAlive keepAlive; - private final ExponentialBucketHistogram outboundHandlingTimeMillisHistogram = new ExponentialBucketHistogram(); + private final HandlingTimeTracker outboundHandlingTimeTracker = new HandlingTimeTracker(); private final OutboundHandler outboundHandler; private final InboundHandler inboundHandler; private final ResponseHandlers responseHandlers = new ResponseHandlers(); @@ -162,7 +162,7 @@ public TcpTransport( statsTracker, threadPool, recycler, - outboundHandlingTimeMillisHistogram, + outboundHandlingTimeTracker, rstOnClose ); @@ -193,7 +193,7 @@ public TcpTransport( keepAlive, requestHandlers, responseHandlers, - networkService.getHandlingTimeMillisHistogram(), + networkService.getHandlingTimeTracker(), ignoreDeserializationErrors ); } @@ -1001,8 +1001,8 @@ public final TransportStats getStats() { bytesRead, messagesSent, bytesWritten, - networkService.getHandlingTimeMillisHistogram().getSnapshot(), - outboundHandlingTimeMillisHistogram.getSnapshot(), + networkService.getHandlingTimeTracker().getSnapshot(), + outboundHandlingTimeTracker.getSnapshot(), requestHandlers.getStats() ); } diff --git a/server/src/main/java/org/elasticsearch/transport/TransportStats.java b/server/src/main/java/org/elasticsearch/transport/TransportStats.java index 2eb1698fb8f41..6554e0392c043 100644 --- a/server/src/main/java/org/elasticsearch/transport/TransportStats.java +++ b/server/src/main/java/org/elasticsearch/transport/TransportStats.java @@ -14,7 +14,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.metrics.ExponentialBucketHistogram; +import org.elasticsearch.common.network.HandlingTimeTracker; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.xcontent.ChunkedToXContent; import org.elasticsearch.core.TimeValue; @@ -73,11 +73,11 @@ public TransportStats(StreamInput in) throws IOException { && in.getTransportVersion().isPatchFrom(TransportVersions.V_9_0_0) == false) { in.readBoolean(); } - inboundHandlingTimeBucketFrequencies = new long[ExponentialBucketHistogram.BUCKET_COUNT]; + inboundHandlingTimeBucketFrequencies = new long[HandlingTimeTracker.BUCKET_COUNT]; for (int i = 0; i < inboundHandlingTimeBucketFrequencies.length; i++) { inboundHandlingTimeBucketFrequencies[i] = in.readVLong(); } - outboundHandlingTimeBucketFrequencies = new long[ExponentialBucketHistogram.BUCKET_COUNT]; + outboundHandlingTimeBucketFrequencies = new long[HandlingTimeTracker.BUCKET_COUNT]; for (int i = 0; i < inboundHandlingTimeBucketFrequencies.length; i++) { outboundHandlingTimeBucketFrequencies[i] = in.readVLong(); } @@ -97,8 +97,8 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVLong(rxSize); out.writeVLong(txCount); out.writeVLong(txSize); - assert inboundHandlingTimeBucketFrequencies.length == ExponentialBucketHistogram.BUCKET_COUNT; - assert outboundHandlingTimeBucketFrequencies.length == ExponentialBucketHistogram.BUCKET_COUNT; + assert inboundHandlingTimeBucketFrequencies.length == HandlingTimeTracker.BUCKET_COUNT; + assert outboundHandlingTimeBucketFrequencies.length == HandlingTimeTracker.BUCKET_COUNT; if (out.getTransportVersion().before(TransportVersions.TRANSPORT_STATS_HANDLING_TIME_REQUIRED) && out.getTransportVersion().isPatchFrom(TransportVersions.V_9_0_0) == false) { out.writeBoolean(true); @@ -168,7 +168,7 @@ public Map getTransportActionStats() { private boolean assertHistogramsConsistent() { assert inboundHandlingTimeBucketFrequencies.length == outboundHandlingTimeBucketFrequencies.length; - assert inboundHandlingTimeBucketFrequencies.length == ExponentialBucketHistogram.BUCKET_COUNT; + assert inboundHandlingTimeBucketFrequencies.length == HandlingTimeTracker.BUCKET_COUNT; return true; } @@ -200,7 +200,7 @@ public Iterator toXContentChunked(ToXContent.Params outerP } static void histogramToXContent(XContentBuilder builder, long[] bucketFrequencies, String fieldName) throws IOException { - final int[] bucketBounds = ExponentialBucketHistogram.getBucketUpperBounds(); + final int[] bucketBounds = HandlingTimeTracker.getBucketUpperBounds(); int firstBucket = 0; long remainingCount = 0L; diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStatsTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStatsTests.java index df3127977ad5a..b99ae142dabc4 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStatsTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStatsTests.java @@ -28,7 +28,7 @@ import org.elasticsearch.cluster.service.ClusterStateUpdateStats; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.metrics.ExponentialBucketHistogram; +import org.elasticsearch.common.network.HandlingTimeTracker; import org.elasticsearch.common.util.Maps; import org.elasticsearch.common.xcontent.ChunkedToXContent; import org.elasticsearch.core.Tuple; @@ -846,8 +846,8 @@ public static NodeStats createNodeStats() { randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong(), - IntStream.range(0, ExponentialBucketHistogram.BUCKET_COUNT).mapToLong(i -> randomNonNegativeLong()).toArray(), - IntStream.range(0, ExponentialBucketHistogram.BUCKET_COUNT).mapToLong(i -> randomNonNegativeLong()).toArray(), + IntStream.range(0, HandlingTimeTracker.BUCKET_COUNT).mapToLong(i -> randomNonNegativeLong()).toArray(), + IntStream.range(0, HandlingTimeTracker.BUCKET_COUNT).mapToLong(i -> randomNonNegativeLong()).toArray(), Map.of("test-action", new TransportActionStats(1, 2, new long[29], 3, 4, new long[29])) ) : null; diff --git a/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java b/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java index a1950333ddc24..cb9c3fecf6c7d 100644 --- a/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java +++ b/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java @@ -22,7 +22,7 @@ public class ExponentialBucketHistogramTests extends ESTestCase { public void testSnapshot() { - final ExponentialBucketHistogram histogram = new ExponentialBucketHistogram(); + final ExponentialBucketHistogram histogram = new ExponentialBucketHistogram(18); assertArrayEquals(new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, histogram.getSnapshot()); @@ -64,9 +64,10 @@ public void testSnapshot() { } public void testHistogramRandom() { - final int[] upperBounds = ExponentialBucketHistogram.getBucketUpperBounds(); + final int bucketCount = randomIntBetween(5, 20); + final int[] upperBounds = ExponentialBucketHistogram.getBucketUpperBounds(bucketCount); final long[] expectedCounts = new long[upperBounds.length + 1]; - final ExponentialBucketHistogram histogram = new ExponentialBucketHistogram(); + final ExponentialBucketHistogram histogram = new ExponentialBucketHistogram(bucketCount); for (int i = between(0, 1000); i > 0; i--) { final int bucket = between(0, expectedCounts.length - 1); expectedCounts[bucket] += 1; @@ -80,7 +81,8 @@ public void testHistogramRandom() { } public void testBoundsConsistency() { - final int[] upperBounds = ExponentialBucketHistogram.getBucketUpperBounds(); + final int bucketCount = randomIntBetween(5, 20); + final int[] upperBounds = ExponentialBucketHistogram.getBucketUpperBounds(bucketCount); assertThat(upperBounds[0], greaterThan(0)); for (int i = 1; i < upperBounds.length; i++) { assertThat(upperBounds[i], greaterThan(upperBounds[i - 1])); @@ -88,13 +90,14 @@ public void testBoundsConsistency() { } public void testPercentile() { - ExponentialBucketHistogram histogram = new ExponentialBucketHistogram(); + final int bucketCount = randomIntBetween(5, 20); + ExponentialBucketHistogram histogram = new ExponentialBucketHistogram(bucketCount); int valueCount = randomIntBetween(100, 10_000); for (int i = 0; i < valueCount; i++) { histogram.addObservation(i); } final long[] snapshot = histogram.getSnapshot(); - final int[] bucketUpperBounds = ExponentialBucketHistogram.getBucketUpperBounds(); + final int[] bucketUpperBounds = histogram.calculateBucketUpperBounds(); for (int i = 0; i <= 100; i++) { final float percentile = i / 100.0f; final long actualPercentile = (long) Math.ceil(valueCount * percentile); @@ -105,15 +108,17 @@ public void testPercentile() { } public void testMaxPercentile() { - ExponentialBucketHistogram histogram = new ExponentialBucketHistogram(); - int[] bucketUpperBounds = ExponentialBucketHistogram.getBucketUpperBounds(); + final int bucketCount = randomIntBetween(5, 20); + ExponentialBucketHistogram histogram = new ExponentialBucketHistogram(bucketCount); + int[] bucketUpperBounds = histogram.calculateBucketUpperBounds(); int secondToLastBucketUpperBound = bucketUpperBounds[bucketUpperBounds.length - 1]; histogram.addObservation(secondToLastBucketUpperBound + 1); assertThat(histogram.getPercentile(1.0f), equalTo(Long.MAX_VALUE)); } public void testClear() { - ExponentialBucketHistogram histogram = new ExponentialBucketHistogram(); + final int bucketCount = randomIntBetween(5, 20); + ExponentialBucketHistogram histogram = new ExponentialBucketHistogram(bucketCount); for (int i = 0; i < 100; i++) { histogram.addObservation(randomIntBetween(1, 100_000)); } @@ -123,7 +128,8 @@ public void testClear() { } public void testPercentileValidation() { - ExponentialBucketHistogram histogram = new ExponentialBucketHistogram(); + final int bucketCount = randomIntBetween(5, 20); + ExponentialBucketHistogram histogram = new ExponentialBucketHistogram(bucketCount); for (int i = 0; i < 100; i++) { histogram.addObservation(randomIntBetween(1, 100_000)); } diff --git a/server/src/test/java/org/elasticsearch/common/network/HandlingTimeTrackerTests.java b/server/src/test/java/org/elasticsearch/common/network/HandlingTimeTrackerTests.java new file mode 100644 index 0000000000000..cb37128de3ede --- /dev/null +++ b/server/src/test/java/org/elasticsearch/common/network/HandlingTimeTrackerTests.java @@ -0,0 +1,84 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.common.network; + +import org.elasticsearch.test.ESTestCase; + +import static org.hamcrest.Matchers.greaterThan; + +public class HandlingTimeTrackerTests extends ESTestCase { + + public void testHistogram() { + final HandlingTimeTracker handlingTimeTracker = new HandlingTimeTracker(); + + assertArrayEquals(new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, handlingTimeTracker.getSnapshot()); + + handlingTimeTracker.addObservation(0L); + assertArrayEquals(new long[] { 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, handlingTimeTracker.getSnapshot()); + + handlingTimeTracker.addObservation(1L); + assertArrayEquals(new long[] { 1, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, handlingTimeTracker.getSnapshot()); + + handlingTimeTracker.addObservation(2L); + assertArrayEquals(new long[] { 1, 1, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, handlingTimeTracker.getSnapshot()); + + handlingTimeTracker.addObservation(3L); + assertArrayEquals(new long[] { 1, 1, 2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, handlingTimeTracker.getSnapshot()); + + handlingTimeTracker.addObservation(4L); + assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, handlingTimeTracker.getSnapshot()); + + handlingTimeTracker.addObservation(127L); + assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, handlingTimeTracker.getSnapshot()); + + handlingTimeTracker.addObservation(128L); + assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0 }, handlingTimeTracker.getSnapshot()); + + handlingTimeTracker.addObservation(65535L); + assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 0 }, handlingTimeTracker.getSnapshot()); + + handlingTimeTracker.addObservation(65536L); + assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 1 }, handlingTimeTracker.getSnapshot()); + + handlingTimeTracker.addObservation(Long.MAX_VALUE); + assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 2 }, handlingTimeTracker.getSnapshot()); + + handlingTimeTracker.addObservation(randomLongBetween(65536L, Long.MAX_VALUE)); + assertArrayEquals(new long[] { 1, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 3 }, handlingTimeTracker.getSnapshot()); + + handlingTimeTracker.addObservation(randomLongBetween(Long.MIN_VALUE, 0L)); + assertArrayEquals(new long[] { 2, 1, 2, 1, 0, 0, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0, 1, 3 }, handlingTimeTracker.getSnapshot()); + } + + public void testHistogramRandom() { + final int[] upperBounds = HandlingTimeTracker.getBucketUpperBounds(); + final long[] expectedCounts = new long[upperBounds.length + 1]; + final HandlingTimeTracker handlingTimeTracker = new HandlingTimeTracker(); + for (int i = between(0, 1000); i > 0; i--) { + final int bucket = between(0, expectedCounts.length - 1); + expectedCounts[bucket] += 1; + + final int lowerBound = bucket == 0 ? 0 : upperBounds[bucket - 1]; + final int upperBound = bucket == upperBounds.length ? randomBoolean() ? 100000 : Integer.MAX_VALUE : upperBounds[bucket] - 1; + handlingTimeTracker.addObservation(between(lowerBound, upperBound)); + } + + assertArrayEquals(expectedCounts, handlingTimeTracker.getSnapshot()); + } + + public void testBoundsConsistency() { + final int[] upperBounds = HandlingTimeTracker.getBucketUpperBounds(); + assertThat(upperBounds[0], greaterThan(0)); + for (int i = 1; i < upperBounds.length; i++) { + assertThat(upperBounds[i], greaterThan(upperBounds[i - 1])); + } + } + +} diff --git a/server/src/test/java/org/elasticsearch/http/HttpRouteStatsTests.java b/server/src/test/java/org/elasticsearch/http/HttpRouteStatsTests.java index 796995edd3bc7..b2095238a8c88 100644 --- a/server/src/test/java/org/elasticsearch/http/HttpRouteStatsTests.java +++ b/server/src/test/java/org/elasticsearch/http/HttpRouteStatsTests.java @@ -10,7 +10,7 @@ package org.elasticsearch.http; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.metrics.ExponentialBucketHistogram; +import org.elasticsearch.common.network.HandlingTimeTracker; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.core.TimeValue; import org.elasticsearch.test.ESTestCase; @@ -236,7 +236,7 @@ private static void assertHandlingTimeHistogram(long[] histogram, String expecte "millis", TimeValue::timeValueMillis, histogram, - ExponentialBucketHistogram.getBucketUpperBounds() + HandlingTimeTracker.getBucketUpperBounds() ); return builder; }, false, true)); diff --git a/server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java b/server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java index 79ce5b817d90d..a683c2332c451 100644 --- a/server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java @@ -22,7 +22,7 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.RecyclerBytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.metrics.ExponentialBucketHistogram; +import org.elasticsearch.common.network.HandlingTimeTracker; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.PageCacheRecycler; import org.elasticsearch.common.util.concurrent.EsExecutors; @@ -80,7 +80,7 @@ public void setUp() throws Exception { new StatsTracker(), threadPool, new BytesRefRecycler(PageCacheRecycler.NON_RECYCLING_INSTANCE), - new ExponentialBucketHistogram(), + new HandlingTimeTracker(), false ); requestHandlers = new Transport.RequestHandlers(); @@ -93,7 +93,7 @@ public void setUp() throws Exception { keepAlive, requestHandlers, responseHandlers, - new ExponentialBucketHistogram(), + new HandlingTimeTracker(), ignoreDeserializationErrors ); } diff --git a/server/src/test/java/org/elasticsearch/transport/OutboundHandlerTests.java b/server/src/test/java/org/elasticsearch/transport/OutboundHandlerTests.java index b1dd33696eddf..8d18841f8e18e 100644 --- a/server/src/test/java/org/elasticsearch/transport/OutboundHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/transport/OutboundHandlerTests.java @@ -25,7 +25,7 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.metrics.ExponentialBucketHistogram; +import org.elasticsearch.common.network.HandlingTimeTracker; import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.util.PageCacheRecycler; @@ -83,7 +83,7 @@ public void setUp() throws Exception { statsTracker, threadPool, recycler, - new ExponentialBucketHistogram(), + new HandlingTimeTracker(), false ); diff --git a/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java b/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java index 2067a8dedeae9..7099c33dda75f 100644 --- a/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java +++ b/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java @@ -16,7 +16,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.component.Lifecycle; import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.metrics.ExponentialBucketHistogram; +import org.elasticsearch.common.network.HandlingTimeTracker; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.network.NetworkUtils; import org.elasticsearch.common.settings.Settings; @@ -599,7 +599,7 @@ private void testExceptionHandling( new StatsTracker(), testThreadPool, new BytesRefRecycler(new MockPageCacheRecycler(Settings.EMPTY)), - new ExponentialBucketHistogram(), + new HandlingTimeTracker(), false ) ); diff --git a/server/src/test/java/org/elasticsearch/transport/TransportStatsTests.java b/server/src/test/java/org/elasticsearch/transport/TransportStatsTests.java index e07b0ca9dc43e..1c9cb4c9afc0f 100644 --- a/server/src/test/java/org/elasticsearch/transport/TransportStatsTests.java +++ b/server/src/test/java/org/elasticsearch/transport/TransportStatsTests.java @@ -10,7 +10,7 @@ package org.elasticsearch.transport; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.metrics.ExponentialBucketHistogram; +import org.elasticsearch.common.network.HandlingTimeTracker; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xcontent.ToXContentFragment; @@ -20,7 +20,7 @@ public class TransportStatsTests extends ESTestCase { public void testToXContent() { - final var histogram = new long[ExponentialBucketHistogram.BUCKET_COUNT]; + final var histogram = new long[HandlingTimeTracker.BUCKET_COUNT]; histogram[4] = 10; final var requestSizeHistogram = new long[29]; @@ -67,7 +67,7 @@ private static void assertHistogram(long[] histogram, String expectedJson) { } public void testHistogram() { - final var histogram = new long[ExponentialBucketHistogram.BUCKET_COUNT]; + final var histogram = new long[HandlingTimeTracker.BUCKET_COUNT]; assertHistogram(histogram, """ {"h":[]}"""); @@ -102,11 +102,11 @@ public void testHistogram() { ]}"""); Arrays.fill(histogram, 0L); - histogram[ExponentialBucketHistogram.BUCKET_COUNT - 1] = 5; + histogram[HandlingTimeTracker.BUCKET_COUNT - 1] = 5; assertHistogram(histogram, """ {"h":[{"ge":"1m","ge_millis":65536,"count":5}]}"""); - histogram[ExponentialBucketHistogram.BUCKET_COUNT - 3] = 6; + histogram[HandlingTimeTracker.BUCKET_COUNT - 3] = 6; assertHistogram(histogram, """ {"h":[\ {"ge":"16.3s","ge_millis":16384,"lt":"32.7s","lt_millis":32768,"count":6},\ diff --git a/test/framework/src/main/java/org/elasticsearch/transport/TestTransportChannels.java b/test/framework/src/main/java/org/elasticsearch/transport/TestTransportChannels.java index 97fd4348a2316..effabd85591f9 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/TestTransportChannels.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/TestTransportChannels.java @@ -10,7 +10,7 @@ package org.elasticsearch.transport; import org.elasticsearch.TransportVersion; -import org.elasticsearch.common.metrics.ExponentialBucketHistogram; +import org.elasticsearch.common.network.HandlingTimeTracker; import org.elasticsearch.common.util.PageCacheRecycler; import org.elasticsearch.threadpool.ThreadPool; @@ -26,7 +26,7 @@ public static TcpTransportChannel newFakeTcpTransportChannel( ) { BytesRefRecycler recycler = new BytesRefRecycler(PageCacheRecycler.NON_RECYCLING_INSTANCE); return new TcpTransportChannel( - new OutboundHandler(nodeName, version, new StatsTracker(), threadPool, recycler, new ExponentialBucketHistogram(), false), + new OutboundHandler(nodeName, version, new StatsTracker(), threadPool, recycler, new HandlingTimeTracker(), false), channel, action, requestId, From 91261d68adae2db57d41edab667c5af666172c7f Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Mon, 5 May 2025 14:44:47 +1000 Subject: [PATCH 32/33] Validate that bucket count is sane --- .../common/metrics/ExponentialBucketHistogram.java | 3 +++ .../metrics/ExponentialBucketHistogramTests.java | 13 +++++++++++++ 2 files changed, 16 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java b/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java index 49c27d6de9f1f..8c88998ce09a0 100644 --- a/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java +++ b/server/src/main/java/org/elasticsearch/common/metrics/ExponentialBucketHistogram.java @@ -47,6 +47,9 @@ private int getBucket(long observedValue) { private final LongAdder[] buckets; public ExponentialBucketHistogram(int bucketCount) { + if (bucketCount < 2 || bucketCount > Integer.SIZE) { + throw new IllegalArgumentException("Bucket count must be in [2, " + Integer.SIZE + "], got " + bucketCount); + } this.bucketCount = bucketCount; this.lastBucketLowerBound = getBucketUpperBounds(bucketCount)[bucketCount - 2]; buckets = new LongAdder[bucketCount]; diff --git a/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java b/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java index cb9c3fecf6c7d..3b8e44df83df4 100644 --- a/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java +++ b/server/src/test/java/org/elasticsearch/common/metrics/ExponentialBucketHistogramTests.java @@ -146,4 +146,17 @@ public void testPercentileValidation() { () -> histogram.getPercentile(randomFloatBetween(1.0f, Float.POSITIVE_INFINITY, false)) ); } + + public void testBucketCountValidation() { + // Valid values + for (int i = 2; i <= Integer.SIZE; i++) { + new ExponentialBucketHistogram(i); + } + // Invalid values + assertThrows(IllegalArgumentException.class, () -> new ExponentialBucketHistogram(randomIntBetween(Integer.MIN_VALUE, 1))); + assertThrows( + IllegalArgumentException.class, + () -> new ExponentialBucketHistogram(randomIntBetween(Integer.SIZE, Integer.MAX_VALUE)) + ); + } } From a3707eb72eaa1d871a9370789a77ea1d6c047000 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Mon, 5 May 2025 14:49:16 +1000 Subject: [PATCH 33/33] Restore comment --- .../org/elasticsearch/common/network/HandlingTimeTracker.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/common/network/HandlingTimeTracker.java b/server/src/main/java/org/elasticsearch/common/network/HandlingTimeTracker.java index 0c7c81d3f3d16..e466d71b883ef 100644 --- a/server/src/main/java/org/elasticsearch/common/network/HandlingTimeTracker.java +++ b/server/src/main/java/org/elasticsearch/common/network/HandlingTimeTracker.java @@ -11,6 +11,9 @@ import org.elasticsearch.common.metrics.ExponentialBucketHistogram; +/** + * Tracks how long message handling takes on a transport thread as a histogram with fixed buckets. + */ public class HandlingTimeTracker extends ExponentialBucketHistogram { public static final int BUCKET_COUNT = 18;