() {
+
+ @Override
+ protected LocalData initialValue() throws Exception {
+ LocalData localData = new LocalData();
+ map.put(localData, Boolean.TRUE);
+ return localData;
+ }
+
+ @Override
+ protected void onRemoval(LocalData value) throws Exception {
+ map.remove(value);
+ }
+ };
+ }
+
+ @Override
+ public String toString() {
+ return "DataSketchesOpStatsLogger{labels=" + labels + ", id=" + System.identityHashCode(this) + "}";
+ }
+}
diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/LongAdderCounter.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/LongAdderCounter.java
new file mode 100644
index 0000000000000..a5a8e4d2f029f
--- /dev/null
+++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/LongAdderCounter.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.metrics.prometheus.bookkeeper;
+
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.LongAdder;
+import org.apache.bookkeeper.stats.Counter;
+
+/**
+ * {@link Counter} implementation based on {@link LongAdder}.
+ *
+ * LongAdder keeps a counter per-thread and then aggregates to get the result, in order to avoid contention between
+ * multiple threads.
+ */
+public class LongAdderCounter implements Counter {
+ private final LongAdder counter = new LongAdder();
+
+ private Map labels;
+
+ // used for lazy registration for thread scoped metric
+ private boolean threadInitialized;
+
+ public LongAdderCounter(Map labels) {
+ this.labels = labels;
+ }
+
+ @Override
+ public void clear() {
+ counter.reset();
+ }
+
+ @Override
+ public void inc() {
+ counter.increment();
+ }
+
+ @Override
+ public void dec() {
+ counter.decrement();
+ }
+
+ @Override
+ public void addCount(long delta) {
+ counter.add(delta);
+ }
+
+ /**
+ * When counter is used to count time.
+ * consistent with the {@link DataSketchesOpStatsLogger#registerSuccessfulEvent(long, TimeUnit)} 's logic
+ * */
+ @Override
+ public void addLatency(long eventLatency, TimeUnit unit) {
+ long valueMillis = unit.toMillis(eventLatency);
+ counter.add(valueMillis);
+ }
+
+ @Override
+ public Long get() {
+ return counter.sum();
+ }
+
+ public Map getLabels() {
+ return labels;
+ }
+
+ public boolean isThreadInitialized() {
+ return threadInitialized;
+ }
+
+ public void initializeThread(Map labels) {
+ this.labels = labels;
+ this.threadInitialized = true;
+ }
+}
diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusMetricsProvider.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusMetricsProvider.java
new file mode 100644
index 0000000000000..e5568625328f4
--- /dev/null
+++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusMetricsProvider.java
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.metrics.prometheus.bookkeeper;
+
+// CHECKSTYLE.OFF: IllegalImport
+
+import com.google.common.annotations.VisibleForTesting;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import io.netty.util.internal.PlatformDependent;
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Gauge;
+import io.prometheus.client.Gauge.Child;
+import io.prometheus.client.hotspot.GarbageCollectorExports;
+import io.prometheus.client.hotspot.MemoryPoolsExports;
+import io.prometheus.client.hotspot.StandardExports;
+import io.prometheus.client.hotspot.ThreadExports;
+import java.io.IOException;
+import java.io.Writer;
+import java.lang.management.BufferPoolMXBean;
+import java.lang.management.ManagementFactory;
+import java.lang.reflect.Field;
+import java.net.InetSocketAddress;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Supplier;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.StatsProvider;
+import org.apache.bookkeeper.stats.ThreadRegistry;
+import org.apache.commons.configuration2.Configuration;
+import org.apache.commons.lang3.StringUtils;
+import org.eclipse.jetty.ee8.servlet.ServletContextHandler;
+import org.eclipse.jetty.ee8.servlet.ServletHolder;
+import org.eclipse.jetty.server.Server;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+// CHECKSTYLE.ON: IllegalImport
+
+/**
+ * A Prometheus based {@link StatsProvider} implementation.
+ */
+public class PrometheusMetricsProvider implements StatsProvider {
+
+ private ScheduledExecutorService executor;
+
+ public static final String PROMETHEUS_STATS_HTTP_ENABLE = "prometheusStatsHttpEnable";
+ public static final boolean DEFAULT_PROMETHEUS_STATS_HTTP_ENABLE = true;
+
+ public static final String PROMETHEUS_STATS_HTTP_ADDRESS = "prometheusStatsHttpAddress";
+ public static final String DEFAULT_PROMETHEUS_STATS_HTTP_ADDR = "0.0.0.0";
+
+ public static final String PROMETHEUS_STATS_HTTP_PORT = "prometheusStatsHttpPort";
+ public static final int DEFAULT_PROMETHEUS_STATS_HTTP_PORT = 8000;
+
+ public static final String PROMETHEUS_STATS_LATENCY_ROLLOVER_SECONDS = "prometheusStatsLatencyRolloverSeconds";
+ public static final int DEFAULT_PROMETHEUS_STATS_LATENCY_ROLLOVER_SECONDS = 60;
+
+ final CollectorRegistry registry;
+
+ Server server;
+
+ /*
+ * These acts a registry of the metrics defined in this provider
+ */
+ final ConcurrentMap counters = new ConcurrentHashMap<>();
+ final ConcurrentMap> gauges = new ConcurrentHashMap<>();
+ final ConcurrentMap opStats = new ConcurrentHashMap<>();
+ final ConcurrentMap threadScopedOpStats =
+ new ConcurrentHashMap<>();
+ final ConcurrentMap threadScopedCounters =
+ new ConcurrentHashMap<>();
+
+ public PrometheusMetricsProvider() {
+ this(CollectorRegistry.defaultRegistry);
+ }
+
+ public PrometheusMetricsProvider(CollectorRegistry registry) {
+ this.registry = registry;
+ }
+
+ @Override
+ public void start(Configuration conf) {
+ boolean httpEnabled = conf.getBoolean(PROMETHEUS_STATS_HTTP_ENABLE, DEFAULT_PROMETHEUS_STATS_HTTP_ENABLE);
+ boolean bkHttpServerEnabled = conf.getBoolean("httpServerEnabled", false);
+ boolean exposeDefaultJVMMetrics = conf.getBoolean("exposeDefaultJVMMetrics", true);
+ // only start its own http server when prometheus http is enabled and bk http server is not enabled.
+ if (httpEnabled && !bkHttpServerEnabled) {
+ String httpAddr = conf.getString(PROMETHEUS_STATS_HTTP_ADDRESS, DEFAULT_PROMETHEUS_STATS_HTTP_ADDR);
+ int httpPort = conf.getInt(PROMETHEUS_STATS_HTTP_PORT, DEFAULT_PROMETHEUS_STATS_HTTP_PORT);
+ InetSocketAddress httpEndpoint = InetSocketAddress.createUnresolved(httpAddr, httpPort);
+ this.server = new Server(httpEndpoint);
+ ServletContextHandler context = new ServletContextHandler();
+ context.setContextPath("/");
+ server.setHandler(context);
+
+ context.addServlet(new ServletHolder(new PrometheusServlet(this)), "/metrics");
+
+ try {
+ server.start();
+ log.info("Started Prometheus stats endpoint at {}", httpEndpoint);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ if (exposeDefaultJVMMetrics) {
+ // Include standard JVM stats
+ registerMetrics(new StandardExports());
+ registerMetrics(new MemoryPoolsExports());
+ registerMetrics(new GarbageCollectorExports());
+ registerMetrics(new ThreadExports());
+
+ // Add direct memory allocated through unsafe
+ registerMetrics(Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Child() {
+ @Override
+ public double get() {
+ return getDirectMemoryUsage.get();
+ }
+ }));
+
+ registerMetrics(Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Child() {
+ @Override
+ public double get() {
+ return PlatformDependent.estimateMaxDirectMemory();
+ }
+ }));
+ }
+
+ executor = Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("metrics"));
+
+ int latencyRolloverSeconds = conf.getInt(PROMETHEUS_STATS_LATENCY_ROLLOVER_SECONDS,
+ DEFAULT_PROMETHEUS_STATS_LATENCY_ROLLOVER_SECONDS);
+
+ executor.scheduleAtFixedRate(() -> {
+ rotateLatencyCollection();
+ }, 1, latencyRolloverSeconds, TimeUnit.SECONDS);
+
+ }
+
+ @Override
+ public void stop() {
+ if (server != null) {
+ try {
+ server.stop();
+ } catch (Exception e) {
+ log.warn("Failed to shutdown Jetty server", e);
+ } finally {
+ ThreadRegistry.clear();
+ }
+ }
+ }
+
+ @Override
+ public StatsLogger getStatsLogger(String scope) {
+ return new PrometheusStatsLogger(PrometheusMetricsProvider.this, scope, Collections.emptyMap());
+ }
+
+ @Override
+ public void writeAllMetrics(Writer writer) throws IOException {
+ PrometheusTextFormat prometheusTextFormat = new PrometheusTextFormat();
+ PrometheusTextFormat.writeMetricsCollectedByPrometheusClient(writer, registry);
+
+ gauges.forEach((sc, gauge) -> prometheusTextFormat.writeGauge(writer, sc.getScope(), gauge));
+ counters.forEach((sc, counter) -> prometheusTextFormat.writeCounter(writer, sc.getScope(), counter));
+ opStats.forEach((sc, opStatLogger) ->
+ prometheusTextFormat.writeOpStat(writer, sc.getScope(), opStatLogger));
+ }
+
+ @Override
+ public String getStatsName(String... statsComponents) {
+ String completeName;
+ if (statsComponents.length == 0) {
+ return "";
+ } else if (statsComponents[0].isEmpty()) {
+ completeName = StringUtils.join(statsComponents, '_', 1, statsComponents.length);
+ } else {
+ completeName = StringUtils.join(statsComponents, '_');
+ }
+ return Collector.sanitizeMetricName(completeName);
+ }
+
+ @VisibleForTesting
+ void rotateLatencyCollection() {
+ opStats.forEach((name, metric) -> {
+ metric.rotateLatencyCollection();
+ });
+ }
+
+ private void registerMetrics(Collector collector) {
+ try {
+ collector.register(registry);
+ } catch (Exception e) {
+ // Ignore if these were already registered
+ if (log.isDebugEnabled()) {
+ log.debug("Failed to register Prometheus collector exports", e);
+ }
+ }
+ }
+
+ private static final Logger log = LoggerFactory.getLogger(PrometheusMetricsProvider.class);
+
+ /*
+ * Try to get Netty counter of used direct memory. This will be correct, unlike the JVM values.
+ */
+ private static final AtomicLong directMemoryUsage;
+ private static final Optional poolMxBeanOp;
+ private static final Supplier getDirectMemoryUsage;
+
+ static {
+ if (PlatformDependent.useDirectBufferNoCleaner()) {
+ poolMxBeanOp = Optional.empty();
+ AtomicLong tmpDirectMemoryUsage = null;
+ try {
+ Field field = PlatformDependent.class.getDeclaredField("DIRECT_MEMORY_COUNTER");
+ field.setAccessible(true);
+ tmpDirectMemoryUsage = (AtomicLong) field.get(null);
+ } catch (Throwable t) {
+ log.warn("Failed to access netty DIRECT_MEMORY_COUNTER field {}", t.getMessage());
+ }
+ directMemoryUsage = tmpDirectMemoryUsage;
+ getDirectMemoryUsage = () -> directMemoryUsage != null ? directMemoryUsage.get() : Double.NaN;
+ } else {
+ directMemoryUsage = null;
+ List platformMXBeans = ManagementFactory.getPlatformMXBeans(BufferPoolMXBean.class);
+ poolMxBeanOp = platformMXBeans.stream()
+ .filter(bufferPoolMXBean -> bufferPoolMXBean.getName().equals("direct")).findAny();
+ getDirectMemoryUsage = () -> poolMxBeanOp.isPresent() ? poolMxBeanOp.get().getMemoryUsed() : Double.NaN;
+ }
+ }
+}
diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusServlet.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusServlet.java
new file mode 100644
index 0000000000000..88c572d8c4db3
--- /dev/null
+++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusServlet.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.metrics.prometheus.bookkeeper;
+
+import io.prometheus.client.exporter.common.TextFormat;
+import java.io.IOException;
+import java.io.Writer;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+/**
+ * Servlet used to export metrics in prometheus text format.
+ */
+public class PrometheusServlet extends HttpServlet {
+ private static final long serialVersionUID = 1L;
+
+ private final transient PrometheusMetricsProvider provider;
+
+ public PrometheusServlet(PrometheusMetricsProvider provider) {
+ this.provider = provider;
+ }
+
+ @Override
+ protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException {
+ resp.setStatus(HttpServletResponse.SC_OK);
+ resp.setContentType(TextFormat.CONTENT_TYPE_004);
+
+ Writer writer = resp.getWriter();
+ try {
+ provider.writeAllMetrics(writer);
+ writer.flush();
+ } finally {
+ writer.close();
+ }
+ }
+
+ @Override
+ protected void doPost(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException {
+ doGet(req, resp);
+ }
+
+}
diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusStatsLogger.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusStatsLogger.java
new file mode 100644
index 0000000000000..b3f8098590fe6
--- /dev/null
+++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusStatsLogger.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.metrics.prometheus.bookkeeper;
+
+import com.google.common.base.Joiner;
+import io.prometheus.client.Collector;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.Gauge;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+/**
+ * A {@code Prometheus} based {@link StatsLogger} implementation.
+ */
+public class PrometheusStatsLogger implements StatsLogger {
+
+ private final PrometheusMetricsProvider provider;
+ private final String scope;
+ private final Map labels;
+
+ PrometheusStatsLogger(PrometheusMetricsProvider provider, String scope, Map labels) {
+ this.provider = provider;
+ this.scope = scope;
+ this.labels = labels;
+ }
+
+ @Override
+ public OpStatsLogger getOpStatsLogger(String name) {
+ return provider.opStats.computeIfAbsent(scopeContext(name), x -> new DataSketchesOpStatsLogger(labels));
+ }
+
+ @Override
+ public OpStatsLogger getThreadScopedOpStatsLogger(String name) {
+ return provider.threadScopedOpStats.computeIfAbsent(scopeContext(name),
+ x -> new ThreadScopedDataSketchesStatsLogger(provider, x, labels));
+ }
+
+ @Override
+ public Counter getCounter(String name) {
+ return provider.counters.computeIfAbsent(scopeContext(name), x -> new LongAdderCounter(labels));
+ }
+
+ public Counter getThreadScopedCounter(String name) {
+ return provider.threadScopedCounters.computeIfAbsent(scopeContext(name),
+ x -> new ThreadScopedLongAdderCounter(provider, x, labels));
+ }
+
+ @Override
+ public void registerGauge(String name, Gauge gauge) {
+ provider.gauges.computeIfAbsent(scopeContext(name), x -> new SimpleGauge(gauge, labels));
+ }
+
+ @Override
+ public void unregisterGauge(String name, Gauge gauge) {
+ // no-op
+ }
+
+ @Override
+ public void removeScope(String name, StatsLogger statsLogger) {
+ // no-op
+ }
+
+ @Override
+ public StatsLogger scope(String name) {
+ return new PrometheusStatsLogger(provider, completeName(name), labels);
+ }
+
+ @Override
+ public StatsLogger scopeLabel(String labelName, String labelValue) {
+ Map newLabels = new TreeMap<>(labels);
+ newLabels.put(labelName, labelValue);
+ return new PrometheusStatsLogger(provider, scope, newLabels);
+ }
+
+ private ScopeContext scopeContext(String name) {
+ return new ScopeContext(completeName(name), labels);
+ }
+
+ private String completeName(String name) {
+ return Collector.sanitizeMetricName(scope.isEmpty() ? name : Joiner.on('_').join(scope, name));
+ }
+}
diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusTextFormat.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusTextFormat.java
new file mode 100644
index 0000000000000..97ad0ea8fbef2
--- /dev/null
+++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusTextFormat.java
@@ -0,0 +1,240 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.metrics.prometheus.bookkeeper;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.Collector.MetricFamilySamples;
+import io.prometheus.client.Collector.MetricFamilySamples.Sample;
+import io.prometheus.client.CollectorRegistry;
+import java.io.IOException;
+import java.io.Writer;
+import java.util.Enumeration;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Logic to write metrics in Prometheus text format.
+ */
+public class PrometheusTextFormat {
+
+ Set metricNameSet = new HashSet<>();
+
+ void writeGauge(Writer w, String name, SimpleGauge extends Number> gauge) {
+ // Example:
+ // # TYPE bookie_storage_entries_count gauge
+ // bookie_storage_entries_count 519
+ try {
+ writeType(w, name, "gauge");
+ w.append(name);
+ writeLabels(w, gauge.getLabels());
+ w.append(' ').append(gauge.getSample().toString()).append('\n');
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ void writeCounter(Writer w, String name, LongAdderCounter counter) {
+ // Example:
+ // # TYPE jvm_threads_started_total counter
+ // jvm_threads_started_total 59
+ try {
+ writeType(w, name, "counter");
+ w.append(name);
+ writeLabels(w, counter.getLabels());
+ w.append(' ').append(counter.get().toString()).append('\n');
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ void writeOpStat(Writer w, String name, DataSketchesOpStatsLogger opStat) {
+ // Example:
+ // # TYPE bookie_journal_JOURNAL_ADD_ENTRY summary
+ // bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.5",} NaN
+ // bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.75",} NaN
+ // bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.95",} NaN
+ // bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.99",} NaN
+ // bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.999",} NaN
+ // bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.9999",} NaN
+ // bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="1.0",} NaN
+ // bookie_journal_JOURNAL_ADD_ENTRY_count{success="false",} 0.0
+ // bookie_journal_JOURNAL_ADD_ENTRY_sum{success="false",} 0.0
+ // bookie_journal_JOURNAL_ADD_ENTRY{success="true",quantile="0.5",} 1.706
+ // bookie_journal_JOURNAL_ADD_ENTRY{success="true",quantile="0.75",} 1.89
+ // bookie_journal_JOURNAL_ADD_ENTRY{success="true",quantile="0.95",} 2.121
+ // bookie_journal_JOURNAL_ADD_ENTRY{success="true",quantile="0.99",} 10.708
+ // bookie_journal_JOURNAL_ADD_ENTRY{success="true",quantile="0.999",} 10.902
+ // bookie_journal_JOURNAL_ADD_ENTRY{success="true",quantile="0.9999",} 10.902
+ // bookie_journal_JOURNAL_ADD_ENTRY{success="true",quantile="1.0",} 10.902
+ // bookie_journal_JOURNAL_ADD_ENTRY_count{success="true",} 658.0
+ // bookie_journal_JOURNAL_ADD_ENTRY_sum{success="true",} 1265.0800000000002
+ try {
+ writeType(w, name, "summary");
+ writeQuantile(w, opStat, name, false, 0.5);
+ writeQuantile(w, opStat, name, false, 0.75);
+ writeQuantile(w, opStat, name, false, 0.95);
+ writeQuantile(w, opStat, name, false, 0.99);
+ writeQuantile(w, opStat, name, false, 0.999);
+ writeQuantile(w, opStat, name, false, 0.9999);
+ writeQuantile(w, opStat, name, false, 1.0);
+ writeCount(w, opStat, name, false);
+ writeSum(w, opStat, name, false);
+
+ writeQuantile(w, opStat, name, true, 0.5);
+ writeQuantile(w, opStat, name, true, 0.75);
+ writeQuantile(w, opStat, name, true, 0.95);
+ writeQuantile(w, opStat, name, true, 0.99);
+ writeQuantile(w, opStat, name, true, 0.999);
+ writeQuantile(w, opStat, name, true, 0.9999);
+ writeQuantile(w, opStat, name, true, 1.0);
+ writeCount(w, opStat, name, true);
+ writeSum(w, opStat, name, true);
+
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ private void writeLabels(Writer w, Map labels) throws IOException {
+ if (labels.isEmpty()) {
+ return;
+ }
+
+ w.append('{');
+ writeLabelsNoBraces(w, labels);
+ w.append('}');
+ }
+
+ private void writeLabelsNoBraces(Writer w, Map labels) throws IOException {
+ if (labels.isEmpty()) {
+ return;
+ }
+
+ boolean isFirst = true;
+ for (Map.Entry e : labels.entrySet()) {
+ if (!isFirst) {
+ w.append(',');
+ }
+ isFirst = false;
+ w.append(e.getKey())
+ .append("=\"")
+ .append(e.getValue())
+ .append('"');
+ }
+ }
+
+ private void writeQuantile(Writer w, DataSketchesOpStatsLogger opStat, String name, Boolean success,
+ double quantile) throws IOException {
+ w.append(name)
+ .append("{success=\"").append(success.toString())
+ .append("\",quantile=\"").append(Double.toString(quantile))
+ .append("\"");
+ if (!opStat.getLabels().isEmpty()) {
+ w.append(", ");
+ writeLabelsNoBraces(w, opStat.getLabels());
+ }
+ w.append("} ")
+ .append(Double.toString(opStat.getQuantileValue(success, quantile))).append('\n');
+ }
+
+ private void writeCount(Writer w, DataSketchesOpStatsLogger opStat, String name, Boolean success)
+ throws IOException {
+ w.append(name).append("_count{success=\"").append(success.toString()).append("\"");
+ if (!opStat.getLabels().isEmpty()) {
+ w.append(", ");
+ writeLabelsNoBraces(w, opStat.getLabels());
+ }
+ w.append("} ")
+ .append(Long.toString(opStat.getCount(success))).append('\n');
+ }
+
+ private void writeSum(Writer w, DataSketchesOpStatsLogger opStat, String name, Boolean success)
+ throws IOException {
+ w.append(name).append("_sum{success=\"").append(success.toString()).append("\"");
+ if (!opStat.getLabels().isEmpty()) {
+ w.append(", ");
+ writeLabelsNoBraces(w, opStat.getLabels());
+ }
+ w.append("} ")
+ .append(Double.toString(opStat.getSum(success))).append('\n');
+ }
+
+ static void writeMetricsCollectedByPrometheusClient(Writer w, CollectorRegistry registry) throws IOException {
+ Enumeration metricFamilySamples = registry.metricFamilySamples();
+ while (metricFamilySamples.hasMoreElements()) {
+ MetricFamilySamples metricFamily = metricFamilySamples.nextElement();
+ // Write type of metric
+ w.append("# TYPE ").append(metricFamily.name).append(getTypeNameSuffix(metricFamily.type)).append(' ')
+ .append(getTypeStr(metricFamily.type)).write('\n');
+
+ for (int i = 0; i < metricFamily.samples.size(); i++) {
+ Sample sample = metricFamily.samples.get(i);
+ w.write(sample.name);
+ w.write('{');
+ for (int j = 0; j < sample.labelNames.size(); j++) {
+ if (j != 0) {
+ w.write(", ");
+ }
+ w.write(sample.labelNames.get(j));
+ w.write("=\"");
+ w.write(sample.labelValues.get(j));
+ w.write('"');
+ }
+
+ w.write("} ");
+ w.write(Collector.doubleToGoString(sample.value));
+ w.write('\n');
+ }
+ }
+ }
+
+ static String getTypeNameSuffix(Collector.Type type) {
+ if (type.equals(Collector.Type.INFO)) {
+ return "_info";
+ }
+ return "";
+ }
+
+ static String getTypeStr(Collector.Type type) {
+ switch (type) {
+ case COUNTER:
+ return "counter";
+ case GAUGE:
+ case INFO:
+ return "gauge";
+ case SUMMARY:
+ return "summary";
+ case HISTOGRAM:
+ return "histogram";
+ case UNKNOWN:
+ default:
+ return "unknown";
+ }
+ }
+
+ void writeType(Writer w, String name, String type) throws IOException {
+ if (metricNameSet.contains(name)) {
+ return;
+ }
+ metricNameSet.add(name);
+ w.append("# TYPE ").append(name).append(" ").append(type).append("\n");
+ }
+
+}
diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/ScopeContext.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/ScopeContext.java
new file mode 100644
index 0000000000000..4f15c006569ef
--- /dev/null
+++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/ScopeContext.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.metrics.prometheus.bookkeeper;
+
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Holder for a scope and a set of associated labels.
+ */
+public class ScopeContext {
+ private final String scope;
+ private final Map labels;
+
+ public ScopeContext(String scope, Map labels) {
+ this.scope = scope;
+ this.labels = labels;
+ }
+
+ public String getScope() {
+ return scope;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ ScopeContext that = (ScopeContext) o;
+ return Objects.equals(scope, that.scope) && Objects.equals(labels, that.labels);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(scope, labels);
+ }
+}
diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/SimpleGauge.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/SimpleGauge.java
new file mode 100644
index 0000000000000..384cc682c2ff3
--- /dev/null
+++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/SimpleGauge.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.metrics.prometheus.bookkeeper;
+
+import java.util.Map;
+import org.apache.bookkeeper.stats.Gauge;
+
+/**
+ * A {@link Gauge} implementation that forwards on the value supplier.
+ */
+public class SimpleGauge {
+
+ private final Map labels;
+ private final Gauge gauge;
+
+ public SimpleGauge(final Gauge gauge, Map labels) {
+ this.gauge = gauge;
+ this.labels = labels;
+ }
+
+ Number getSample() {
+ return gauge.getSample();
+ }
+
+ public Map getLabels() {
+ return labels;
+ }
+}
diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/ThreadScopedDataSketchesStatsLogger.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/ThreadScopedDataSketchesStatsLogger.java
new file mode 100644
index 0000000000000..4d3d5662d9c9b
--- /dev/null
+++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/ThreadScopedDataSketchesStatsLogger.java
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.metrics.prometheus.bookkeeper;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.stats.OpStatsData;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.ThreadRegistry;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * OpStatsLogger implementation that lazily registers OpStatsLoggers per thread
+ * with added labels for the threadpool/thresd name and thread no.
+ */
+public class ThreadScopedDataSketchesStatsLogger implements OpStatsLogger {
+
+ private static Logger logger = LoggerFactory.getLogger(ThreadScopedDataSketchesStatsLogger.class);
+
+ private ThreadLocal statsLoggers;
+ private DataSketchesOpStatsLogger defaultStatsLogger;
+ private Map originalLabels;
+ private ScopeContext scopeContext;
+ private PrometheusMetricsProvider provider;
+
+ public ThreadScopedDataSketchesStatsLogger(PrometheusMetricsProvider provider,
+ ScopeContext scopeContext,
+ Map labels) {
+ this.provider = provider;
+ this.scopeContext = scopeContext;
+ this.originalLabels = labels;
+ this.defaultStatsLogger = new DataSketchesOpStatsLogger(labels);
+
+ Map defaultLabels = new HashMap<>(labels);
+ defaultLabels.put("threadPool", "?");
+ defaultLabels.put("thread", "?");
+ this.defaultStatsLogger.initializeThread(defaultLabels);
+
+ this.statsLoggers = ThreadLocal.withInitial(() -> {
+ return new DataSketchesOpStatsLogger(labels);
+ });
+ }
+
+ @Override
+ public void registerFailedEvent(long eventLatency, TimeUnit unit) {
+ getStatsLogger().registerFailedEvent(eventLatency, unit);
+ }
+
+ @Override
+ public void registerSuccessfulEvent(long eventLatency, TimeUnit unit) {
+ getStatsLogger().registerSuccessfulEvent(eventLatency, unit);
+ }
+
+ @Override
+ public void registerSuccessfulValue(long value) {
+ getStatsLogger().registerSuccessfulValue(value);
+ }
+
+ @Override
+ public void registerFailedValue(long value) {
+ getStatsLogger().registerFailedValue(value);
+ }
+
+ @Override
+ public OpStatsData toOpStatsData() {
+ // Not relevant as we don't use JMX here
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void clear() {
+ // Not relevant as we don't use JMX here
+ throw new UnsupportedOperationException();
+ }
+
+ private DataSketchesOpStatsLogger getStatsLogger() {
+ DataSketchesOpStatsLogger statsLogger = statsLoggers.get();
+
+ // Lazy registration
+ // Update the stats logger with the thread labels then add to the provider
+ // If for some reason this thread did not get registered,
+ // then we fallback to a standard OpsStatsLogger (defaultStatsLogger)
+ if (!statsLogger.isThreadInitialized()) {
+ ThreadRegistry.ThreadPoolThread tpt = ThreadRegistry.get();
+ if (tpt == null) {
+ logger.warn("Thread {} was not registered in the thread registry. Using default stats logger {}.",
+ Thread.currentThread(), defaultStatsLogger);
+ statsLoggers.set(defaultStatsLogger);
+ DataSketchesOpStatsLogger previous = provider.opStats
+ .put(new ScopeContext(scopeContext.getScope(), originalLabels), defaultStatsLogger);
+ // If we overwrite a logger, metrics will not be collected correctly
+ if (previous != null && previous != defaultStatsLogger) {
+ logger.error("Invalid state for thead " + Thread.currentThread() + ". Overwrote a stats logger."
+ + "New is {}, previous was {}",
+ defaultStatsLogger, previous);
+ throw new IllegalStateException("Invalid state. Overwrote a stats logger.");
+ }
+ return defaultStatsLogger;
+ } else {
+ Map threadScopedlabels = new HashMap<>(originalLabels);
+ threadScopedlabels.put("threadPool", tpt.getThreadPool());
+ threadScopedlabels.put("thread", String.valueOf(tpt.getOrdinal()));
+
+ statsLogger.initializeThread(threadScopedlabels);
+ DataSketchesOpStatsLogger previous = provider.opStats
+ .put(new ScopeContext(scopeContext.getScope(), threadScopedlabels), statsLogger);
+ // If we overwrite a logger, metrics will not be collected correctly
+ if (previous != null && previous != statsLogger) {
+ logger.error("Invalid state for thead " + Thread.currentThread() + ". Overwrote a stats logger."
+ + "New is {}, previous was {}",
+ defaultStatsLogger, previous);
+ throw new IllegalStateException("Invalid state. Overwrote a stats logger.");
+ }
+ }
+ }
+
+ return statsLogger;
+ }
+}
\ No newline at end of file
diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/ThreadScopedLongAdderCounter.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/ThreadScopedLongAdderCounter.java
new file mode 100644
index 0000000000000..1c2287f5b9c85
--- /dev/null
+++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/ThreadScopedLongAdderCounter.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.metrics.prometheus.bookkeeper;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.ThreadRegistry;
+
+/**
+ * {@link Counter} implementation that lazily registers LongAdderCounters per thread
+ * * with added labels for the threadpool/thread name and thread no.
+ */
+public class ThreadScopedLongAdderCounter implements Counter {
+ private ThreadLocal counters;
+ private LongAdderCounter defaultCounter;
+ private Map originalLabels;
+ private ScopeContext scopeContext;
+ private PrometheusMetricsProvider provider;
+
+ public ThreadScopedLongAdderCounter(PrometheusMetricsProvider provider,
+ ScopeContext scopeContext,
+ Map labels) {
+ this.provider = provider;
+ this.scopeContext = scopeContext;
+ this.originalLabels = new HashMap<>(labels);
+ this.defaultCounter = new LongAdderCounter(labels);
+ Map defaultLabels = new HashMap<>(labels);
+ defaultLabels.put("threadPool", "?");
+ defaultLabels.put("thread", "?");
+ this.defaultCounter.initializeThread(defaultLabels);
+
+ this.counters = ThreadLocal.withInitial(() -> {
+ return new LongAdderCounter(labels);
+ });
+ }
+
+ @Override
+ public void clear() {
+ getCounter().clear();
+ }
+
+ @Override
+ public void inc() {
+ getCounter().inc();
+ }
+
+ @Override
+ public void dec() {
+ getCounter().dec();
+ }
+
+ @Override
+ public void addCount(long delta) {
+ getCounter().addCount(delta);
+ }
+
+ @Override
+ public void addLatency(long eventLatency, TimeUnit unit) {
+ getCounter().addLatency(eventLatency, unit);
+ }
+
+ @Override
+ public Long get() {
+ return getCounter().get();
+ }
+
+ private LongAdderCounter getCounter() {
+ LongAdderCounter counter = counters.get();
+
+ // Lazy registration
+ // Update the counter with the thread labels then add to the provider
+ // If for some reason this thread did not get registered,
+ // then we fallback to a standard counter (defaultCounter)
+ if (!counter.isThreadInitialized()) {
+ ThreadRegistry.ThreadPoolThread tpt = ThreadRegistry.get();
+
+ if (tpt == null) {
+ counters.set(defaultCounter);
+ provider.counters.put(new ScopeContext(scopeContext.getScope(), originalLabels), defaultCounter);
+ return defaultCounter;
+ } else {
+ Map threadScopedlabels = new HashMap<>(originalLabels);
+ threadScopedlabels.put("threadPool", tpt.getThreadPool());
+ threadScopedlabels.put("thread", String.valueOf(tpt.getOrdinal()));
+
+ counter.initializeThread(threadScopedlabels);
+ provider.counters.put(new ScopeContext(scopeContext.getScope(), threadScopedlabels), counter);
+ }
+ }
+
+ return counter;
+ }
+}
diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/package-info.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/package-info.java
new file mode 100644
index 0000000000000..fd5cf0e154a06
--- /dev/null
+++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/**
+ * A stats provider implementation for BookKeeper that uses Jetty 12.
+ */
+package org.apache.pulsar.metrics.prometheus.bookkeeper;
diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/test/java/org/apache/pulsar/metrics/prometheus/bookkeeper/TestPrometheusFormatter.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/test/java/org/apache/pulsar/metrics/prometheus/bookkeeper/TestPrometheusFormatter.java
new file mode 100644
index 0000000000000..fceee727270a6
--- /dev/null
+++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/test/java/org/apache/pulsar/metrics/prometheus/bookkeeper/TestPrometheusFormatter.java
@@ -0,0 +1,296 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.metrics.prometheus.bookkeeper;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.testng.AssertJUnit.assertEquals;
+import static org.testng.AssertJUnit.assertTrue;
+import static org.testng.AssertJUnit.fail;
+import com.google.common.base.MoreObjects;
+import com.google.common.base.Splitter;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Gauge;
+import io.prometheus.client.hotspot.GarbageCollectorExports;
+import io.prometheus.client.hotspot.MemoryPoolsExports;
+import io.prometheus.client.hotspot.StandardExports;
+import io.prometheus.client.hotspot.ThreadExports;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.testng.annotations.Test;
+
+/**
+ * Test for {@link PrometheusMetricsProvider}.
+ */
+public class TestPrometheusFormatter {
+
+ @Test(timeOut = 30000)
+ public void testStatsOutput() throws Exception {
+ PrometheusMetricsProvider provider = new PrometheusMetricsProvider();
+ StatsLogger statsLogger = provider.getStatsLogger("test");
+ Counter counter = statsLogger.getCounter("my_counter");
+
+ counter.inc();
+ counter.inc();
+
+ OpStatsLogger opStats = statsLogger.getOpStatsLogger("op");
+ opStats.registerSuccessfulEvent(10, TimeUnit.MILLISECONDS);
+ opStats.registerSuccessfulEvent(5, TimeUnit.MILLISECONDS);
+
+ OpStatsLogger opStats1 = statsLogger.scopeLabel("test_label", "test_value")
+ .getOpStatsLogger("op_label");
+ opStats1.registerSuccessfulEvent(10, TimeUnit.MILLISECONDS);
+ opStats1.registerSuccessfulEvent(5, TimeUnit.MILLISECONDS);
+ opStats1.registerFailedEvent(1, TimeUnit.MILLISECONDS);
+
+ provider.rotateLatencyCollection();
+
+ StringWriter writer = new StringWriter();
+ provider.writeAllMetrics(writer);
+
+ writer.write("jvm_memory_direct_bytes_max{} 4.77626368E8\n");
+ writer.write("jvm_memory_pool_bytes_used{pool=\"Code Cache\"} 3347712.0\n");
+ writer.write("jvm_memory_pool_bytes_used{pool=\"CodeHeap 'non-nmethods'\"} 1207168.0\n");
+ System.out.println(writer);
+ Multimap metrics = parseMetrics(writer.toString());
+ System.out.println(metrics);
+
+ List cm = (List) metrics.get("test_my_counter");
+ assertEquals(1, cm.size());
+ assertEquals(0, cm.get(0).tags.size());
+ assertEquals(2.0, cm.get(0).value, 0.0);
+
+ // test_op_sum
+ cm = (List) metrics.get("test_op_sum");
+ assertEquals(2, cm.size());
+ Metric m = cm.get(0);
+ assertEquals(1, cm.get(0).tags.size());
+ assertEquals(0.0, m.value, 0.0);
+ assertEquals(1, m.tags.size());
+ assertEquals("false", m.tags.get("success"));
+
+ m = cm.get(1);
+ assertEquals(1, cm.get(0).tags.size());
+ assertEquals(15.0, m.value, 0.0);
+ assertEquals(1, m.tags.size());
+ assertEquals("true", m.tags.get("success"));
+
+ // test_op_count
+ cm = (List) metrics.get("test_op_count");
+ assertEquals(2, cm.size());
+ m = cm.get(0);
+ assertEquals(1, cm.get(0).tags.size());
+ assertEquals(0.0, m.value, 0.0);
+ assertEquals(1, m.tags.size());
+ assertEquals("false", m.tags.get("success"));
+
+ m = cm.get(1);
+ assertEquals(1, cm.get(0).tags.size());
+ assertEquals(2.0, m.value, 0.0);
+ assertEquals(1, m.tags.size());
+ assertEquals("true", m.tags.get("success"));
+
+ // Latency
+ cm = (List) metrics.get("test_op");
+ assertEquals(14, cm.size());
+
+ boolean found = false;
+ for (Metric mt : cm) {
+ if ("true".equals(mt.tags.get("success")) && "1.0".equals(mt.tags.get("quantile"))) {
+ assertEquals(10.0, mt.value, 0.0);
+ found = true;
+ }
+ }
+
+ assertTrue(found);
+
+ // test_op_label_sum
+ cm = (List) metrics.get("test_op_label_sum");
+ assertEquals(2, cm.size());
+ m = cm.get(0);
+ assertEquals(2, m.tags.size());
+ assertEquals(1.0, m.value, 0.0);
+ assertEquals("false", m.tags.get("success"));
+ assertEquals("test_value", m.tags.get("test_label"));
+
+ m = cm.get(1);
+ assertEquals(15.0, m.value, 0.0);
+ assertEquals(2, m.tags.size());
+ assertEquals("true", m.tags.get("success"));
+ assertEquals("test_value", m.tags.get("test_label"));
+
+ // test_op_label_count
+ cm = (List) metrics.get("test_op_label_count");
+ assertEquals(2, cm.size());
+ m = cm.get(0);
+ assertEquals(1, m.value, 0.0);
+ assertEquals(2, m.tags.size());
+ assertEquals("false", m.tags.get("success"));
+ assertEquals("test_value", m.tags.get("test_label"));
+
+ m = cm.get(1);
+ assertEquals(2.0, m.value, 0.0);
+ assertEquals(2, m.tags.size());
+ assertEquals("true", m.tags.get("success"));
+ assertEquals("test_value", m.tags.get("test_label"));
+
+ // Latency
+ cm = (List) metrics.get("test_op_label");
+ assertEquals(14, cm.size());
+
+ found = false;
+ for (Metric mt : cm) {
+ if ("true".equals(mt.tags.get("success"))
+ && "test_value".equals(mt.tags.get("test_label"))
+ && "1.0".equals(mt.tags.get("quantile"))) {
+ assertEquals(10.0, mt.value, 0.0);
+ found = true;
+ }
+ }
+
+ assertTrue(found);
+ }
+
+ @Test
+ public void testWriteMetricsCollectedByPrometheusClient() {
+ CollectorRegistry registry = CollectorRegistry.defaultRegistry;
+ registry.register(new StandardExports());
+ registry.register(new MemoryPoolsExports());
+ registry.register(new GarbageCollectorExports());
+ registry.register(new ThreadExports());
+ registry.register(Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Gauge.Child() {
+ @Override
+ public double get() {
+ return 1.0;
+ }
+ }));
+ registry.register(Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Gauge.Child() {
+ @Override
+ public double get() {
+ return 100.0;
+ }
+ }));
+ PrometheusMetricsProvider provider = new PrometheusMetricsProvider(registry);
+ StringWriter writer = new StringWriter();
+ try {
+ provider.rotateLatencyCollection();
+ provider.writeAllMetrics(writer);
+ String output = writer.toString();
+ parseMetrics(output);
+ assertTrue(output.contains("# TYPE jvm_memory_direct_bytes_max gauge"));
+ assertTrue(output.contains("# TYPE jvm_memory_direct_bytes_used gauge"));
+ assertTrue(output.contains("# TYPE jvm_gc_collection_seconds summary"));
+ assertTrue(output.contains("# TYPE jvm_memory_pool_bytes_committed gauge"));
+ assertTrue(output.contains("# TYPE process_cpu_seconds counter"));
+ } catch (Exception e) {
+ fail();
+ }
+
+ }
+
+ @Test
+ public void testPrometheusTypeDuplicate() throws IOException {
+ PrometheusTextFormat prometheusTextFormat = new PrometheusTextFormat();
+ StringWriter writer = new StringWriter();
+ prometheusTextFormat.writeType(writer, "counter", "gauge");
+ prometheusTextFormat.writeType(writer, "counter", "gauge");
+ String string = writer.toString();
+ assertEquals("# TYPE counter gauge\n", string);
+ }
+
+
+ /**
+ * Hacky parsing of Prometheus text format. Sould be good enough for unit tests
+ */
+ private static Multimap parseMetrics(String metrics) {
+ Multimap parsed = ArrayListMultimap.create();
+
+ // Example of lines are
+ // jvm_threads_current{cluster="standalone",} 203.0
+ // or
+ // pulsar_subscriptions_count{cluster="standalone", namespace="sample/standalone/ns1",
+ // topic="persistent://sample/standalone/ns1/test-2"} 0.0 1517945780897
+ Pattern pattern = Pattern.compile("^(\\w+)(\\{([^\\}]*)\\})?\\s(-?[\\d\\w\\.]+)(\\s(\\d+))?$");
+ Pattern formatPattern =
+ Pattern.compile("^(\\w+)(\\{((\\w+=[-\\s\\\'\\\"\\.\\w]+(,\\s?\\w+=[\\\"\\.\\w]+)*))?\\})?"
+ + "\\s(-?[\\d\\w\\.]+)(\\s(\\d+))?$");
+ Pattern tagsPattern = Pattern.compile("(\\w+)=\"([^\"]+)\"(,\\s?)?");
+
+ Splitter.on("\n").split(metrics).forEach(line -> {
+ if (line.isEmpty() || line.startsWith("#")) {
+ return;
+ }
+
+ System.err.println("LINE: '" + line + "'");
+ Matcher matcher = pattern.matcher(line);
+ Matcher formatMatcher = formatPattern.matcher(line);
+ System.err.println("Matches: " + matcher.matches());
+ System.err.println(matcher);
+ assertTrue(matcher.matches());
+ assertTrue("failed to validate line: " + line, formatMatcher.matches());
+
+ assertEquals(6, matcher.groupCount());
+ System.err.println("groups: " + matcher.groupCount());
+ for (int i = 0; i < matcher.groupCount(); i++) {
+ System.err.println(" GROUP " + i + " -- " + matcher.group(i));
+ }
+
+ checkArgument(matcher.matches());
+ checkArgument(formatMatcher.matches());
+ String name = matcher.group(1);
+
+ Metric m = new Metric();
+ m.value = Double.parseDouble(matcher.group(4));
+
+ String tags = matcher.group(3);
+ if (tags != null) {
+ Matcher tagsMatcher = tagsPattern.matcher(tags);
+ while (tagsMatcher.find()) {
+ String tag = tagsMatcher.group(1);
+ String value = tagsMatcher.group(2);
+ m.tags.put(tag, value);
+ }
+ }
+
+ parsed.put(name, m);
+ });
+
+ return parsed;
+ }
+
+ static class Metric {
+ Map tags = new TreeMap<>();
+ double value;
+
+ @Override
+ public String toString() {
+ return MoreObjects.toStringHelper(this).add("tags", tags).add("value", value).toString();
+ }
+ }
+}
diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/test/java/org/apache/pulsar/metrics/prometheus/bookkeeper/TestPrometheusMetricsProvider.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/test/java/org/apache/pulsar/metrics/prometheus/bookkeeper/TestPrometheusMetricsProvider.java
new file mode 100644
index 0000000000000..a92616062c3c7
--- /dev/null
+++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/test/java/org/apache/pulsar/metrics/prometheus/bookkeeper/TestPrometheusMetricsProvider.java
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.metrics.prometheus.bookkeeper;
+
+import static org.testng.Assert.assertNotEquals;
+import static org.testng.AssertJUnit.assertEquals;
+import static org.testng.AssertJUnit.assertNotNull;
+import static org.testng.AssertJUnit.assertNull;
+import static org.testng.AssertJUnit.assertSame;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import java.io.StringWriter;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.concurrent.TimeUnit;
+import lombok.Cleanup;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.commons.configuration2.PropertiesConfiguration;
+import org.testng.annotations.Test;
+
+/**
+ * Unit test of {@link PrometheusMetricsProvider}.
+ */
+public class TestPrometheusMetricsProvider {
+
+ @Test
+ public void testStartNoHttp() {
+ PropertiesConfiguration config = new PropertiesConfiguration();
+ config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_ENABLE, false);
+ PrometheusMetricsProvider provider = new PrometheusMetricsProvider();
+ try {
+ provider.start(config);
+ assertNull(provider.server);
+ } finally {
+ provider.stop();
+ }
+ }
+
+ @Test
+ public void testStartNoHttpWhenBkHttpEnabled() {
+ PropertiesConfiguration config = new PropertiesConfiguration();
+ config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_ENABLE, true);
+ config.setProperty("httpServerEnabled", true);
+ @Cleanup("stop") PrometheusMetricsProvider provider = new PrometheusMetricsProvider();
+ provider.start(config);
+ assertNull(provider.server);
+ }
+
+ @Test
+ public void testStartWithHttp() {
+ PropertiesConfiguration config = new PropertiesConfiguration();
+ config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_ENABLE, true);
+ config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_PORT, 0); // ephemeral
+ PrometheusMetricsProvider provider = new PrometheusMetricsProvider();
+ try {
+ provider.start(config);
+ assertNotNull(provider.server);
+ } finally {
+ provider.stop();
+ }
+ }
+
+ @Test
+ public void testStartWithHttpSpecifyAddr() {
+ PropertiesConfiguration config = new PropertiesConfiguration();
+ config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_ENABLE, true);
+ config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_PORT, 0); // ephemeral
+ config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_ADDRESS, "127.0.0.1");
+ PrometheusMetricsProvider provider = new PrometheusMetricsProvider();
+ try {
+ provider.start(config);
+ assertNotNull(provider.server);
+ } finally {
+ provider.stop();
+ }
+ }
+
+ @Test
+ public void testCounter() {
+ LongAdderCounter counter = new LongAdderCounter(Collections.emptyMap());
+ long value = counter.get();
+ assertEquals(0L, value);
+ counter.inc();
+ assertEquals(1L, counter.get().longValue());
+ counter.dec();
+ assertEquals(0L, counter.get().longValue());
+ counter.addCount(3);
+ assertEquals(3L, counter.get().longValue());
+ }
+
+ @Test
+ public void testCounter2() {
+ LongAdderCounter counter = new LongAdderCounter(Collections.emptyMap());
+ long value = counter.get();
+ assertEquals(0L, value);
+ counter.addLatency(3 * 1000 * 1000L, TimeUnit.NANOSECONDS);
+ assertEquals(3L, counter.get().longValue());
+ }
+
+ @Test
+ public void testTwoCounters() throws Exception {
+ PrometheusMetricsProvider provider = new PrometheusMetricsProvider();
+ StatsLogger statsLogger = provider.getStatsLogger("test");
+
+ Counter counter1 = statsLogger.getCounter("counter");
+ Counter counter2 = statsLogger.getCounter("counter");
+ assertEquals(counter1, counter2);
+ assertSame(counter1, counter2);
+
+ assertEquals(1, provider.counters.size());
+ }
+
+ @Test
+ public void testJvmDirectMemoryMetrics() throws Exception {
+ PropertiesConfiguration config = new PropertiesConfiguration();
+ config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_ENABLE, true);
+ config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_PORT, 0);
+ config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_ADDRESS, "127.0.0.1");
+ ByteBuf byteBuf = ByteBufAllocator.DEFAULT.directBuffer(25);
+ PrometheusMetricsProvider provider = new PrometheusMetricsProvider();
+ try {
+ provider.start(config);
+ assertNotNull(provider.server);
+ StringWriter writer = new StringWriter();
+ provider.writeAllMetrics(writer);
+ String s = writer.toString();
+ String[] split = s.split(System.lineSeparator());
+ HashMap map = new HashMap<>();
+ for (String str : split) {
+ String[] aux = str.split(" ");
+ map.put(aux[0], aux[1]);
+ }
+ String directBytesMax = map.get("jvm_memory_direct_bytes_max{}");
+ assertNotNull(directBytesMax);
+ assertNotEquals("Nan", directBytesMax);
+ assertNotEquals("-1", directBytesMax);
+ String directBytesUsed = map.get("jvm_memory_direct_bytes_used{}");
+ assertNotNull(directBytesUsed);
+ assertNotEquals("Nan", directBytesUsed);
+ // this condition is flaky
+ //assertTrue(Double.parseDouble(directBytesUsed) > 25);
+ // ensure byteBuffer doesn't gc
+ byteBuf.release();
+ } finally {
+ provider.stop();
+ }
+ }
+
+}
diff --git a/jetty-upgrade/pom.xml b/jetty-upgrade/pom.xml
new file mode 100644
index 0000000000000..6c35a3b41e22c
--- /dev/null
+++ b/jetty-upgrade/pom.xml
@@ -0,0 +1,39 @@
+
+
+ 4.0.0
+ pom
+
+ org.apache.pulsar
+ pulsar
+ 4.2.0-SNAPSHOT
+
+
+ jetty-upgrade
+ Pulsar Jetty Upgrade :: Parent
+
+
+ bookkeeper-prometheus-metrics-provider
+ zookeeper-prometheus-metrics
+ zookeeper-with-patched-admin
+
+
diff --git a/jetty-upgrade/zookeeper-prometheus-metrics/pom.xml b/jetty-upgrade/zookeeper-prometheus-metrics/pom.xml
new file mode 100755
index 0000000000000..6adc8f628be27
--- /dev/null
+++ b/jetty-upgrade/zookeeper-prometheus-metrics/pom.xml
@@ -0,0 +1,94 @@
+
+
+ 4.0.0
+
+ org.apache.pulsar
+ jetty-upgrade
+ 4.2.0-SNAPSHOT
+
+
+ pulsar-zookeeper-prometheus-metrics
+ jar
+ Apache Pulsar :: Apache ZooKeeper Prometheus Metrics Provider
+ ZooKeeper Prometheus.io Metrics Provider implementation for Jetty 12
+
+
+
+ org.apache.zookeeper
+ zookeeper
+ ${zookeeper.version}
+ provided
+
+
+ io.prometheus
+ simpleclient
+
+
+ io.prometheus
+ simpleclient_hotspot
+
+
+ io.prometheus
+ simpleclient_servlet
+
+
+ org.eclipse.jetty
+ jetty-server
+ provided
+
+
+ org.eclipse.jetty.ee8
+ jetty-ee8-servlet
+ provided
+
+
+ org.mockito
+ mockito-core
+ test
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-dependency-plugin
+
+
+ copy-dependencies
+ package
+
+ copy-dependencies
+
+
+ ${project.build.directory}/lib
+ false
+ true
+ false
+
+
+
+
+
+
+
diff --git a/jetty-upgrade/zookeeper-prometheus-metrics/src/main/java/org/apache/pulsar/metrics/prometheus/zookeeper/PrometheusMetricsProvider.java b/jetty-upgrade/zookeeper-prometheus-metrics/src/main/java/org/apache/pulsar/metrics/prometheus/zookeeper/PrometheusMetricsProvider.java
new file mode 100644
index 0000000000000..7b1999004caf1
--- /dev/null
+++ b/jetty-upgrade/zookeeper-prometheus-metrics/src/main/java/org/apache/pulsar/metrics/prometheus/zookeeper/PrometheusMetricsProvider.java
@@ -0,0 +1,632 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.metrics.prometheus.zookeeper;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.exporter.MetricsServlet;
+import io.prometheus.client.hotspot.DefaultExports;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Enumeration;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.BiConsumer;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.zookeeper.metrics.Counter;
+import org.apache.zookeeper.metrics.CounterSet;
+import org.apache.zookeeper.metrics.Gauge;
+import org.apache.zookeeper.metrics.GaugeSet;
+import org.apache.zookeeper.metrics.MetricsContext;
+import org.apache.zookeeper.metrics.MetricsProvider;
+import org.apache.zookeeper.metrics.MetricsProviderLifeCycleException;
+import org.apache.zookeeper.metrics.Summary;
+import org.apache.zookeeper.metrics.SummarySet;
+import org.apache.zookeeper.server.RateLogger;
+import org.eclipse.jetty.ee8.nested.ServletConstraint;
+import org.eclipse.jetty.ee8.security.ConstraintMapping;
+import org.eclipse.jetty.ee8.security.ConstraintSecurityHandler;
+import org.eclipse.jetty.ee8.servlet.ServletContextHandler;
+import org.eclipse.jetty.ee8.servlet.ServletHolder;
+import org.eclipse.jetty.server.Server;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A Metrics Provider implementation based on https://prometheus.io.
+ *
+ * @since 3.6.0
+ */
+public class PrometheusMetricsProvider implements MetricsProvider {
+
+ private static final Logger LOG = LoggerFactory.getLogger(PrometheusMetricsProvider.class);
+ private static final String LABEL = "key";
+ private static final String[] LABELS = {LABEL};
+
+ /**
+ * Number of worker threads for reporting Prometheus summary metrics.
+ * Default value is 1.
+ * If the number is less than 1, the main thread will be used.
+ */
+ static final String NUM_WORKER_THREADS = "numWorkerThreads";
+
+ /**
+ * The max queue size for Prometheus summary metrics reporting task.
+ * Default value is 1000000.
+ */
+ static final String MAX_QUEUE_SIZE = "maxQueueSize";
+
+ /**
+ * The timeout in ms for Prometheus worker threads shutdown.
+ * Default value is 1000ms.
+ */
+ static final String WORKER_SHUTDOWN_TIMEOUT_MS = "workerShutdownTimeoutMs";
+
+ /**
+ * We are using the 'defaultRegistry'.
+ *
+ * When you are running ZooKeeper (server or client) together with other
+ * libraries every metrics will be expected as a single view.
+ *
+ */
+ private final CollectorRegistry collectorRegistry = CollectorRegistry.defaultRegistry;
+ private final RateLogger rateLogger = new RateLogger(LOG, 60 * 1000);
+ private String host = "0.0.0.0";
+ private int port = 7000;
+ private boolean exportJvmInfo = true;
+ private Server server;
+ private final MetricsServletImpl servlet = new MetricsServletImpl();
+ private final Context rootContext = new Context();
+ private int numWorkerThreads = 1;
+ private int maxQueueSize = 1000000;
+ private long workerShutdownTimeoutMs = 1000;
+ private Optional executorOptional = Optional.empty();
+
+ @Override
+ public void configure(Properties configuration) throws MetricsProviderLifeCycleException {
+ LOG.info("Initializing metrics, configuration: {}", configuration);
+ this.host = configuration.getProperty("httpHost", "0.0.0.0");
+ this.port = Integer.parseInt(configuration.getProperty("httpPort", "7000"));
+ this.exportJvmInfo = Boolean.parseBoolean(configuration.getProperty("exportJvmInfo", "true"));
+ this.numWorkerThreads = Integer.parseInt(
+ configuration.getProperty(NUM_WORKER_THREADS, "1"));
+ this.maxQueueSize = Integer.parseInt(
+ configuration.getProperty(MAX_QUEUE_SIZE, "1000000"));
+ this.workerShutdownTimeoutMs = Long.parseLong(
+ configuration.getProperty(WORKER_SHUTDOWN_TIMEOUT_MS, "1000"));
+ }
+
+ @Override
+ public void start() throws MetricsProviderLifeCycleException {
+ this.executorOptional = createExecutor();
+ try {
+ LOG.info("Starting /metrics HTTP endpoint at host: {}, port: {}, exportJvmInfo: {}",
+ host, port, exportJvmInfo);
+ if (exportJvmInfo) {
+ DefaultExports.initialize();
+ }
+ server = new Server(new InetSocketAddress(host, port));
+ ServletContextHandler context = new ServletContextHandler();
+ context.setContextPath("/");
+ constrainTraceMethod(context);
+ server.setHandler(context);
+ context.addServlet(new ServletHolder(servlet), "/metrics");
+ server.start();
+ } catch (Exception err) {
+ LOG.error("Cannot start /metrics server", err);
+ if (server != null) {
+ try {
+ server.stop();
+ } catch (Exception suppressed) {
+ err.addSuppressed(suppressed);
+ } finally {
+ server = null;
+ }
+ }
+ throw new MetricsProviderLifeCycleException(err);
+ }
+ }
+
+ // for tests
+ MetricsServletImpl getServlet() {
+ return servlet;
+ }
+
+ @Override
+ public MetricsContext getRootContext() {
+ return rootContext;
+ }
+
+ @Override
+ public void stop() {
+ shutdownExecutor();
+ if (server != null) {
+ try {
+ server.stop();
+ } catch (Exception err) {
+ LOG.error("Cannot safely stop Jetty server", err);
+ } finally {
+ server = null;
+ }
+ }
+ }
+
+ /**
+ * Dump all values to the 4lw interface and to the Admin server.
+ *
+ * This method is not expected to be used to serve metrics to Prometheus. We
+ * are using the MetricsServlet provided by Prometheus for that, leaving the
+ * real representation to the Prometheus Java client.
+ *
+ *
+ * @param sink the receiver of data (4lw interface, Admin server or tests)
+ */
+ @Override
+ public void dump(BiConsumer sink) {
+ sampleGauges();
+ Enumeration samplesFamilies = collectorRegistry.metricFamilySamples();
+ while (samplesFamilies.hasMoreElements()) {
+ Collector.MetricFamilySamples samples = samplesFamilies.nextElement();
+ samples.samples.forEach(sample -> {
+ String key = buildKeyForDump(sample);
+ sink.accept(key, sample.value);
+ });
+ }
+ }
+
+ private static String buildKeyForDump(Collector.MetricFamilySamples.Sample sample) {
+ StringBuilder keyBuilder = new StringBuilder();
+ keyBuilder.append(sample.name);
+ if (sample.labelNames.size() > 0) {
+ keyBuilder.append('{');
+ for (int i = 0; i < sample.labelNames.size(); ++i) {
+ if (i > 0) {
+ keyBuilder.append(',');
+ }
+ keyBuilder.append(sample.labelNames.get(i));
+ keyBuilder.append("=\"");
+ keyBuilder.append(sample.labelValues.get(i));
+ keyBuilder.append('"');
+ }
+ keyBuilder.append('}');
+ }
+ return keyBuilder.toString();
+ }
+
+ /**
+ * Update Gauges. In ZooKeeper Metrics API Gauges are callbacks served by
+ * internal components and the value is not held by Prometheus structures.
+ */
+ private void sampleGauges() {
+ rootContext.gauges.values()
+ .forEach(PrometheusGaugeWrapper::sample);
+
+ rootContext.gaugeSets.values()
+ .forEach(PrometheusLabelledGaugeWrapper::sample);
+ }
+
+ @Override
+ public void resetAllValues() {
+ // not supported on Prometheus
+ }
+
+ /**
+ * Add constraint to a given context to disallow TRACE method.
+ * @param ctxHandler the context to modify
+ */
+ private void constrainTraceMethod(ServletContextHandler ctxHandler) {
+ ServletConstraint c = new ServletConstraint();
+ c.setAuthenticate(true);
+
+ ConstraintMapping cmt = new ConstraintMapping();
+ cmt.setConstraint(c);
+ cmt.setMethod("TRACE");
+ cmt.setPathSpec("/*");
+
+ ConstraintSecurityHandler securityHandler = new ConstraintSecurityHandler();
+ securityHandler.setConstraintMappings(new ConstraintMapping[] {cmt});
+
+ ctxHandler.setSecurityHandler(securityHandler);
+ }
+
+ private class Context implements MetricsContext {
+
+ private final ConcurrentMap gauges = new ConcurrentHashMap<>();
+ private final ConcurrentMap gaugeSets = new ConcurrentHashMap<>();
+ private final ConcurrentMap counters = new ConcurrentHashMap<>();
+ private final ConcurrentMap counterSets = new ConcurrentHashMap<>();
+ private final ConcurrentMap basicSummaries = new ConcurrentHashMap<>();
+ private final ConcurrentMap summaries = new ConcurrentHashMap<>();
+ private final ConcurrentMap basicSummarySets = new ConcurrentHashMap<>();
+ private final ConcurrentMap summarySets = new ConcurrentHashMap<>();
+
+ @Override
+ public MetricsContext getContext(String name) {
+ // no hierarchy yet
+ return this;
+ }
+
+ @Override
+ public Counter getCounter(String name) {
+ return counters.computeIfAbsent(name, PrometheusCounter::new);
+ }
+
+ @Override
+ public CounterSet getCounterSet(final String name) {
+ Objects.requireNonNull(name, "Cannot register a CounterSet with null name");
+ return counterSets.computeIfAbsent(name, PrometheusLabelledCounter::new);
+ }
+
+ /**
+ * Gauges may go up and down, in ZooKeeper they are a way to export
+ * internal values with a callback.
+ *
+ * @param name the name of the gauge
+ * @param gauge the callback
+ */
+ @Override
+ public void registerGauge(String name, Gauge gauge) {
+ Objects.requireNonNull(name);
+ gauges.compute(name, (id, prev) ->
+ new PrometheusGaugeWrapper(id, gauge, prev != null ? prev.inner : null));
+ }
+
+ @Override
+ public void unregisterGauge(String name) {
+ PrometheusGaugeWrapper existing = gauges.remove(name);
+ if (existing != null) {
+ existing.unregister();
+ }
+ }
+
+ @Override
+ public void registerGaugeSet(final String name, final GaugeSet gaugeSet) {
+ Objects.requireNonNull(name, "Cannot register a GaugeSet with null name");
+ Objects.requireNonNull(gaugeSet, "Cannot register a null GaugeSet for " + name);
+
+ gaugeSets.compute(name, (id, prev) ->
+ new PrometheusLabelledGaugeWrapper(name, gaugeSet, prev != null ? prev.inner : null));
+ }
+
+ @Override
+ public void unregisterGaugeSet(final String name) {
+ Objects.requireNonNull(name, "Cannot unregister GaugeSet with null name");
+
+ final PrometheusLabelledGaugeWrapper existing = gaugeSets.remove(name);
+ if (existing != null) {
+ existing.unregister();
+ }
+ }
+
+ @Override
+ public Summary getSummary(String name, DetailLevel detailLevel) {
+ if (detailLevel == DetailLevel.BASIC) {
+ return basicSummaries.computeIfAbsent(name, (n) -> {
+ if (summaries.containsKey(n)) {
+ throw new IllegalArgumentException("Already registered a non basic summary as " + n);
+ }
+ return new PrometheusSummary(name, detailLevel);
+ });
+ } else {
+ return summaries.computeIfAbsent(name, (n) -> {
+ if (basicSummaries.containsKey(n)) {
+ throw new IllegalArgumentException("Already registered a basic summary as " + n);
+ }
+ return new PrometheusSummary(name, detailLevel);
+ });
+ }
+ }
+
+ @Override
+ public SummarySet getSummarySet(String name, DetailLevel detailLevel) {
+ if (detailLevel == DetailLevel.BASIC) {
+ return basicSummarySets.computeIfAbsent(name, (n) -> {
+ if (summarySets.containsKey(n)) {
+ throw new IllegalArgumentException("Already registered a non basic summary set as " + n);
+ }
+ return new PrometheusLabelledSummary(name, detailLevel);
+ });
+ } else {
+ return summarySets.computeIfAbsent(name, (n) -> {
+ if (basicSummarySets.containsKey(n)) {
+ throw new IllegalArgumentException("Already registered a basic summary set as " + n);
+ }
+ return new PrometheusLabelledSummary(name, detailLevel);
+ });
+ }
+ }
+
+ }
+
+ private class PrometheusCounter implements Counter {
+
+ private final io.prometheus.client.Counter inner;
+ private final String name;
+
+ public PrometheusCounter(String name) {
+ this.name = name;
+ this.inner = io.prometheus.client.Counter
+ .build(name, name)
+ .register(collectorRegistry);
+ }
+
+ @Override
+ public void add(long delta) {
+ try {
+ inner.inc(delta);
+ } catch (IllegalArgumentException err) {
+ LOG.error("invalid delta {} for metric {}", delta, name, err);
+ }
+ }
+
+ @Override
+ public long get() {
+ // this method is used only for tests
+ // Prometheus returns a "double"
+ // it is safe to fine to a long
+ // we are never setting non-integer values
+ return (long) inner.get();
+ }
+
+ }
+
+ private class PrometheusLabelledCounter implements CounterSet {
+ private final String name;
+ private final io.prometheus.client.Counter inner;
+
+ public PrometheusLabelledCounter(final String name) {
+ this.name = name;
+ this.inner = io.prometheus.client.Counter
+ .build(name, name)
+ .labelNames(LABELS)
+ .register(collectorRegistry);
+ }
+
+ @Override
+ public void add(final String key, final long delta) {
+ try {
+ inner.labels(key).inc(delta);
+ } catch (final IllegalArgumentException e) {
+ LOG.error("invalid delta {} for metric {} with key {}", delta, name, key, e);
+ }
+ }
+ }
+
+ private class PrometheusGaugeWrapper {
+
+ private final io.prometheus.client.Gauge inner;
+ private final Gauge gauge;
+ private final String name;
+
+ public PrometheusGaugeWrapper(String name, Gauge gauge, io.prometheus.client.Gauge prev) {
+ this.name = name;
+ this.gauge = gauge;
+ this.inner = prev != null ? prev
+ : io.prometheus.client.Gauge
+ .build(name, name)
+ .register(collectorRegistry);
+ }
+
+ /**
+ * Call the callack and update Prometheus Gauge. This method is called
+ * when the server is polling for a value.
+ */
+ private void sample() {
+ Number value = gauge.get();
+ this.inner.set(value != null ? value.doubleValue() : 0);
+ }
+
+ private void unregister() {
+ collectorRegistry.unregister(inner);
+ }
+ }
+
+ /**
+ * Prometheus implementation of GaugeSet interface. It wraps the GaugeSet object and
+ * uses the callback API to update the Prometheus Gauge.
+ */
+ private class PrometheusLabelledGaugeWrapper {
+ private final GaugeSet gaugeSet;
+ private final io.prometheus.client.Gauge inner;
+
+ private PrometheusLabelledGaugeWrapper(final String name,
+ final GaugeSet gaugeSet,
+ final io.prometheus.client.Gauge prev) {
+ this.gaugeSet = gaugeSet;
+ this.inner = prev != null ? prev :
+ io.prometheus.client.Gauge
+ .build(name, name)
+ .labelNames(LABELS)
+ .register(collectorRegistry);
+ }
+
+ /**
+ * Call the callback provided by the GaugeSet and update Prometheus Gauge.
+ * This method is called when the server is polling for a value.
+ */
+ private void sample() {
+ gaugeSet.values().forEach((key, value) ->
+ this.inner.labels(key).set(value != null ? value.doubleValue() : 0));
+ }
+
+ private void unregister() {
+ collectorRegistry.unregister(inner);
+ }
+ }
+
+ private class PrometheusSummary implements Summary {
+
+ private final io.prometheus.client.Summary inner;
+ private final String name;
+
+ public PrometheusSummary(String name, MetricsContext.DetailLevel level) {
+ this.name = name;
+ if (level == MetricsContext.DetailLevel.ADVANCED) {
+ this.inner = io.prometheus.client.Summary
+ .build(name, name)
+ .quantile(0.5, 0.05) // Add 50th percentile (= median) with 5% tolerated error
+ .quantile(0.9, 0.01) // Add 90th percentile with 1% tolerated error
+ .quantile(0.99, 0.001) // Add 99th percentile with 0.1% tolerated error
+ .register(collectorRegistry);
+ } else {
+ this.inner = io.prometheus.client.Summary
+ .build(name, name)
+ .quantile(0.5, 0.05) // Add 50th percentile (= median) with 5% tolerated error
+ .register(collectorRegistry);
+ }
+ }
+
+ @Override
+ public void add(long delta) {
+ reportMetrics(() -> observe(delta));
+ }
+
+ private void observe(final long delta) {
+ try {
+ inner.observe(delta);
+ } catch (final IllegalArgumentException err) {
+ LOG.error("invalid delta {} for metric {}", delta, name, err);
+ }
+ }
+ }
+
+ private class PrometheusLabelledSummary implements SummarySet {
+
+ private final io.prometheus.client.Summary inner;
+ private final String name;
+
+ public PrometheusLabelledSummary(String name, MetricsContext.DetailLevel level) {
+ this.name = name;
+ if (level == MetricsContext.DetailLevel.ADVANCED) {
+ this.inner = io.prometheus.client.Summary
+ .build(name, name)
+ .labelNames(LABELS)
+ .quantile(0.5, 0.05) // Add 50th percentile (= median) with 5% tolerated error
+ .quantile(0.9, 0.01) // Add 90th percentile with 1% tolerated error
+ .quantile(0.99, 0.001) // Add 99th percentile with 0.1% tolerated error
+ .register(collectorRegistry);
+ } else {
+ this.inner = io.prometheus.client.Summary
+ .build(name, name)
+ .labelNames(LABELS)
+ .quantile(0.5, 0.05) // Add 50th percentile (= median) with 5% tolerated error
+ .register(collectorRegistry);
+ }
+ }
+
+ @Override
+ public void add(String key, long value) {
+ reportMetrics(() -> observe(key, value));
+ }
+
+ private void observe(final String key, final long value) {
+ try {
+ inner.labels(key).observe(value);
+ } catch (final IllegalArgumentException err) {
+ LOG.error("invalid value {} for metric {} with key {}", value, name, key, err);
+ }
+ }
+
+ }
+
+ class MetricsServletImpl extends MetricsServlet {
+
+ @Override
+ protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws IOException {
+ // little trick: update the Gauges before serving data
+ // from Prometheus CollectorRegistry
+ sampleGauges();
+ // serve data using Prometheus built in client.
+ super.doGet(req, resp);
+ }
+ }
+
+ private Optional createExecutor() {
+ if (numWorkerThreads < 1) {
+ LOG.info("Executor service was not created as numWorkerThreads {} is less than 1", numWorkerThreads);
+ return Optional.empty();
+ }
+
+ final BlockingQueue queue = new LinkedBlockingQueue<>(maxQueueSize);
+ final ThreadPoolExecutor executor = new ThreadPoolExecutor(numWorkerThreads,
+ numWorkerThreads,
+ 0L,
+ TimeUnit.MILLISECONDS,
+ queue, new PrometheusWorkerThreadFactory());
+ LOG.info("Executor service was created with numWorkerThreads {} and maxQueueSize {}",
+ numWorkerThreads,
+ maxQueueSize);
+ return Optional.of(executor);
+ }
+
+ private void shutdownExecutor() {
+ if (executorOptional.isPresent()) {
+ LOG.info("Shutdown executor service with timeout {}", workerShutdownTimeoutMs);
+ final ExecutorService executor = executorOptional.get();
+ executor.shutdown();
+ try {
+ if (!executor.awaitTermination(workerShutdownTimeoutMs, TimeUnit.MILLISECONDS)) {
+ LOG.error("Not all the Prometheus worker threads terminated properly after {} timeout",
+ workerShutdownTimeoutMs);
+ executor.shutdownNow();
+ }
+ } catch (final Exception e) {
+ LOG.error("Error occurred while terminating Prometheus worker threads", e);
+ executor.shutdownNow();
+ }
+ }
+ }
+
+ private static class PrometheusWorkerThreadFactory implements ThreadFactory {
+ private static final AtomicInteger workerCounter = new AtomicInteger(1);
+
+ @Override
+ public Thread newThread(final Runnable runnable) {
+ final String threadName = "PrometheusMetricsProviderWorker-" + workerCounter.getAndIncrement();
+ final Thread thread = new Thread(runnable, threadName);
+ thread.setDaemon(true);
+ return thread;
+ }
+ }
+
+ private void reportMetrics(final Runnable task) {
+ if (executorOptional.isPresent()) {
+ try {
+ executorOptional.get().submit(task);
+ } catch (final RejectedExecutionException e) {
+ rateLogger.rateLimitLog("Prometheus metrics reporting task queue size exceeded the max",
+ String.valueOf(maxQueueSize));
+ }
+ } else {
+ task.run();
+ }
+ }
+}
diff --git a/jetty-upgrade/zookeeper-prometheus-metrics/src/main/java/org/apache/pulsar/metrics/prometheus/zookeeper/package-info.java b/jetty-upgrade/zookeeper-prometheus-metrics/src/main/java/org/apache/pulsar/metrics/prometheus/zookeeper/package-info.java
new file mode 100644
index 0000000000000..59e2add5b97be
--- /dev/null
+++ b/jetty-upgrade/zookeeper-prometheus-metrics/src/main/java/org/apache/pulsar/metrics/prometheus/zookeeper/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/**
+ * Prometheus metrics provider for ZooKeeper that is using Jetty 12.
+ */
+package org.apache.pulsar.metrics.prometheus.zookeeper;
diff --git a/jetty-upgrade/zookeeper-prometheus-metrics/src/test/java/org/apache/pulsar/metrics/prometheus/zookeeper/ExportJvmInfoTest.java b/jetty-upgrade/zookeeper-prometheus-metrics/src/test/java/org/apache/pulsar/metrics/prometheus/zookeeper/ExportJvmInfoTest.java
new file mode 100644
index 0000000000000..635fe1b0833a4
--- /dev/null
+++ b/jetty-upgrade/zookeeper-prometheus-metrics/src/test/java/org/apache/pulsar/metrics/prometheus/zookeeper/ExportJvmInfoTest.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.metrics.prometheus.zookeeper;
+
+import static org.testng.AssertJUnit.assertEquals;
+import io.prometheus.client.CollectorRegistry;
+import java.util.Properties;
+import org.testng.annotations.Test;
+
+/**
+ * Tests about Prometheus Metrics Provider. Please note that we are not testing
+ * Prometheus but our integration.
+ */
+public class ExportJvmInfoTest {
+
+ @Test
+ public void exportInfo() throws Exception {
+ runTest(true);
+ }
+
+ @Test
+ public void doNotExportInfo() throws Exception {
+ runTest(false);
+ }
+
+ private void runTest(boolean exportJvmInfo) throws Exception {
+ CollectorRegistry.defaultRegistry.clear();
+ PrometheusMetricsProvider provider = new PrometheusMetricsProvider();
+ try {
+ Properties configuration = new Properties();
+ configuration.setProperty("httpPort", "0"); // ephemeral port
+ configuration.setProperty("exportJvmInfo", exportJvmInfo + "");
+ provider.configure(configuration);
+ provider.start();
+ boolean[] found = {false};
+ provider.dump((k, v) -> {
+ found[0] = found[0] || k.contains("heap");
+ });
+ assertEquals(exportJvmInfo, found[0]);
+ } finally {
+ provider.stop();
+ }
+ }
+
+}
diff --git a/jetty-upgrade/zookeeper-prometheus-metrics/src/test/java/org/apache/pulsar/metrics/prometheus/zookeeper/PrometheusMetricsProviderConfigTest.java b/jetty-upgrade/zookeeper-prometheus-metrics/src/test/java/org/apache/pulsar/metrics/prometheus/zookeeper/PrometheusMetricsProviderConfigTest.java
new file mode 100644
index 0000000000000..f076a15db9380
--- /dev/null
+++ b/jetty-upgrade/zookeeper-prometheus-metrics/src/test/java/org/apache/pulsar/metrics/prometheus/zookeeper/PrometheusMetricsProviderConfigTest.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.metrics.prometheus.zookeeper;
+
+import io.prometheus.client.CollectorRegistry;
+import java.util.Properties;
+import org.apache.zookeeper.metrics.MetricsProviderLifeCycleException;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+
+public class PrometheusMetricsProviderConfigTest {
+
+ @Test
+ public void testInvalidPort() {
+ Assert.assertThrows(MetricsProviderLifeCycleException.class, () -> {
+ CollectorRegistry.defaultRegistry.clear();
+ PrometheusMetricsProvider provider = new PrometheusMetricsProvider();
+ Properties configuration = new Properties();
+ configuration.setProperty("httpPort", "65536");
+ configuration.setProperty("exportJvmInfo", "false");
+ provider.configure(configuration);
+ provider.start();
+ });
+ }
+
+ @Test
+ public void testInvalidAddr() {
+ Assert.assertThrows(MetricsProviderLifeCycleException.class, () -> {
+ CollectorRegistry.defaultRegistry.clear();
+ PrometheusMetricsProvider provider = new PrometheusMetricsProvider();
+ Properties configuration = new Properties();
+ configuration.setProperty("httpHost", "master");
+ provider.configure(configuration);
+ provider.start();
+ });
+ }
+
+ @Test
+ public void testValidConfig() throws MetricsProviderLifeCycleException {
+ CollectorRegistry.defaultRegistry.clear();
+ PrometheusMetricsProvider provider = new PrometheusMetricsProvider();
+ Properties configuration = new Properties();
+ configuration.setProperty("httpHost", "0.0.0.0");
+ configuration.setProperty("httpPort", "0");
+ provider.configure(configuration);
+ provider.start();
+ }
+
+}
diff --git a/jetty-upgrade/zookeeper-prometheus-metrics/src/test/java/org/apache/pulsar/metrics/prometheus/zookeeper/PrometheusMetricsProviderTest.java b/jetty-upgrade/zookeeper-prometheus-metrics/src/test/java/org/apache/pulsar/metrics/prometheus/zookeeper/PrometheusMetricsProviderTest.java
new file mode 100644
index 0000000000000..55b993b3ab5e7
--- /dev/null
+++ b/jetty-upgrade/zookeeper-prometheus-metrics/src/test/java/org/apache/pulsar/metrics/prometheus/zookeeper/PrometheusMetricsProviderTest.java
@@ -0,0 +1,714 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.metrics.prometheus.zookeeper;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.testng.Assert.assertThrows;
+import static org.testng.AssertJUnit.assertEquals;
+import static org.testng.AssertJUnit.assertSame;
+import static org.testng.AssertJUnit.assertTrue;
+import static org.testng.AssertJUnit.fail;
+import io.prometheus.client.CollectorRegistry;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.lang.reflect.Field;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicInteger;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.zookeeper.metrics.Counter;
+import org.apache.zookeeper.metrics.CounterSet;
+import org.apache.zookeeper.metrics.Gauge;
+import org.apache.zookeeper.metrics.GaugeSet;
+import org.apache.zookeeper.metrics.MetricsContext;
+import org.apache.zookeeper.metrics.Summary;
+import org.apache.zookeeper.metrics.SummarySet;
+import org.apache.zookeeper.server.util.QuotaMetricsUtils;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+
+/**
+ * Tests about Prometheus Metrics Provider. Please note that we are not testing
+ * Prometheus but only our integration.
+ */
+public class PrometheusMetricsProviderTest {
+
+ private static final String URL_FORMAT = "http://localhost:%d/metrics";
+ private PrometheusMetricsProvider provider;
+
+ @BeforeMethod
+ public void setup() throws Exception {
+ CollectorRegistry.defaultRegistry.clear();
+ provider = new PrometheusMetricsProvider();
+ Properties configuration = new Properties();
+ configuration.setProperty("numWorkerThreads", "0"); // sync behavior for test
+ configuration.setProperty("httpHost", "127.0.0.1"); // local host for test
+ configuration.setProperty("httpPort", "0"); // ephemeral port
+ configuration.setProperty("exportJvmInfo", "false");
+ provider.configure(configuration);
+ provider.start();
+ }
+
+ @AfterMethod(alwaysRun = true)
+ public void tearDown() {
+ if (provider != null) {
+ provider.stop();
+ }
+ CollectorRegistry.defaultRegistry.clear();
+ }
+
+ @Test
+ public void testCounters() throws Exception {
+ Counter counter = provider.getRootContext().getCounter("cc");
+ counter.add(10);
+ int[] count = {0};
+ provider.dump((k, v) -> {
+ if (k.contains("_created")) {
+ return;
+ }
+ assertEquals("cc_total", k);
+ assertEquals(10, ((Number) v).intValue());
+ count[0]++;
+ }
+ );
+ assertEquals(1, count[0]);
+ count[0] = 0;
+
+ // this is not allowed but it must not throw errors
+ counter.add(-1);
+
+ provider.dump((k, v) -> {
+ if (k.contains("_created")) {
+ return;
+ }
+ assertEquals("cc_total", k);
+ assertEquals(10, ((Number) v).intValue());
+ count[0]++;
+ }
+ );
+ assertEquals(1, count[0]);
+
+ // we always must get the same object
+ assertSame(counter, provider.getRootContext().getCounter("cc"));
+
+ String res = callServlet();
+ assertThat(res).contains("# TYPE cc_total counter");
+ assertThat(res).contains("cc_total 10.0");
+ }
+
+ @Test
+ public void testCounterSet_single() throws Exception {
+ // create and register a CounterSet
+ final String name = QuotaMetricsUtils.QUOTA_EXCEEDED_ERROR_PER_NAMESPACE;
+ final CounterSet counterSet = provider.getRootContext().getCounterSet(name);
+ final String[] keys = {"ns1", "ns2"};
+ final int count = 3;
+
+ // update the CounterSet multiple times
+ for (int i = 0; i < count; i++) {
+ Arrays.asList(keys).forEach(key -> counterSet.inc(key));
+ Arrays.asList(keys).forEach(key -> counterSet.add(key, 2));
+ }
+
+ // validate with dump call
+ final Map expectedMetricsMap = new HashMap<>();
+ for (final String key : keys) {
+ expectedMetricsMap.put(String.format("%s_total{key=\"%s\"}", name, key), count * 3.0);
+ }
+ validateWithDump(expectedMetricsMap);
+
+ // validate with servlet call
+ final List expectedNames = Collections.singletonList(String.format("# TYPE %s count", name));
+ final List expectedMetrics = new ArrayList<>();
+ for (final String key : keys) {
+ expectedMetrics.add(String.format("%s{key=\"%s\",} %s", name, key, count * 3.0));
+ }
+ validateWithServletCall(expectedNames, expectedMetrics);
+
+ // validate registering with same name, no overwriting
+ assertSame(counterSet, provider.getRootContext().getCounterSet(name));
+ }
+
+ @Test
+ public void testCounterSet_multiple() throws Exception {
+ final String name = QuotaMetricsUtils.QUOTA_EXCEEDED_ERROR_PER_NAMESPACE;
+
+ final String[] names = new String[]{name + "_1", name + "_2"};
+ final String[] keys = new String[]{"ns21", "ns22"};
+ final int[] counts = new int[]{3, 5};
+
+ final int length = names.length;
+ final CounterSet[] counterSets = new CounterSet[length];
+
+ // create and register the CounterSets
+ for (int i = 0; i < length; i++) {
+ counterSets[i] = provider.getRootContext().getCounterSet(names[i]);
+ }
+
+ // update each CounterSet multiple times
+ for (int i = 0; i < length; i++) {
+ for (int j = 0; j < counts[i]; j++) {
+ counterSets[i].inc(keys[i]);
+ }
+ }
+
+ // validate with dump call
+ final Map expectedMetricsMap = new HashMap<>();
+ for (int i = 0; i < length; i++) {
+ expectedMetricsMap.put(String.format("%s_total{key=\"%s\"}", names[i], keys[i]), counts[i] * 1.0);
+ }
+ validateWithDump(expectedMetricsMap);
+
+ // validate with servlet call
+ final List expectedNames = new ArrayList<>();
+ final List expectedMetrics = new ArrayList<>();
+ for (int i = 0; i < length; i++) {
+ expectedNames.add(String.format("# TYPE %s count", names[i]));
+ expectedMetrics.add(String.format("%s{key=\"%s\",} %s", names[i], keys[i], counts[i] * 1.0));
+ }
+ validateWithServletCall(expectedNames, expectedMetrics);
+ }
+
+ @Test
+ public void testCounterSet_registerWithNullName() {
+ assertThrows(NullPointerException.class,
+ () -> provider.getRootContext().getCounterSet(null));
+ }
+
+ @Test
+ public void testCounterSet_negativeValue() {
+ // create and register a CounterSet
+ final String name = QuotaMetricsUtils.QUOTA_EXCEEDED_ERROR_PER_NAMESPACE;
+ final CounterSet counterSet = provider.getRootContext().getCounterSet(name);
+
+ // add negative value and make sure no exception is thrown
+ counterSet.add("ns1", -1);
+ }
+
+ @Test
+ public void testCounterSet_nullKey() {
+ // create and register a CounterSet
+ final String name = QuotaMetricsUtils.QUOTA_EXCEEDED_ERROR_PER_NAMESPACE;
+ final CounterSet counterSet = provider.getRootContext().getCounterSet(name);
+
+ // increment the count with null key and make sure no exception is thrown
+ counterSet.inc(null);
+ counterSet.add(null, 2);
+ }
+
+ @Test
+ public void testGauge() throws Exception {
+ int[] values = {78, -89};
+ int[] callCounts = {0, 0};
+ Gauge gauge0 = () -> {
+ callCounts[0]++;
+ return values[0];
+ };
+ Gauge gauge1 = () -> {
+ callCounts[1]++;
+ return values[1];
+ };
+ provider.getRootContext().registerGauge("gg", gauge0);
+
+ int[] count = {0};
+ provider.dump((k, v) -> {
+ assertEquals("gg", k);
+ assertEquals(values[0], ((Number) v).intValue());
+ count[0]++;
+ }
+ );
+ assertEquals(1, callCounts[0]);
+ assertEquals(0, callCounts[1]);
+ assertEquals(1, count[0]);
+ count[0] = 0;
+ String res2 = callServlet();
+ assertThat(res2).contains("# TYPE gg gauge");
+ assertThat(res2.contains("gg 78.0"));
+
+ provider.getRootContext().unregisterGauge("gg");
+ provider.dump((k, v) -> {
+ count[0]++;
+ }
+ );
+ assertEquals(2, callCounts[0]);
+ assertEquals(0, callCounts[1]);
+ assertEquals(0, count[0]);
+ String res3 = callServlet();
+ assertTrue(res3.isEmpty());
+
+ provider.getRootContext().registerGauge("gg", gauge1);
+
+ provider.dump((k, v) -> {
+ assertEquals("gg", k);
+ assertEquals(values[1], ((Number) v).intValue());
+ count[0]++;
+ }
+ );
+ assertEquals(2, callCounts[0]);
+ assertEquals(1, callCounts[1]);
+ assertEquals(1, count[0]);
+ count[0] = 0;
+
+ String res4 = callServlet();
+ assertThat(res4.contains("# TYPE gg gauge"));
+ assertThat(res4.contains("gg -89.0"));
+ assertEquals(2, callCounts[0]);
+ // the servlet must sample the value again (from gauge1)
+ assertEquals(2, callCounts[1]);
+
+ // override gauge, without unregister
+ provider.getRootContext().registerGauge("gg", gauge0);
+
+ provider.dump((k, v) -> {
+ count[0]++;
+ }
+ );
+ assertEquals(1, count[0]);
+ assertEquals(3, callCounts[0]);
+ assertEquals(2, callCounts[1]);
+ }
+
+ @Test
+ public void testBasicSummary() throws Exception {
+ Summary summary = provider.getRootContext()
+ .getSummary("cc", MetricsContext.DetailLevel.BASIC);
+ summary.add(10);
+ summary.add(10);
+ int[] count = {0};
+ provider.dump((k, v) -> {
+ count[0]++;
+ int value = ((Number) v).intValue();
+
+ switch (k) {
+ case "cc{quantile=\"0.5\"}":
+ assertEquals(10, value);
+ break;
+ case "cc_count":
+ assertEquals(2, value);
+ break;
+ case "cc_sum":
+ assertEquals(20, value);
+ break;
+ case "cc_created":
+ break;
+ default:
+ fail("unexpected key " + k);
+ break;
+ }
+ }
+ );
+ assertEquals(4, count[0]);
+ count[0] = 0;
+
+ // we always must get the same object
+ assertSame(summary, provider.getRootContext()
+ .getSummary("cc", MetricsContext.DetailLevel.BASIC));
+
+ try {
+ provider.getRootContext()
+ .getSummary("cc", MetricsContext.DetailLevel.ADVANCED);
+ fail("Can't get the same summary with a different DetailLevel");
+ } catch (IllegalArgumentException err) {
+ assertThat(err.getMessage()).contains("Already registered");
+ }
+
+ String res = callServlet();
+ assertThat(res).contains("# TYPE cc summary");
+ assertThat(res).contains("cc_sum 20.0");
+ assertThat(res).contains("cc_count 2.0");
+ assertThat(res).contains("cc{quantile=\"0.5\",} 10.0");
+ }
+
+ @Test
+ public void testAdvancedSummary() throws Exception {
+ Summary summary = provider.getRootContext()
+ .getSummary("cc", MetricsContext.DetailLevel.ADVANCED);
+ summary.add(10);
+ summary.add(10);
+ int[] count = {0};
+ provider.dump((k, v) -> {
+ count[0]++;
+ int value = ((Number) v).intValue();
+
+ switch (k) {
+ case "cc{quantile=\"0.5\"}":
+ assertEquals(10, value);
+ break;
+ case "cc{quantile=\"0.9\"}":
+ assertEquals(10, value);
+ break;
+ case "cc{quantile=\"0.99\"}":
+ assertEquals(10, value);
+ break;
+ case "cc_count":
+ assertEquals(2, value);
+ break;
+ case "cc_sum":
+ assertEquals(20, value);
+ break;
+ case "cc_created":
+ break;
+ default:
+ fail("unexpected key " + k);
+ break;
+ }
+ }
+ );
+ assertEquals(6, count[0]);
+ count[0] = 0;
+
+ // we always must get the same object
+ assertSame(summary, provider.getRootContext()
+ .getSummary("cc", MetricsContext.DetailLevel.ADVANCED));
+
+ try {
+ provider.getRootContext()
+ .getSummary("cc", MetricsContext.DetailLevel.BASIC);
+ fail("Can't get the same summary with a different DetailLevel");
+ } catch (IllegalArgumentException err) {
+ assertThat(err.getMessage()).contains("Already registered");
+ }
+
+ String res = callServlet();
+ assertThat(res).contains("# TYPE cc summary");
+ assertThat(res).contains("cc_sum 20.0");
+ assertThat(res).contains("cc_count 2.0");
+ assertThat(res).contains("cc{quantile=\"0.5\",} 10.0");
+ assertThat(res).contains("cc{quantile=\"0.9\",} 10.0");
+ assertThat(res).contains("cc{quantile=\"0.99\",} 10.0");
+ }
+
+ /**
+ * Using TRACE method to visit metrics provider, the response should be 403 forbidden.
+ */
+ @Test
+ public void testTraceCall() throws IOException, IllegalAccessException, NoSuchFieldException {
+ Field privateServerField = provider.getClass().getDeclaredField("server");
+ privateServerField.setAccessible(true);
+ Server server = (Server) privateServerField.get(provider);
+ int port = ((ServerConnector) server.getConnectors()[0]).getLocalPort();
+
+ String metricsUrl = String.format(URL_FORMAT, port);
+ HttpURLConnection conn = (HttpURLConnection) new URL(metricsUrl).openConnection();
+ conn.setRequestMethod("TRACE");
+ conn.connect();
+ assertEquals(HttpURLConnection.HTTP_FORBIDDEN, conn.getResponseCode());
+ }
+
+ @Test
+ public void testSummary_asyncAndExceedMaxQueueSize() throws Exception {
+ final Properties config = new Properties();
+ config.setProperty("numWorkerThreads", "1");
+ config.setProperty("maxQueueSize", "1");
+ config.setProperty("httpPort", "0"); // ephemeral port
+ config.setProperty("exportJvmInfo", "false");
+
+ PrometheusMetricsProvider metricsProvider = null;
+ try {
+ metricsProvider = new PrometheusMetricsProvider();
+ metricsProvider.configure(config);
+ metricsProvider.start();
+ final Summary summary =
+ metricsProvider.getRootContext().getSummary("cc", MetricsContext.DetailLevel.ADVANCED);
+
+ // make sure no error is thrown
+ for (int i = 0; i < 10; i++) {
+ summary.add(10);
+ }
+ } finally {
+ if (metricsProvider != null) {
+ metricsProvider.stop();
+ }
+ }
+ }
+
+ @Test
+ public void testSummarySet() throws Exception {
+ final String name = "ss";
+ final String[] keys = {"ns1", "ns2"};
+ final double count = 3.0;
+
+ // create and register a SummarySet
+ final SummarySet summarySet = provider.getRootContext()
+ .getSummarySet(name, MetricsContext.DetailLevel.BASIC);
+
+ // update the SummarySet multiple times
+ for (int i = 0; i < count; i++) {
+ Arrays.asList(keys).forEach(key -> summarySet.add(key, 1));
+ }
+
+ // validate with dump call
+ final Map expectedMetricsMap = new HashMap<>();
+ for (final String key : keys) {
+ expectedMetricsMap.put(String.format("%s{key=\"%s\",quantile=\"0.5\"}", name, key), 1.0);
+ expectedMetricsMap.put(String.format("%s_count{key=\"%s\"}", name, key), count);
+ expectedMetricsMap.put(String.format("%s_sum{key=\"%s\"}", name, key), count);
+ }
+ validateWithDump(expectedMetricsMap);
+
+ // validate with servlet call
+ final List expectedNames = Collections.singletonList(String.format("# TYPE %s summary", name));
+ final List expectedMetrics = new ArrayList<>();
+ for (final String key : keys) {
+ expectedMetrics.add(String.format("%s{key=\"%s\",quantile=\"0.5\",} %s", name, key, 1.0));
+ expectedMetrics.add(String.format("%s_count{key=\"%s\",} %s", name, key, count));
+ expectedMetrics.add(String.format("%s_sum{key=\"%s\",} %s", name, key, count));
+ }
+ validateWithServletCall(expectedNames, expectedMetrics);
+
+ // validate registering with same name, no overwriting
+ assertSame(summarySet, provider.getRootContext()
+ .getSummarySet(name, MetricsContext.DetailLevel.BASIC));
+
+ // validate registering with different DetailLevel, not allowed
+ try {
+ provider.getRootContext()
+ .getSummarySet(name, MetricsContext.DetailLevel.ADVANCED);
+ fail("Can't get the same summarySet with a different DetailLevel");
+ } catch (final IllegalArgumentException e) {
+ assertThat(e.getMessage()).contains("Already registered");
+ }
+ }
+
+ private String callServlet() throws ServletException, IOException {
+ // we are not performing an HTTP request
+ // but we are calling directly the servlet
+ StringWriter writer = new StringWriter();
+ HttpServletResponse response = mock(HttpServletResponse.class);
+ when(response.getWriter()).thenReturn(new PrintWriter(writer));
+ HttpServletRequest req = mock(HttpServletRequest.class);
+ provider.getServlet().doGet(req, response);
+ String res = writer.toString();
+ return res;
+ }
+
+ @Test
+ public void testGaugeSet_singleGaugeSet() throws Exception {
+ final String name = QuotaMetricsUtils.QUOTA_BYTES_LIMIT_PER_NAMESPACE;
+ final Number[] values = {10.0, 100.0};
+ final String[] keys = {"ns11", "ns12"};
+ final Map metricsMap = new HashMap<>();
+ for (int i = 0; i < values.length; i++) {
+ metricsMap.put(keys[i], values[i]);
+ }
+ final AtomicInteger callCount = new AtomicInteger(0);
+
+ // create and register GaugeSet
+ createAndRegisterGaugeSet(name, metricsMap, callCount);
+
+ // validate with dump call
+ final Map expectedMetricsMap = new HashMap<>();
+ for (int i = 0; i < values.length; i++) {
+ expectedMetricsMap.put(String.format("%s{key=\"%s\"}", name, keys[i]), values[i]);
+ }
+ validateWithDump(expectedMetricsMap);
+ assertEquals(1, callCount.get());
+
+ // validate with servlet call
+ final List expectedNames = Collections.singletonList(String.format("# TYPE %s gauge", name));
+ final List expectedMetrics = new ArrayList<>();
+ for (int i = 0; i < values.length; i++) {
+ expectedMetrics.add(String.format("%s{key=\"%s\",} %s", name, keys[i], values[i]));
+ }
+ validateWithServletCall(expectedNames, expectedMetrics);
+ assertEquals(2, callCount.get());
+
+ // unregister the GaugeSet
+ callCount.set(0);
+ provider.getRootContext().unregisterGaugeSet(name);
+
+ // validate with dump call
+ validateWithDump(Collections.emptyMap());
+ assertEquals(0, callCount.get());
+
+ // validate with servlet call
+ validateWithServletCall(new ArrayList<>(), new ArrayList<>());
+ assertEquals(0, callCount.get());
+ }
+
+ @Test
+ public void testGaugeSet_multipleGaugeSets() throws Exception {
+ final String[] names = new String[]{
+ QuotaMetricsUtils.QUOTA_COUNT_LIMIT_PER_NAMESPACE,
+ QuotaMetricsUtils.QUOTA_COUNT_USAGE_PER_NAMESPACE
+ };
+
+ final Number[] values = new Number[]{20.0, 200.0};
+ final String[] keys = new String[]{"ns21", "ns22"};
+ final int count = names.length;
+ final AtomicInteger[] callCounts = new AtomicInteger[count];
+
+ // create and register the GaugeSets
+ for (int i = 0; i < count; i++) {
+ final Map metricsMap = new HashMap<>();
+ metricsMap.put(keys[i], values[i]);
+ callCounts[i] = new AtomicInteger(0);
+ createAndRegisterGaugeSet(names[i], metricsMap, callCounts[i]);
+ }
+
+ // validate with dump call
+ final Map expectedMetricsMap = new HashMap<>();
+ for (int i = 0; i < count; i++) {
+ expectedMetricsMap.put(String.format("%s{key=\"%s\"}", names[i], keys[i]), values[i]);
+ }
+ validateWithDump(expectedMetricsMap);
+ for (int i = 0; i < count; i++) {
+ assertEquals(1, callCounts[i].get());
+ }
+
+ // validate with servlet call
+ final List expectedNames = new ArrayList<>();
+ final List expectedMetrics = new ArrayList<>();
+ for (int i = 0; i < count; i++) {
+ expectedNames.add(String.format("# TYPE %s gauge", names[i]));
+ expectedMetrics.add(String.format("%s{key=\"%s\",} %s", names[i], keys[i], values[i]));
+ }
+ validateWithServletCall(expectedNames, expectedMetrics);
+ for (int i = 0; i < count; i++) {
+ assertEquals(2, callCounts[i].get());
+ }
+
+ // unregister the GaugeSets
+ for (int i = 0; i < count; i++) {
+ callCounts[i].set(0);
+ provider.getRootContext().unregisterGaugeSet(names[i]);
+ }
+
+ // validate with dump call
+ validateWithDump(Collections.emptyMap());
+ for (int i = 0; i < count; i++) {
+ assertEquals(0, callCounts[i].get());
+ }
+
+ // validate with servlet call
+ validateWithServletCall(new ArrayList<>(), new ArrayList<>());
+ for (int i = 0; i < count; i++) {
+ assertEquals(0, callCounts[i].get());
+ }
+ }
+
+ @Test
+ public void testGaugeSet_overwriteRegister() {
+ final String[] names = new String[]{
+ QuotaMetricsUtils.QUOTA_COUNT_LIMIT_PER_NAMESPACE,
+ QuotaMetricsUtils.QUOTA_COUNT_USAGE_PER_NAMESPACE
+ };
+
+ final int count = names.length;
+ final Number[] values = new Number[]{30.0, 300.0};
+ final String[] keys = new String[]{"ns31", "ns32"};
+ final AtomicInteger[] callCounts = new AtomicInteger[count];
+
+ // create and register the GaugeSets
+ for (int i = 0; i < count; i++) {
+ final Map metricsMap = new HashMap<>();
+ metricsMap.put(keys[i], values[i]);
+ callCounts[i] = new AtomicInteger(0);
+ // use the same name so the first GaugeSet got overwrite
+ createAndRegisterGaugeSet(names[0], metricsMap, callCounts[i]);
+ }
+
+ // validate with dump call to make sure the second GaugeSet overwrites the first
+ final Map expectedMetricsMap = new HashMap<>();
+ expectedMetricsMap.put(String.format("%s{key=\"%s\"}", names[0], keys[1]), values[1]);
+ validateWithDump(expectedMetricsMap);
+ assertEquals(0, callCounts[0].get());
+ assertEquals(1, callCounts[1].get());
+ }
+
+ @Test
+ public void testGaugeSet_nullKey() {
+ final String name = QuotaMetricsUtils.QUOTA_COUNT_LIMIT_PER_NAMESPACE;
+ final Map metricsMap = new HashMap<>();
+ metricsMap.put(null, 10.0);
+
+ final AtomicInteger callCount = new AtomicInteger(0);
+
+ // create and register GaugeSet
+ createAndRegisterGaugeSet(name, metricsMap, callCount);
+
+ // validate with dump call
+ assertThrows(IllegalArgumentException.class, () -> provider.dump(new HashMap<>()::put));
+
+ // validate with servlet call
+ assertThrows(IllegalArgumentException.class, this::callServlet);
+ }
+
+ @Test
+ public void testGaugeSet_registerWithNullGaugeSet() {
+ assertThrows(NullPointerException.class,
+ () -> provider.getRootContext().registerGaugeSet("name", null));
+
+ assertThrows(NullPointerException.class,
+ () -> provider.getRootContext().registerGaugeSet(null, HashMap::new));
+ }
+
+ @Test
+ public void testGaugeSet_unregisterNull() {
+ assertThrows(NullPointerException.class,
+ () -> provider.getRootContext().unregisterGaugeSet(null));
+ }
+
+ private void createAndRegisterGaugeSet(final String name,
+ final Map metricsMap,
+ final AtomicInteger callCount) {
+ final GaugeSet gaugeSet = () -> {
+ callCount.addAndGet(1);
+ return metricsMap;
+ };
+ provider.getRootContext().registerGaugeSet(name, gaugeSet);
+ }
+
+ private void validateWithDump(final Map expectedMetrics) {
+ final Map returnedMetrics = new HashMap<>();
+ provider.dump((key, value) -> {
+ if (!key.contains("_created{")) {
+ returnedMetrics.put(key, value);
+ }
+ });
+ assertThat(returnedMetrics).isEqualTo(expectedMetrics);
+ }
+
+ private void validateWithServletCall(final List expectedNames,
+ final List expectedMetrics) throws Exception {
+ final String response = callServlet();
+ if (expectedNames.isEmpty() && expectedMetrics.isEmpty()) {
+ assertTrue(response.isEmpty());
+ } else {
+ expectedNames.forEach(name -> assertThat(response.contains(name)));
+ expectedMetrics.forEach(metric -> assertThat(response.contains(metric)));
+ }
+ }
+}
diff --git a/jetty-upgrade/zookeeper-with-patched-admin/pom.xml b/jetty-upgrade/zookeeper-with-patched-admin/pom.xml
new file mode 100644
index 0000000000000..bf309c74b5da3
--- /dev/null
+++ b/jetty-upgrade/zookeeper-with-patched-admin/pom.xml
@@ -0,0 +1,105 @@
+
+
+
+ 4.0.0
+
+ org.apache.pulsar
+ jetty-upgrade
+ 4.2.0-SNAPSHOT
+
+
+ zookeeper-with-patched-admin
+ Apache Pulsar :: patched Apache ZooKeeper with Admin API using Jetty 12.1.x
+ Apache ZooKeeper with patched admin servlet using Jetty 12.1.x API
+
+
+
+ org.apache.zookeeper
+ zookeeper
+
+
+ org.apache.zookeeper
+ zookeeper-jute
+
+
+ org.eclipse.jetty
+ jetty-server
+ provided
+
+
+ org.eclipse.jetty.ee8
+ jetty-ee8-servlet
+ provided
+
+
+ com.github.spotbugs
+ spotbugs-annotations
+ provided
+ true
+
+
+ com.fasterxml.jackson.core
+ jackson-databind
+ provided
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-shade-plugin
+
+
+ ${shadePluginPhase}
+
+ shade
+
+
+ true
+ true
+ true
+
+
+ org.apache.zookeeper:zookeeper
+
+
+
+
+ org.apache.zookeeper:zookeeper
+
+ **
+
+
+ org/apache/zookeeper/server/admin/**
+
+
+
+
+
+
+
+
+
+
\ No newline at end of file
diff --git a/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/AdminServer.java b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/AdminServer.java
new file mode 100644
index 0000000000000..095b4e0ddf064
--- /dev/null
+++ b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/AdminServer.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.zookeeper.server.admin;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.server.ZooKeeperServer;
+
+/**
+ * Interface for an embedded admin server that runs Commands. There is only one
+ * functional implementation, JettyAdminServer. DummyAdminServer, which does
+ * nothing, is used when we do not wish to run a server.
+ */
+@InterfaceAudience.Public
+public interface AdminServer {
+
+ void start() throws AdminServerException;
+
+ void shutdown() throws AdminServerException;
+
+ void setZooKeeperServer(ZooKeeperServer zkServer);
+
+ @InterfaceAudience.Public
+ class AdminServerException extends Exception {
+
+ private static final long serialVersionUID = 1L;
+
+ public AdminServerException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+ public AdminServerException(Throwable cause) {
+ super(cause);
+ }
+
+ }
+
+}
diff --git a/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/AdminServerFactory.java b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/AdminServerFactory.java
new file mode 100644
index 0000000000000..cdf83486e69bc
--- /dev/null
+++ b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/AdminServerFactory.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.zookeeper.server.admin;
+
+import java.lang.reflect.InvocationTargetException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Factory class for creating an AdminServer.
+ */
+public class AdminServerFactory {
+
+ private static final Logger LOG = LoggerFactory.getLogger(AdminServerFactory.class);
+
+ /**
+ * This method encapsulates the logic for whether we should use a
+ * JettyAdminServer (i.e., the AdminServer is enabled) or a DummyAdminServer
+ * (i.e., the AdminServer is disabled). It uses reflection when attempting
+ * to create a JettyAdminServer, rather than referencing the class directly,
+ * so that it's ok to omit Jetty from the classpath if a user doesn't wish
+ * to pull in Jetty with ZooKeeper.
+ */
+ public static AdminServer createAdminServer() {
+ if (!"false".equals(System.getProperty("zookeeper.admin.enableServer"))) {
+ try {
+ Class> jettyAdminServerC = Class.forName("org.apache.zookeeper.server.admin.JettyAdminServer");
+ Object adminServer = jettyAdminServerC.getConstructor().newInstance();
+ return (AdminServer) adminServer;
+
+ } catch (ClassNotFoundException e) {
+ LOG.warn("Unable to start JettyAdminServer", e);
+ } catch (InstantiationException e) {
+ LOG.warn("Unable to start JettyAdminServer", e);
+ } catch (IllegalAccessException e) {
+ LOG.warn("Unable to start JettyAdminServer", e);
+ } catch (InvocationTargetException e) {
+ LOG.warn("Unable to start JettyAdminServer", e);
+ } catch (NoSuchMethodException e) {
+ LOG.warn("Unable to start JettyAdminServer", e);
+ } catch (NoClassDefFoundError e) {
+ LOG.warn("Unable to load jetty, not starting JettyAdminServer", e);
+ }
+ }
+ return new DummyAdminServer();
+ }
+
+}
diff --git a/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/AuthRequest.java b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/AuthRequest.java
new file mode 100644
index 0000000000000..421d4f92931fb
--- /dev/null
+++ b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/AuthRequest.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.zookeeper.server.admin;
+
+import org.apache.zookeeper.ZooDefs;
+
+/**
+ * Class representing auth data for performing ACL check on admin server commands.
+ *
+ * For example, SnapshotCommand requires {@link ZooDefs.Perms.ALL} permission on
+ * the root node.
+ *
+ */
+public class AuthRequest {
+ private final int permission;
+ private final String path;
+
+ /**
+ * @param permission the required permission for auth check
+ * @param path the ZNode path for auth check
+ */
+ public AuthRequest(final int permission, final String path) {
+ this.permission = permission;
+ this.path = path;
+ }
+
+ /**
+ * @return permission
+ */
+ public int getPermission() {
+ return permission;
+ }
+
+ /**
+ * @return ZNode path
+ */
+ public String getPath() {
+ return path;
+ }
+
+ @Override
+ public String toString() {
+ return "AuthRequest{"
+ + "permission=" + permission
+ + ", path='" + path + '\''
+ + '}';
+ }
+}
diff --git a/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/Command.java b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/Command.java
new file mode 100644
index 0000000000000..8a8089e2d6039
--- /dev/null
+++ b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/Command.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.zookeeper.server.admin;
+
+import java.io.InputStream;
+import java.util.Map;
+import java.util.Set;
+import org.apache.zookeeper.server.ZooKeeperServer;
+
+/**
+ * Interface implemented by all commands runnable by JettyAdminServer.
+ *
+ * @see CommandBase
+ * @see Commands
+ * @see JettyAdminServer
+ */
+public interface Command {
+
+ /**
+ * The set of all names that can be used to refer to this command (e.g.,
+ * "configuration", "config", and "conf").
+ */
+ Set getNames();
+
+ /**
+ * The name that is returned with the command response and that appears in
+ * the list of all commands. This should be a member of the set returned by
+ * getNames().
+ */
+ String getPrimaryName();
+
+ /**
+ * @return true if the command requires an active ZooKeeperServer or a
+ * synced peer in order to resolve
+ */
+ boolean isServerRequired();
+
+ /**
+ * @return AuthRequest associated to the command. Null means auth check is not required.
+ */
+ AuthRequest getAuthRequest();
+
+ /**
+ * Run this command for HTTP GET request. Commands take a ZooKeeperServer, String-valued keyword
+ * arguments and return a CommandResponse object containing any information
+ * constituting the response to the command. Commands are responsible for
+ * parsing keyword arguments and performing any error handling if necessary.
+ * Errors should be reported by setting the "error" entry of the returned
+ * map with an appropriate message rather than throwing an exception.
+ *
+ * @param zkServer ZooKeeper server
+ * @param kwargs keyword -> argument value mapping
+ * @return CommandResponse representing response to command containing at minimum:
+ * - "command" key containing the command's primary name
+ * - "error" key containing a String error message or null if no error
+ */
+ CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs);
+
+ /**
+ * Run this command for HTTP POST. Commands take a ZooKeeperServer and InputStream and
+ * return a CommandResponse object containing any information
+ * constituting the response to the command. Commands are responsible for
+ * parsing keyword arguments and performing any error handling if necessary.
+ * Errors should be reported by setting the "error" entry of the returned
+ * map with an appropriate message rather than throwing an exception.
+ *
+ * @param zkServer ZooKeeper server
+ * @param inputStream InputStream from request
+ * @return CommandResponse representing response to command containing at minimum:
+ * - "command" key containing the command's primary name
+ * - "error" key containing a String error message or null if no error
+ */
+ CommandResponse runPost(ZooKeeperServer zkServer, InputStream inputStream);
+}
diff --git a/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/CommandBase.java b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/CommandBase.java
new file mode 100644
index 0000000000000..f6cb35066c384
--- /dev/null
+++ b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/CommandBase.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.zookeeper.server.admin;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+public abstract class CommandBase implements Command {
+
+ private final String primaryName;
+ private final Set names;
+ private final boolean serverRequired;
+ private final AuthRequest authRequest;
+
+ /**
+ * @param names The possible names of this command, with the primary name first.
+ */
+ protected CommandBase(List names) {
+ this(names, true);
+ }
+
+ protected CommandBase(List names, boolean serverRequired) {
+ this(names, serverRequired, null);
+ }
+
+ protected CommandBase(List names, boolean serverRequired, AuthRequest authRequest) {
+ if (authRequest != null && !serverRequired) {
+ throw new IllegalArgumentException("An active server is required for auth check");
+ }
+ this.primaryName = names.get(0);
+ this.names = new HashSet<>(names);
+ this.serverRequired = serverRequired;
+ this.authRequest = authRequest;
+ }
+
+ @Override
+ public String getPrimaryName() {
+ return primaryName;
+ }
+
+ @Override
+ public Set getNames() {
+ return names;
+ }
+
+
+ @Override
+ public boolean isServerRequired() {
+ return serverRequired;
+ }
+
+ @Override
+ public AuthRequest getAuthRequest() {
+ return authRequest;
+ }
+
+ /**
+ * @return A response with the command set to the primary name and the
+ * error set to null (these are the two entries that all command
+ * responses are required to include).
+ */
+ protected CommandResponse initializeResponse() {
+ return new CommandResponse(primaryName);
+ }
+
+}
diff --git a/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/CommandOutputter.java b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/CommandOutputter.java
new file mode 100644
index 0000000000000..2571605c5d522
--- /dev/null
+++ b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/CommandOutputter.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.zookeeper.server.admin;
+
+import java.io.OutputStream;
+import java.io.PrintWriter;
+
+/**
+ * CommandOutputters are used to format the responses from Commands.
+ *
+ * @see Command
+ * @see JettyAdminServer
+ */
+public interface CommandOutputter {
+
+ /**
+ * The MIME type of this output (e.g., "application/json").
+ */
+ String getContentType();
+
+ /**
+ * Print out data as output.
+ */
+ default void output(CommandResponse response, PrintWriter pw) {
+ }
+
+ /**
+ * Stream out data as output.
+ */
+ default void output(final CommandResponse response, final OutputStream os) {
+ }
+}
diff --git a/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/CommandResponse.java b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/CommandResponse.java
new file mode 100644
index 0000000000000..35ba6707650c5
--- /dev/null
+++ b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/CommandResponse.java
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.zookeeper.server.admin;
+
+import java.io.InputStream;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import javax.servlet.http.HttpServletResponse;
+
+/**
+ * A response from running a {@link Command}.
+ */
+public class CommandResponse {
+
+ /**
+ * The key in the map returned by {@link #toMap()} for the command name.
+ */
+ public static final String KEY_COMMAND = "command";
+ /**
+ * The key in the map returned by {@link #toMap()} for the error string.
+ */
+ public static final String KEY_ERROR = "error";
+
+ private final String command;
+ private final String error;
+ private final Map data;
+ private final Map headers;
+ private int statusCode;
+ private InputStream inputStream;
+
+ /**
+ * Creates a new response with no error string.
+ *
+ * @param command command name
+ */
+ public CommandResponse(String command) {
+ this(command, null, HttpServletResponse.SC_OK);
+ }
+
+ /**
+ * Creates a new response.
+ *
+ * @param command command name
+ * @param error error string (may be null)
+ * @param statusCode http status code
+ */
+ public CommandResponse(String command, String error, int statusCode) {
+ this(command, error, statusCode, null);
+ }
+
+
+ /**
+ * Creates a new response.
+ *
+ * @param command command name
+ * @param error error string (may be null)
+ * @param statusCode http status code
+ * @param inputStream inputStream to send out data (may be null)
+ */
+ public CommandResponse(final String command, final String error, final int statusCode,
+ final InputStream inputStream) {
+ this.command = command;
+ this.error = error;
+ data = new LinkedHashMap<>();
+ headers = new HashMap<>();
+ this.statusCode = statusCode;
+ this.inputStream = inputStream;
+ }
+
+ /**
+ * Gets the command name.
+ *
+ * @return command name
+ */
+ public String getCommand() {
+ return command;
+ }
+
+ /**
+ * Gets the error string (may be null).
+ *
+ * @return error string
+ */
+ public String getError() {
+ return error;
+ }
+
+ /**
+ * Gets the http status code.
+ *
+ * @return http status code
+ */
+ public int getStatusCode() {
+ return statusCode;
+ }
+
+ /**
+ * Sets the http status code.
+ */
+ public void setStatusCode(int statusCode) {
+ this.statusCode = statusCode;
+ }
+
+ /**
+ * Gets the InputStream (may be null).
+ *
+ * @return InputStream
+ */
+ public InputStream getInputStream() {
+ return inputStream;
+ }
+
+ /**
+ * Sets the InputStream.
+ */
+ public void setInputStream(final InputStream inputStream) {
+ this.inputStream = inputStream;
+ }
+
+ /**
+ * Adds a key/value pair to this response.
+ *
+ * @param key key
+ * @param value value
+ * @return prior value for key, or null if none
+ */
+ public Object put(String key, Object value) {
+ return data.put(key, value);
+ }
+
+ /**
+ * Adds all key/value pairs in the given map to this response.
+ *
+ * @param m map of key/value pairs
+ */
+ public void putAll(Map extends String, ?> m) {
+ data.putAll(m);
+ }
+
+ /**
+ * Adds a header to this response.
+ *
+ * @param name name of the header
+ * @param value value of the header
+ */
+ public void addHeader(final String name, final String value) {
+ headers.put(name, value);
+ }
+
+ /**
+ * Returns all headers.
+ *
+ * @return map representation of all headers
+ */
+ public Map getHeaders() {
+ return headers;
+ }
+
+ /**
+ * Converts this response to a map. The returned map is mutable, and
+ * changes to it do not reflect back into this response.
+ *
+ * @return map representation of response
+ */
+ public Map toMap() {
+ Map m = new LinkedHashMap<>(data);
+ m.put(KEY_COMMAND, command);
+ m.put(KEY_ERROR, error);
+ m.putAll(data);
+ return m;
+ }
+
+}
diff --git a/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/Commands.java b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/Commands.java
new file mode 100644
index 0000000000000..2b4b89a422f34
--- /dev/null
+++ b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/Commands.java
@@ -0,0 +1,1168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.zookeeper.server.admin;
+
+import static org.apache.zookeeper.server.persistence.FileSnap.SNAPSHOT_FILE_PREFIX;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.net.InetSocketAddress;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.zookeeper.Environment;
+import org.apache.zookeeper.Environment.Entry;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Version;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+import org.apache.zookeeper.server.ServerCnxnFactory;
+import org.apache.zookeeper.server.ServerMetrics;
+import org.apache.zookeeper.server.ZooKeeperServer;
+import org.apache.zookeeper.server.ZooTrace;
+import org.apache.zookeeper.server.auth.ProviderRegistry;
+import org.apache.zookeeper.server.auth.ServerAuthenticationProvider;
+import org.apache.zookeeper.server.persistence.SnapshotInfo;
+import org.apache.zookeeper.server.persistence.Util;
+import org.apache.zookeeper.server.quorum.Follower;
+import org.apache.zookeeper.server.quorum.FollowerZooKeeperServer;
+import org.apache.zookeeper.server.quorum.Leader;
+import org.apache.zookeeper.server.quorum.LeaderZooKeeperServer;
+import org.apache.zookeeper.server.quorum.MultipleAddresses;
+import org.apache.zookeeper.server.quorum.QuorumPeer;
+import org.apache.zookeeper.server.quorum.QuorumPeer.LearnerType;
+import org.apache.zookeeper.server.quorum.QuorumZooKeeperServer;
+import org.apache.zookeeper.server.quorum.ReadOnlyZooKeeperServer;
+import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
+import org.apache.zookeeper.server.util.RateLimiter;
+import org.apache.zookeeper.server.util.ZxidUtils;
+import org.eclipse.jetty.http.HttpStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Class containing static methods for registering and running Commands, as well
+ * as default Command definitions.
+ *
+ * @see Command
+ * @see JettyAdminServer
+ */
+public class Commands {
+
+ static final Logger LOG = LoggerFactory.getLogger(Commands.class);
+ // VisibleForTesting
+ static final String ADMIN_RATE_LIMITER_INTERVAL = "zookeeper.admin.rateLimiterIntervalInMS";
+ private static final long rateLimiterInterval =
+ Integer.parseInt(System.getProperty(ADMIN_RATE_LIMITER_INTERVAL, "300000"));
+ // VisibleForTesting
+ static final String AUTH_INFO_SEPARATOR = " ";
+ // VisibleForTesting
+ static final String ROOT_PATH = "/";
+
+ /**
+ * Maps command names to Command instances.
+ */
+ private static Map commands = new HashMap<>();
+ private static Set primaryNames = new HashSet<>();
+
+ /**
+ * Registers the given command. Registered commands can be run by passing
+ * any of their names to runCommand.
+ */
+ public static void registerCommand(Command command) {
+ for (String name : command.getNames()) {
+ Command prev = commands.put(name, command);
+ if (prev != null) {
+ LOG.warn("Re-registering command {} (primary name = {})", name, command.getPrimaryName());
+ }
+ }
+ primaryNames.add(command.getPrimaryName());
+ }
+
+ /**
+ * Run the registered command with name cmdName. Commands should not produce
+ * any exceptions; any (anticipated) errors should be reported in the
+ * "error" entry of the returned map. Likewise, if no command with the given
+ * name is registered, this will be noted in the "error" entry.
+ *
+ * @param cmdName
+ * @param zkServer
+ * @param kwargs String-valued keyword arguments to the command from HTTP GET request
+ * (may be null if command requires no additional arguments)
+ * @param authInfo auth info for auth check
+ * (null if command requires no auth check)
+ * @param request HTTP request
+ * @return Map representing response to command containing at minimum:
+ * - "command" key containing the command's primary name
+ * - "error" key containing a String error message or null if no error
+ */
+ public static CommandResponse runGetCommand(
+ String cmdName,
+ ZooKeeperServer zkServer,
+ Map kwargs,
+ String authInfo,
+ HttpServletRequest request) {
+ return runCommand(cmdName, zkServer, kwargs, null, authInfo, request, true);
+ }
+
+ /**
+ * Run the registered command with name cmdName. Commands should not produce
+ * any exceptions; any (anticipated) errors should be reported in the
+ * "error" entry of the returned map. Likewise, if no command with the given
+ * name is registered, this will be noted in the "error" entry.
+ *
+ * @param cmdName
+ * @param zkServer
+ * @param inputStream InputStream from HTTP POST request
+ * @return Map representing response to command containing at minimum:
+ * - "command" key containing the command's primary name
+ * - "error" key containing a String error message or null if no error
+ */
+ public static CommandResponse runPostCommand(
+ String cmdName,
+ ZooKeeperServer zkServer,
+ InputStream inputStream,
+ String authInfo,
+ HttpServletRequest request) {
+ return runCommand(cmdName, zkServer, null, inputStream, authInfo, request, false);
+ }
+
+ private static CommandResponse runCommand(
+ String cmdName,
+ ZooKeeperServer zkServer,
+ Map kwargs,
+ InputStream inputStream,
+ String authInfo,
+ HttpServletRequest request,
+ boolean isGet) {
+ Command command = getCommand(cmdName);
+ if (command == null) {
+ // set the status code to 200 to keep the current behavior of existing commands
+ LOG.warn("Unknown command");
+ return new CommandResponse(cmdName, "Unknown command: " + cmdName, HttpServletResponse.SC_OK);
+ }
+ if (command.isServerRequired() && (zkServer == null || !zkServer.isRunning())) {
+ // set the status code to 200 to keep the current behavior of existing commands
+ LOG.warn("This ZooKeeper instance is not currently serving requests for command");
+ return new CommandResponse(cmdName, "This ZooKeeper instance is not currently serving requests",
+ HttpServletResponse.SC_OK);
+ }
+
+ final AuthRequest authRequest = command.getAuthRequest();
+ if (authRequest != null) {
+ if (authInfo == null) {
+ LOG.warn("Auth info is missing for command");
+ return new CommandResponse(cmdName, "Auth info is missing for the command",
+ HttpServletResponse.SC_UNAUTHORIZED);
+ }
+ try {
+ final List ids = handleAuthentication(request, authInfo);
+ handleAuthorization(zkServer, ids, authRequest.getPermission(), authRequest.getPath());
+ } catch (final KeeperException.AuthFailedException e) {
+ return new CommandResponse(cmdName, "Not authenticated", HttpServletResponse.SC_UNAUTHORIZED);
+ } catch (final KeeperException.NoAuthException e) {
+ return new CommandResponse(cmdName, "Not authorized", HttpServletResponse.SC_FORBIDDEN);
+ } catch (final Exception e) {
+ LOG.warn("Error occurred during auth for command", e);
+ return new CommandResponse(cmdName, "Error occurred during auth",
+ HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+ }
+ }
+ return isGet ? command.runGet(zkServer, kwargs) : command.runPost(zkServer, inputStream);
+ }
+
+ private static List handleAuthentication(final HttpServletRequest request, final String authInfo)
+ throws KeeperException.AuthFailedException {
+ final String[] authData = authInfo.split(AUTH_INFO_SEPARATOR);
+ // for IP and x509, auth info only contains the schema and Auth Id will be extracted from HTTP request
+ if (authData.length != 1 && authData.length != 2) {
+ LOG.warn("Invalid auth info length");
+ throw new KeeperException.AuthFailedException();
+ }
+
+ final String schema = authData[0];
+ final ServerAuthenticationProvider authProvider = ProviderRegistry.getServerProvider(schema);
+ if (authProvider != null) {
+ try {
+ final byte[] auth = authData.length == 2 ? authData[1].getBytes(StandardCharsets.UTF_8) : null;
+ final List ids = authProvider.handleAuthentication(request, auth);
+ if (ids.isEmpty()) {
+ LOG.warn("Auth Id list is empty");
+ throw new KeeperException.AuthFailedException();
+ }
+ return ids;
+ } catch (final RuntimeException e) {
+ LOG.warn("Caught runtime exception from AuthenticationProvider", e);
+ throw new KeeperException.AuthFailedException();
+ }
+ } else {
+ LOG.warn("Auth provider not found for schema");
+ throw new KeeperException.AuthFailedException();
+ }
+ }
+
+ /**
+ * Grant or deny authorization for a command by matching
+ * request-provided credentials with the ACLs present on a node.
+ *
+ * @param zkServer the ZooKeeper server object.
+ * @param ids the credentials extracted from the Authorization header.
+ * @param perm the set of permission bits required by the command.
+ * @param path the ZooKeeper node path whose ACLs should be used
+ * to satisfy the perm bits.
+ * @throws KeeperException.NoAuthException if one or more perm
+ * bits could not be satisfied.
+ */
+ private static void handleAuthorization(final ZooKeeperServer zkServer,
+ final List ids,
+ final int perm,
+ final String path)
+ throws KeeperException.NoNodeException, KeeperException.NoAuthException {
+ final DataNode dataNode = zkServer.getZKDatabase().getNode(path);
+ if (dataNode == null) {
+ throw new KeeperException.NoNodeException(path);
+ }
+ final List acls = zkServer.getZKDatabase().aclForNode(dataNode);
+ // Check the individual bits of perm.
+ final int bitWidth = Integer.SIZE - Integer.numberOfLeadingZeros(perm);
+ for (int b = 0; b < bitWidth; b++) {
+ final int permBit = 1 << b;
+ if ((perm & permBit) != 0) {
+ zkServer.checkACL(null, acls, permBit, ids, path, null);
+ }
+ }
+ }
+
+ /**
+ * Returns the primary names of all registered commands.
+ */
+ public static Set getPrimaryNames() {
+ return primaryNames;
+ }
+
+ /**
+ * Returns the commands registered under cmdName with registerCommand, or
+ * null if no command is registered with that name.
+ */
+ public static Command getCommand(String cmdName) {
+ return commands.get(cmdName);
+ }
+
+ static {
+ registerCommand(new CnxnStatResetCommand());
+ registerCommand(new ConfCommand());
+ registerCommand(new ConsCommand());
+ registerCommand(new DigestCommand());
+ registerCommand(new DirsCommand());
+ registerCommand(new DumpCommand());
+ registerCommand(new EnvCommand());
+ registerCommand(new GetTraceMaskCommand());
+ registerCommand(new InitialConfigurationCommand());
+ registerCommand(new IsroCommand());
+ registerCommand(new LastSnapshotCommand());
+ registerCommand(new LeaderCommand());
+ registerCommand(new MonitorCommand());
+ registerCommand(new ObserverCnxnStatResetCommand());
+ registerCommand(new RestoreCommand());
+ registerCommand(new RuokCommand());
+ registerCommand(new SetTraceMaskCommand());
+ registerCommand(new SnapshotCommand());
+ registerCommand(new SrvrCommand());
+ registerCommand(new StatCommand());
+ registerCommand(new StatResetCommand());
+ registerCommand(new SyncedObserverConsCommand());
+ registerCommand(new SystemPropertiesCommand());
+ registerCommand(new VotingViewCommand());
+ registerCommand(new WatchCommand());
+ registerCommand(new WatchesByPathCommand());
+ registerCommand(new WatchSummaryCommand());
+ registerCommand(new ZabStateCommand());
+ }
+
+ /**
+ * Reset all connection statistics.
+ */
+ public static class CnxnStatResetCommand extends GetCommand {
+
+ public CnxnStatResetCommand() {
+ super(Arrays.asList("connection_stat_reset", "crst"));
+ }
+
+ @Override
+ public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) {
+ CommandResponse response = initializeResponse();
+ zkServer.getServerCnxnFactory().resetAllConnectionStats();
+ return response;
+
+ }
+
+ }
+
+ /**
+ * Server configuration parameters.
+ *
+ * @see ZooKeeperServer#getConf()
+ */
+ public static class ConfCommand extends GetCommand {
+
+ public ConfCommand() {
+ super(Arrays.asList("configuration", "conf", "config"));
+ }
+
+ @Override
+ public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) {
+ CommandResponse response = initializeResponse();
+ response.putAll(zkServer.getConf().toMap());
+ return response;
+ }
+
+ }
+
+ /**
+ * Information on client connections to server. Returned Map contains:
+ * - "connections": list of connection info objects
+ *
+ * @see org.apache.zookeeper.server.ServerCnxn#getConnectionInfo(boolean)
+ */
+ public static class ConsCommand extends GetCommand {
+
+ public ConsCommand() {
+ super(Arrays.asList("connections", "cons"));
+ }
+
+ @Override
+ public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) {
+ CommandResponse response = initializeResponse();
+ ServerCnxnFactory serverCnxnFactory = zkServer.getServerCnxnFactory();
+ if (serverCnxnFactory != null) {
+ response.put("connections", serverCnxnFactory.getAllConnectionInfo(false));
+ } else {
+ response.put("connections", Collections.emptyList());
+ }
+ ServerCnxnFactory secureServerCnxnFactory = zkServer.getSecureServerCnxnFactory();
+ if (secureServerCnxnFactory != null) {
+ response.put("secure_connections", secureServerCnxnFactory.getAllConnectionInfo(false));
+ } else {
+ response.put("secure_connections", Collections.emptyList());
+ }
+ return response;
+ }
+
+ }
+
+ /**
+ * Information on ZK datadir and snapdir size in bytes.
+ */
+ public static class DirsCommand extends GetCommand {
+
+ public DirsCommand() {
+ super(Arrays.asList("dirs"));
+ }
+
+ @Override
+ public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) {
+ CommandResponse response = initializeResponse();
+ response.put("datadir_size", zkServer.getDataDirSize());
+ response.put("logdir_size", zkServer.getLogDirSize());
+ return response;
+ }
+
+ }
+
+ /**
+ * Information on session expirations and ephemerals. Returned map contains:
+ * - "expiry_time_to_session_ids": Map<Long, Set<Long>>
+ * time -> sessions IDs of sessions that expire at time
+ * - "session_id_to_ephemeral_paths": Map<Long, Set<String>>
+ * session ID -> ephemeral paths created by that session
+ *
+ * @see ZooKeeperServer#getSessionExpiryMap()
+ * @see ZooKeeperServer#getEphemerals()
+ */
+ public static class DumpCommand extends GetCommand {
+
+ public DumpCommand() {
+ super(Arrays.asList("dump"));
+ }
+
+ @Override
+ public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) {
+ CommandResponse response = initializeResponse();
+ response.put("expiry_time_to_session_ids", zkServer.getSessionExpiryMap());
+ response.put("session_id_to_ephemeral_paths", zkServer.getEphemerals());
+ return response;
+ }
+
+ }
+
+ /**
+ * All defined environment variables.
+ */
+ public static class EnvCommand extends GetCommand {
+
+ public EnvCommand() {
+ super(Arrays.asList("environment", "env", "envi"), false);
+ }
+
+ @Override
+ public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) {
+ CommandResponse response = initializeResponse();
+ for (Entry e : Environment.list()) {
+ response.put(e.getKey(), e.getValue());
+ }
+ return response;
+ }
+
+ }
+
+ /**
+ * Digest histories for every specific number of txns.
+ */
+ public static class DigestCommand extends GetCommand {
+
+ public DigestCommand() {
+ super(Arrays.asList("hash"));
+ }
+
+ @Override
+ public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) {
+ CommandResponse response = initializeResponse();
+ response.put("digests", zkServer.getZKDatabase().getDataTree().getDigestLog());
+ return response;
+ }
+
+ }
+
+ /**
+ * The current trace mask. Returned map contains:
+ * - "tracemask": Long
+ */
+ public static class GetTraceMaskCommand extends GetCommand {
+
+ public GetTraceMaskCommand() {
+ super(Arrays.asList("get_trace_mask", "gtmk"), false);
+ }
+
+ @Override
+ public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) {
+ CommandResponse response = initializeResponse();
+ response.put("tracemask", ZooTrace.getTextTraceLevel());
+ return response;
+ }
+
+ }
+
+ public static class InitialConfigurationCommand extends GetCommand {
+
+ public InitialConfigurationCommand() {
+ super(Arrays.asList("initial_configuration", "icfg"));
+ }
+
+ @Override
+ public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) {
+ CommandResponse response = initializeResponse();
+ response.put("initial_configuration", zkServer.getInitialConfig());
+ return response;
+ }
+
+ }
+
+ /**
+ * Is this server in read-only mode. Returned map contains:
+ * - "is_read_only": Boolean
+ */
+ public static class IsroCommand extends GetCommand {
+
+ public IsroCommand() {
+ super(Arrays.asList("is_read_only", "isro"));
+ }
+
+ @Override
+ public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) {
+ CommandResponse response = initializeResponse();
+ response.put("read_only", zkServer instanceof ReadOnlyZooKeeperServer);
+ return response;
+ }
+
+ }
+
+ /**
+ * Command returns information of the last snapshot that zookeeper server
+ * has finished saving to disk. During the time between the server starts up
+ * and it finishes saving its first snapshot, the command returns the zxid
+ * and last modified time of the snapshot file used for restoration at
+ * server startup. Returned map contains:
+ * - "zxid": String
+ * - "timestamp": Long
+ */
+ public static class LastSnapshotCommand extends GetCommand {
+
+ public LastSnapshotCommand() {
+ super(Arrays.asList("last_snapshot", "lsnp"));
+ }
+
+ @Override
+ public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) {
+ CommandResponse response = initializeResponse();
+ SnapshotInfo info = zkServer.getTxnLogFactory().getLastSnapshotInfo();
+ response.put("zxid", Long.toHexString(info == null ? -1L : info.zxid));
+ response.put("timestamp", info == null ? -1L : info.timestamp);
+ return response;
+ }
+
+ }
+
+ /**
+ * Returns the leader status of this instance and the leader host string.
+ */
+ public static class LeaderCommand extends GetCommand {
+
+ public LeaderCommand() {
+ super(Arrays.asList("leader", "lead"));
+ }
+
+ @Override
+ public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) {
+ CommandResponse response = initializeResponse();
+ if (zkServer instanceof QuorumZooKeeperServer) {
+ response.put("is_leader", zkServer instanceof LeaderZooKeeperServer);
+ QuorumPeer peer = ((QuorumZooKeeperServer) zkServer).self;
+ response.put("leader_id", peer.getLeaderId());
+ String leaderAddress = peer.getLeaderAddress();
+ response.put("leader_ip", leaderAddress != null ? leaderAddress : "");
+ } else {
+ response.put("error", "server is not initialized");
+ }
+ return response;
+ }
+
+ }
+
+ /**
+ * Some useful info for monitoring. Returned map contains:
+ * - "version": String
+ * server version
+ * - "avg_latency": Long
+ * - "max_latency": Long
+ * - "min_latency": Long
+ * - "packets_received": Long
+ * - "packets_sents": Long
+ * - "num_alive_connections": Integer
+ * - "outstanding_requests": Long
+ * number of unprocessed requests
+ * - "server_state": "leader", "follower", or "standalone"
+ * - "znode_count": Integer
+ * - "watch_count": Integer
+ * - "ephemerals_count": Integer
+ * - "approximate_data_size": Long
+ * - "open_file_descriptor_count": Long (unix only)
+ * - "max_file_descriptor_count": Long (unix only)
+ * - "fsync_threshold_exceed_count": Long
+ * - "non_mtls_conn_count": Long
+ * - "non_mtls_remote_conn_count": Long
+ * - "non_mtls_local_conn_count": Long
+ * - "followers": Integer (leader only)
+ * - "synced_followers": Integer (leader only)
+ * - "pending_syncs": Integer (leader only)
+ */
+ public static class MonitorCommand extends GetCommand {
+
+ public MonitorCommand() {
+ super(Arrays.asList("monitor", "mntr"), false);
+ }
+
+ @Override
+ public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) {
+ CommandResponse response = initializeResponse();
+ zkServer.dumpMonitorValues(response::put);
+ ServerMetrics.getMetrics().getMetricsProvider().dump(response::put);
+ return response;
+
+ }
+
+ }
+
+ /**
+ * Reset all observer connection statistics.
+ */
+ public static class ObserverCnxnStatResetCommand extends GetCommand {
+
+ public ObserverCnxnStatResetCommand() {
+ super(Arrays.asList("observer_connection_stat_reset", "orst"));
+ }
+
+ @Override
+ public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) {
+ CommandResponse response = initializeResponse();
+ if (zkServer instanceof LeaderZooKeeperServer) {
+ Leader leader = ((LeaderZooKeeperServer) zkServer).getLeader();
+ leader.resetObserverConnectionStats();
+ } else if (zkServer instanceof FollowerZooKeeperServer) {
+ Follower follower = ((FollowerZooKeeperServer) zkServer).getFollower();
+ follower.resetObserverConnectionStats();
+ }
+ return response;
+ }
+
+ }
+
+ /**
+ * Restore from snapshot on the current server.
+ *
+ * Returned map contains:
+ * - "last_zxid": String
+ */
+ public static class RestoreCommand extends PostCommand {
+ static final String RESPONSE_DATA_LAST_ZXID = "last_zxid";
+ static final String ADMIN_RESTORE_ENABLED = "zookeeper.admin.restore.enabled";
+
+ private RateLimiter rateLimiter;
+
+ public RestoreCommand() {
+ super(Arrays.asList("restore", "rest"), true, new AuthRequest(ZooDefs.Perms.ALL, ROOT_PATH));
+ rateLimiter = new RateLimiter(1, rateLimiterInterval, TimeUnit.MILLISECONDS);
+ }
+
+ @Override
+ public CommandResponse runPost(final ZooKeeperServer zkServer, final InputStream inputStream) {
+ final CommandResponse response = initializeResponse();
+
+ // check feature flag
+ final boolean restoreEnabled = Boolean.parseBoolean(System.getProperty(ADMIN_RESTORE_ENABLED, "true"));
+ if (!restoreEnabled) {
+ response.setStatusCode(HttpServletResponse.SC_SERVICE_UNAVAILABLE);
+ LOG.warn("Restore command is disabled");
+ return response;
+ }
+
+ if (!zkServer.isSerializeLastProcessedZxidEnabled()) {
+ response.setStatusCode(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+ LOG.warn("Restore command requires serializeLastProcessedZxidEnable flag is set to true");
+ return response;
+ }
+
+ if (inputStream == null) {
+ response.setStatusCode(HttpServletResponse.SC_BAD_REQUEST);
+ LOG.warn("InputStream from restore request is null");
+ return response;
+ }
+
+ // check rate limiting
+ if (!rateLimiter.allow()) {
+ response.setStatusCode(HttpStatus.TOO_MANY_REQUESTS_429);
+ ServerMetrics.getMetrics().RESTORE_RATE_LIMITED_COUNT.add(1);
+ LOG.warn("Restore request was rate limited");
+ return response;
+ }
+
+ // restore from snapshot InputStream
+ try {
+ final long lastZxid = zkServer.restoreFromSnapshot(inputStream);
+ response.put(RESPONSE_DATA_LAST_ZXID, lastZxid);
+ } catch (final Exception e) {
+ response.setStatusCode(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+ ServerMetrics.getMetrics().RESTORE_ERROR_COUNT.add(1);
+ LOG.warn("Exception occurred when restore snapshot via the restore command", e);
+ }
+ return response;
+ }
+ }
+
+ /**
+ * No-op command, check if the server is running.
+ */
+ public static class RuokCommand extends GetCommand {
+
+ public RuokCommand() {
+ super(Arrays.asList("ruok"));
+ }
+
+ @Override
+ public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) {
+ return initializeResponse();
+ }
+
+ }
+
+ /**
+ * Sets the trace mask. Required arguments:
+ * - "traceMask": Long
+ * Returned Map contains:
+ * - "tracemask": Long
+ */
+ public static class SetTraceMaskCommand extends GetCommand {
+
+ public SetTraceMaskCommand() {
+ super(Arrays.asList("set_trace_mask", "stmk"), false);
+ }
+
+ @Override
+ public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) {
+ CommandResponse response = initializeResponse();
+ long traceMask;
+ if (!kwargs.containsKey("traceMask")) {
+ response.put("error", "setTraceMask requires long traceMask argument");
+ return response;
+ }
+ try {
+ traceMask = Long.parseLong(kwargs.get("traceMask"));
+ } catch (NumberFormatException e) {
+ response.put("error", "setTraceMask requires long traceMask argument, got " + kwargs.get("traceMask"));
+ return response;
+ }
+
+ ZooTrace.setTextTraceLevel(traceMask);
+ response.put("tracemask", traceMask);
+ return response;
+ }
+
+ }
+
+ /**
+ * Take a snapshot of current server and stream out the data.
+ *
+ * Argument:
+ * - "streaming": optional String to indicate whether streaming out data
+ *
+ * Returned snapshot as stream if streaming is true and metadata of the snapshot
+ * - "last_zxid": String
+ * - "snapshot_size": String
+ */
+ public static class SnapshotCommand extends GetCommand {
+ static final String REQUEST_QUERY_PARAM_STREAMING = "streaming";
+
+ static final String RESPONSE_HEADER_LAST_ZXID = "last_zxid";
+ static final String RESPONSE_HEADER_SNAPSHOT_SIZE = "snapshot_size";
+
+ static final String ADMIN_SNAPSHOT_ENABLED = "zookeeper.admin.snapshot.enabled";
+
+ private final RateLimiter rateLimiter;
+
+ public SnapshotCommand() {
+ super(Arrays.asList("snapshot", "snap"), true, new AuthRequest(ZooDefs.Perms.ALL, ROOT_PATH));
+ rateLimiter = new RateLimiter(1, rateLimiterInterval, TimeUnit.MICROSECONDS);
+ }
+
+ @SuppressFBWarnings(value = "OBL_UNSATISFIED_OBLIGATION",
+ justification = "FileInputStream is passed to CommandResponse and closed in StreamOutputter")
+ @Override
+ public CommandResponse runGet(final ZooKeeperServer zkServer, final Map kwargs) {
+ final CommandResponse response = initializeResponse();
+
+ // check feature flag
+ final boolean snapshotEnabled = Boolean.parseBoolean(System.getProperty(ADMIN_SNAPSHOT_ENABLED, "true"));
+ if (!snapshotEnabled) {
+ response.setStatusCode(HttpServletResponse.SC_SERVICE_UNAVAILABLE);
+ LOG.warn("Snapshot command is disabled");
+ return response;
+ }
+
+ if (!zkServer.isSerializeLastProcessedZxidEnabled()) {
+ response.setStatusCode(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+ LOG.warn("Snapshot command requires serializeLastProcessedZxidEnable flag is set to true");
+ return response;
+ }
+
+ // check rate limiting
+ if (!rateLimiter.allow()) {
+ response.setStatusCode(HttpStatus.TOO_MANY_REQUESTS_429);
+ ServerMetrics.getMetrics().SNAPSHOT_RATE_LIMITED_COUNT.add(1);
+ LOG.warn("Snapshot request was rate limited");
+ return response;
+ }
+
+ // check the streaming query param
+ boolean streaming = true;
+ if (kwargs.containsKey(REQUEST_QUERY_PARAM_STREAMING)) {
+ streaming = Boolean.parseBoolean(kwargs.get(REQUEST_QUERY_PARAM_STREAMING));
+ }
+
+ // take snapshot and stream out data if needed
+ try {
+ final File snapshotFile = zkServer.takeSnapshot(false, false);
+ final long lastZxid = Util.getZxidFromName(snapshotFile.getName(), SNAPSHOT_FILE_PREFIX);
+ response.addHeader(RESPONSE_HEADER_LAST_ZXID, "0x" + ZxidUtils.zxidToString(lastZxid));
+
+ final long size = snapshotFile.length();
+ response.addHeader(RESPONSE_HEADER_SNAPSHOT_SIZE, String.valueOf(size));
+
+ if (size == 0) {
+ response.setStatusCode(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+ ServerMetrics.getMetrics().SNAPSHOT_ERROR_COUNT.add(1);
+ LOG.warn("Snapshot file {} is empty", snapshotFile);
+ } else if (streaming) {
+ response.setInputStream(new FileInputStream(snapshotFile));
+ }
+ } catch (final Exception e) {
+ response.setStatusCode(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+ ServerMetrics.getMetrics().SNAPSHOT_ERROR_COUNT.add(1);
+ LOG.warn("Exception occurred when taking the snapshot via the snapshot admin command", e);
+ }
+ return response;
+ }
+ }
+
+ /**
+ * Server information. Returned map contains:
+ * - "version": String
+ * version of server
+ * - "read_only": Boolean
+ * is server in read-only mode
+ * - "server_stats": ServerStats object
+ * - "node_count": Integer
+ */
+ public static class SrvrCommand extends GetCommand {
+
+ public SrvrCommand() {
+ super(Arrays.asList("server_stats", "srvr"));
+ }
+
+ // Allow subclasses (e.g. StatCommand) to specify their own names
+ protected SrvrCommand(List names) {
+ super(names);
+ }
+
+ @Override
+ public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) {
+ CommandResponse response = initializeResponse();
+ LOG.info("running stat");
+ response.put("version", Version.getFullVersion());
+ response.put("read_only", zkServer instanceof ReadOnlyZooKeeperServer);
+ response.put("server_stats", zkServer.serverStats());
+ response.put("client_response", zkServer.serverStats().getClientResponseStats());
+ if (zkServer instanceof LeaderZooKeeperServer) {
+ Leader leader = ((LeaderZooKeeperServer) zkServer).getLeader();
+ response.put("proposal_stats", leader.getProposalStats());
+ }
+ response.put("node_count", zkServer.getZKDatabase().getNodeCount());
+ return response;
+ }
+
+ }
+
+ /**
+ * Same as SrvrCommand but has extra "connections" entry.
+ */
+ public static class StatCommand extends SrvrCommand {
+
+ public StatCommand() {
+ super(Arrays.asList("stats", "stat"));
+ }
+
+ @Override
+ public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) {
+ CommandResponse response = super.runGet(zkServer, kwargs);
+
+ final Iterable