Skip to content

Commit a692cba

Browse files
liwangeolivelli
liwang
authored andcommitted
ZOOKEEPER-4289: Reduce the performance impact of Prometheus metrics
Enabling Prometheus provider has significant impact on the performance of both read and write operations. This is to reduce the impact by making the Prometheus summary reporting as an async operation. Load test results showed that the avg read latency and throughput after the fix is on par with the performance when Prometheus is disabled. For writes, the avg latency was reduced 25% and the avg throughput was increased 20% after the fix. Author: liwang <[email protected]> Reviewers: Enrico Olivelli <[email protected]>, Andor Molnar <[email protected]> Closes apache#1698 from li4wang/ZOOKEEPER-4289
1 parent 2aa17ff commit a692cba

File tree

3 files changed

+198
-12
lines changed

3 files changed

+198
-12
lines changed

zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md

+17-1
Original file line numberDiff line numberDiff line change
@@ -2129,7 +2129,23 @@ options are used to configure the [AdminServer](#sc_adminserver).
21292129
* *metricsProvider.exportJvmInfo* :
21302130
If this property is set to **true** Prometheus.io will export useful metrics about the JVM.
21312131
The default is true.
2132-
2132+
2133+
* *metricsProvider.numWorkerThreads* :
2134+
**New in 3.7.1:**
2135+
Number of worker threads for reporting Prometheus summary metrics.
2136+
Default value is 1.
2137+
If the number is less than 1, the main thread will be used.
2138+
2139+
* *metricsProvider.maxQueueSize* :
2140+
**New in 3.7.1:**
2141+
The max queue size for Prometheus summary metrics reporting task.
2142+
Default value is 1000000.
2143+
2144+
* *metricsProvider.workerShutdownTimeoutMs* :
2145+
**New in 3.7.1:**
2146+
The timeout in ms for Prometheus worker threads shutdown.
2147+
Default value is 1000ms.
2148+
21332149
<a name="Communication+using+the+Netty+framework"></a>
21342150

21352151
### Communication using the Netty framework

zookeeper-metrics-providers/zookeeper-prometheus-metrics/src/main/java/org/apache/zookeeper/metrics/prometheus/PrometheusMetricsProvider.java

+116-5
Original file line numberDiff line numberDiff line change
@@ -26,9 +26,18 @@
2626
import java.net.InetSocketAddress;
2727
import java.util.Enumeration;
2828
import java.util.Objects;
29+
import java.util.Optional;
2930
import java.util.Properties;
31+
import java.util.concurrent.BlockingQueue;
3032
import java.util.concurrent.ConcurrentHashMap;
3133
import java.util.concurrent.ConcurrentMap;
34+
import java.util.concurrent.ExecutorService;
35+
import java.util.concurrent.LinkedBlockingQueue;
36+
import java.util.concurrent.RejectedExecutionException;
37+
import java.util.concurrent.ThreadFactory;
38+
import java.util.concurrent.ThreadPoolExecutor;
39+
import java.util.concurrent.TimeUnit;
40+
import java.util.concurrent.atomic.AtomicInteger;
3241
import java.util.function.BiConsumer;
3342
import javax.servlet.ServletException;
3443
import javax.servlet.http.HttpServletRequest;
@@ -40,6 +49,7 @@
4049
import org.apache.zookeeper.metrics.MetricsProviderLifeCycleException;
4150
import org.apache.zookeeper.metrics.Summary;
4251
import org.apache.zookeeper.metrics.SummarySet;
52+
import org.apache.zookeeper.server.RateLogger;
4353
import org.eclipse.jetty.server.Server;
4454
import org.eclipse.jetty.servlet.ServletContextHandler;
4555
import org.eclipse.jetty.servlet.ServletHolder;
@@ -56,6 +66,26 @@ public class PrometheusMetricsProvider implements MetricsProvider {
5666
private static final Logger LOG = LoggerFactory.getLogger(PrometheusMetricsProvider.class);
5767
private static final String LABEL = "key";
5868
private static final String[] LABELS = {LABEL};
69+
70+
/**
71+
* Number of worker threads for reporting Prometheus summary metrics.
72+
* Default value is 1.
73+
* If the number is less than 1, the main thread will be used.
74+
*/
75+
static final String NUM_WORKER_THREADS = "numWorkerThreads";
76+
77+
/**
78+
* The max queue size for Prometheus summary metrics reporting task.
79+
* Default value is 1000000.
80+
*/
81+
static final String MAX_QUEUE_SIZE = "maxQueueSize";
82+
83+
/**
84+
* The timeout in ms for Prometheus worker threads shutdown.
85+
* Default value is 1000ms.
86+
*/
87+
static final String WORKER_SHUTDOWN_TIMEOUT_MS = "workerShutdownTimeoutMs";
88+
5989
/**
6090
* We are using the 'defaultRegistry'.
6191
* <p>
@@ -64,23 +94,35 @@ public class PrometheusMetricsProvider implements MetricsProvider {
6494
* </p>
6595
*/
6696
private final CollectorRegistry collectorRegistry = CollectorRegistry.defaultRegistry;
97+
private final RateLogger rateLogger = new RateLogger(LOG, 60 * 1000);
6798
private String host = "0.0.0.0";
6899
private int port = 7000;
69100
private boolean exportJvmInfo = true;
70101
private Server server;
71102
private final MetricsServletImpl servlet = new MetricsServletImpl();
72103
private final Context rootContext = new Context();
104+
private int numWorkerThreads = 1;
105+
private int maxQueueSize = 1000000;
106+
private long workerShutdownTimeoutMs = 1000;
107+
private Optional<ExecutorService> executorOptional = Optional.empty();
73108

74109
@Override
75110
public void configure(Properties configuration) throws MetricsProviderLifeCycleException {
76111
LOG.info("Initializing metrics, configuration: {}", configuration);
77112
this.host = configuration.getProperty("httpHost", "0.0.0.0");
78113
this.port = Integer.parseInt(configuration.getProperty("httpPort", "7000"));
79114
this.exportJvmInfo = Boolean.parseBoolean(configuration.getProperty("exportJvmInfo", "true"));
115+
this.numWorkerThreads = Integer.parseInt(
116+
configuration.getProperty(NUM_WORKER_THREADS, "1"));
117+
this.maxQueueSize = Integer.parseInt(
118+
configuration.getProperty(MAX_QUEUE_SIZE, "1000000"));
119+
this.workerShutdownTimeoutMs = Long.parseLong(
120+
configuration.getProperty(WORKER_SHUTDOWN_TIMEOUT_MS, "1000"));
80121
}
81122

82123
@Override
83124
public void start() throws MetricsProviderLifeCycleException {
125+
this.executorOptional = createExecutor();
84126
try {
85127
LOG.info("Starting /metrics HTTP endpoint at host: {}, port: {}, exportJvmInfo: {}",
86128
host, port, exportJvmInfo);
@@ -120,6 +162,7 @@ public MetricsContext getRootContext() {
120162

121163
@Override
122164
public void stop() {
165+
shutdownExecutor();
123166
if (server != null) {
124167
try {
125168
server.stop();
@@ -331,7 +374,7 @@ private void unregister() {
331374

332375
}
333376

334-
private class PrometheusSummary implements Summary {
377+
class PrometheusSummary implements Summary {
335378

336379
private final io.prometheus.client.Summary inner;
337380
private final String name;
@@ -355,16 +398,19 @@ public PrometheusSummary(String name, MetricsContext.DetailLevel level) {
355398

356399
@Override
357400
public void add(long delta) {
401+
reportMetrics(() -> observe(delta));
402+
}
403+
404+
void observe(final long delta) {
358405
try {
359406
inner.observe(delta);
360-
} catch (IllegalArgumentException err) {
407+
} catch (final IllegalArgumentException err) {
361408
LOG.error("invalid delta {} for metric {}", delta, name, err);
362409
}
363410
}
364-
365411
}
366412

367-
private class PrometheusLabelledSummary implements SummarySet {
413+
class PrometheusLabelledSummary implements SummarySet {
368414

369415
private final io.prometheus.client.Summary inner;
370416
private final String name;
@@ -390,9 +436,13 @@ public PrometheusLabelledSummary(String name, MetricsContext.DetailLevel level)
390436

391437
@Override
392438
public void add(String key, long value) {
439+
reportMetrics(() -> observe(key, value));
440+
}
441+
442+
void observe(final String key, final long value) {
393443
try {
394444
inner.labels(key).observe(value);
395-
} catch (IllegalArgumentException err) {
445+
} catch (final IllegalArgumentException err) {
396446
LOG.error("invalid value {} for metric {} with key {}", value, name, key, err);
397447
}
398448
}
@@ -410,4 +460,65 @@ protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws Se
410460
super.doGet(req, resp);
411461
}
412462
}
463+
464+
private Optional<ExecutorService> createExecutor() {
465+
if (numWorkerThreads < 1) {
466+
LOG.info("Executor service was not created as numWorkerThreads {} is less than 1", numWorkerThreads);
467+
return Optional.empty();
468+
}
469+
470+
final BlockingQueue<Runnable> queue = new LinkedBlockingQueue<>(maxQueueSize);
471+
final ThreadPoolExecutor executor = new ThreadPoolExecutor(numWorkerThreads,
472+
numWorkerThreads,
473+
0L,
474+
TimeUnit.MILLISECONDS,
475+
queue, new PrometheusWorkerThreadFactory());
476+
LOG.info("Executor service was created with numWorkerThreads {} and maxQueueSize {}",
477+
numWorkerThreads,
478+
maxQueueSize);
479+
return Optional.of(executor);
480+
}
481+
482+
private void shutdownExecutor() {
483+
if (executorOptional.isPresent()) {
484+
LOG.info("Shutdown executor service with timeout {}", workerShutdownTimeoutMs);
485+
final ExecutorService executor = executorOptional.get();
486+
executor.shutdown();
487+
try {
488+
if (!executor.awaitTermination(workerShutdownTimeoutMs, TimeUnit.MILLISECONDS)) {
489+
LOG.error("Not all the Prometheus worker threads terminated properly after {} timeout",
490+
workerShutdownTimeoutMs);
491+
executor.shutdownNow();
492+
}
493+
} catch (final Exception e) {
494+
LOG.error("Error occurred while terminating Prometheus worker threads", e);
495+
executor.shutdownNow();
496+
}
497+
}
498+
}
499+
500+
private static class PrometheusWorkerThreadFactory implements ThreadFactory {
501+
private static final AtomicInteger workerCounter = new AtomicInteger(1);
502+
503+
@Override
504+
public Thread newThread(final Runnable runnable) {
505+
final String threadName = "PrometheusMetricsProviderWorker-" + workerCounter.getAndIncrement();
506+
final Thread thread = new Thread(runnable, threadName);
507+
thread.setDaemon(true);
508+
return thread;
509+
}
510+
}
511+
512+
private void reportMetrics(final Runnable task) {
513+
if (executorOptional.isPresent()) {
514+
try {
515+
executorOptional.get().submit(task);
516+
} catch (final RejectedExecutionException e) {
517+
rateLogger.rateLimitLog("Prometheus metrics reporting task queue size exceeded the max",
518+
String.valueOf(maxQueueSize));
519+
}
520+
} else {
521+
task.run();
522+
}
523+
}
413524
}

zookeeper-metrics-providers/zookeeper-prometheus-metrics/src/test/java/org/apache/zookeeper/metrics/prometheus/PrometheusMetricsProviderTest.java

+65-6
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,8 @@
3030
import java.io.IOException;
3131
import java.io.PrintWriter;
3232
import java.io.StringWriter;
33+
import java.util.HashMap;
34+
import java.util.Map;
3335
import java.util.Properties;
3436
import javax.servlet.ServletException;
3537
import javax.servlet.http.HttpServletRequest;
@@ -178,10 +180,11 @@ public void testGauge() throws Exception {
178180

179181
@Test
180182
public void testBasicSummary() throws Exception {
181-
Summary summary = provider.getRootContext()
183+
final PrometheusMetricsProvider.PrometheusSummary summary =
184+
(PrometheusMetricsProvider.PrometheusSummary) provider.getRootContext()
182185
.getSummary("cc", MetricsContext.DetailLevel.BASIC);
183-
summary.add(10);
184-
summary.add(10);
186+
summary.observe(10);
187+
summary.observe(10);
185188
int[] count = {0};
186189
provider.dump((k, v) -> {
187190
count[0]++;
@@ -227,10 +230,11 @@ public void testBasicSummary() throws Exception {
227230

228231
@Test
229232
public void testAdvancedSummary() throws Exception {
230-
Summary summary = provider.getRootContext()
233+
final PrometheusMetricsProvider.PrometheusSummary summary =
234+
(PrometheusMetricsProvider.PrometheusSummary) provider.getRootContext()
231235
.getSummary("cc", MetricsContext.DetailLevel.ADVANCED);
232-
summary.add(10);
233-
summary.add(10);
236+
summary.observe(10);
237+
summary.observe(10);
234238
int[] count = {0};
235239
provider.dump((k, v) -> {
236240
count[0]++;
@@ -282,6 +286,61 @@ public void testAdvancedSummary() throws Exception {
282286
assertThat(res, CoreMatchers.containsString("cc{quantile=\"0.99\",} 10.0"));
283287
}
284288

289+
@Test
290+
public void testSummary_sync() throws Exception {
291+
final Properties config = new Properties();
292+
config.setProperty("numWorkerThreads", "0");
293+
config.setProperty("httpPort", "0"); // ephemeral port
294+
config.setProperty("exportJvmInfo", "false");
295+
296+
PrometheusMetricsProvider metricsProvider = null;
297+
try {
298+
metricsProvider = new PrometheusMetricsProvider();
299+
metricsProvider.configure(config);
300+
metricsProvider.start();
301+
302+
final Summary summary =
303+
metricsProvider.getRootContext().getSummary("cc", MetricsContext.DetailLevel.BASIC);
304+
summary.add(10);
305+
summary.add(20);
306+
307+
final Map<String, Object> res = new HashMap<>();
308+
metricsProvider.dump(res::put);
309+
assertEquals(3, res.keySet().stream().filter(key -> key.startsWith("cc")).count());
310+
} finally {
311+
if (metricsProvider != null) {
312+
metricsProvider.stop();
313+
}
314+
}
315+
}
316+
317+
@Test
318+
public void testSummary_asyncAndExceedMaxQueueSize() throws Exception {
319+
final Properties config = new Properties();
320+
config.setProperty("numWorkerThreads", "1");
321+
config.setProperty("maxQueueSize", "1");
322+
config.setProperty("httpPort", "0"); // ephemeral port
323+
config.setProperty("exportJvmInfo", "false");
324+
325+
PrometheusMetricsProvider metricsProvider = null;
326+
try {
327+
metricsProvider = new PrometheusMetricsProvider();
328+
metricsProvider.configure(config);
329+
metricsProvider.start();
330+
final Summary summary =
331+
metricsProvider.getRootContext().getSummary("cc", MetricsContext.DetailLevel.ADVANCED);
332+
333+
// make sure no error is thrown
334+
for (int i = 0; i < 10; i++) {
335+
summary.add(10);
336+
}
337+
} finally {
338+
if (metricsProvider != null) {
339+
metricsProvider.stop();
340+
}
341+
}
342+
}
343+
285344
private String callServlet() throws ServletException, IOException {
286345
// we are not performing an HTTP request
287346
// but we are calling directly the servlet

0 commit comments

Comments
 (0)