From ccbfbb7ae7f97b955a9d6384c5d24111186a66c2 Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Wed, 2 Oct 2019 10:21:11 -0700 Subject: [PATCH 01/42] prometheus-emitter --- extensions-contrib/prometheus-emitter/pom.xml | 122 ++++++++++++++++++ .../druid/emitter/prometheus/Metrics.java | 79 ++++++++++++ .../emitter/prometheus/PrometheusEmitter.java | 95 ++++++++++++++ .../prometheus/PrometheusEmitterConfig.java | 44 +++++++ .../prometheus/PrometheusEmitterModule.java | 60 +++++++++ ...rg.apache.druid.initialization.DruidModule | 16 +++ .../prometheus/PrometheusEmitterTest.java | 24 ++++ 7 files changed, 440 insertions(+) create mode 100644 extensions-contrib/prometheus-emitter/pom.xml create mode 100644 extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java create mode 100644 extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java create mode 100644 extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java create mode 100644 extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java create mode 100644 extensions-contrib/prometheus-emitter/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule create mode 100644 extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java diff --git a/extensions-contrib/prometheus-emitter/pom.xml b/extensions-contrib/prometheus-emitter/pom.xml new file mode 100644 index 000000000000..112c5ccc7983 --- /dev/null +++ b/extensions-contrib/prometheus-emitter/pom.xml @@ -0,0 +1,122 @@ + + + + + druid + org.apache.druid + 0.17.0-incubating-SNAPSHOT + ../../pom.xml + + 4.0.0 + + org.apache.druid.extensions.contrib + prometheus-emitter + prometheus-emitter + Extension support for collecting Druid metrics with Prometheus + + + + org.apache.druid + druid-core + ${project.parent.version} + provided + + + io.prometheus + simpleclient + 0.7.0 + + + io.prometheus + simpleclient_httpserver + 0.7.0 + + + com.google.code.findbugs + jsr305 + provided + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + joda-time + joda-time + provided + + + com.google.guava + guava + provided + + + com.google.inject + guice + provided + + + com.fasterxml.jackson.core + jackson-databind + provided + + + com.fasterxml.jackson.core + jackson-core + provided + + + junit + junit + test + + + org.easymock + easymock + test + + + pl.pragmatists + JUnitParams + test + + + org.apache.druid + druid-server + ${project.parent.version} + test-jar + test + + + org.apache.druid + druid-processing + ${project.parent.version} + test-jar + test + + + io.dropwizard.metrics + metrics-core + + + diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java new file mode 100644 index 000000000000..ab586109f8cf --- /dev/null +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java @@ -0,0 +1,79 @@ +package org.apache.druid.emitter.prometheus; + +import java.util.HashMap; +import java.util.Map; + +import io.prometheus.client.Counter; +import io.prometheus.client.Histogram; +import io.prometheus.client.SimpleCollector; + +public class Metrics { + + private Map map = new HashMap<>(); + + public Metric getByName(String name) { + return map.get(name); + } + + public Metrics(String namespace) { + + map.put("query/time", new Metric( + new String[] { "dataSource", "type" }, + new Histogram.Builder() + .namespace(namespace) + .name("query_time") + .labelNames("dataSource", "type") + .buckets(new double[] { .1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300 }) + .register())); + map.put("query/bytes", + new Metric(new String[] { "dataSource", "type" }, + new Counter.Builder() + .namespace(namespace) + .name("query_bytes_total") + .labelNames("dataSource", "type") + .register())); + map.put("query/node/time", + new Metric(new String[] { "server" }, + new Histogram.Builder() + .namespace(namespace) + .name("query_node_time") + .labelNames("server") + .buckets(new double[] { .1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300 }) + .register())); + map.put("query/node/ttfb", + new Metric(new String[] { "server" }, + new Histogram.Builder() + .namespace(namespace) + .name("query_node_ttfb_time") + .labelNames("server") + .buckets(new double[] { .1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300 }) + .register())); + map.put("query/node/bytes", + new Metric(new String[] { "server" }, + new Counter.Builder() + .namespace(namespace) + .name("query_node_bytes_total") + .labelNames("server") + .register())); + } + + + public static class Metric { + private final String[] dimensions; + private final SimpleCollector collector; + + Metric(String[] dimensions, SimpleCollector collector) { + this.dimensions = dimensions; + this.collector = collector; + } + + public String[] getDimensions() { + return dimensions; + } + + public SimpleCollector getCollector() { + return collector; + } + } + +} diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java new file mode 100644 index 000000000000..ca6c42341841 --- /dev/null +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -0,0 +1,95 @@ +/* + * 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.druid.emitter.prometheus; + +import java.util.Map; + +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.emitter.core.Emitter; +import org.apache.druid.java.util.emitter.core.Event; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; + +import io.prometheus.client.Counter; +import io.prometheus.client.Gauge; +import io.prometheus.client.Histogram; + +/** + */ +public class PrometheusEmitter implements Emitter { + + private static final Logger log = new Logger(PrometheusEmitter.class); + private final Metrics metrics; + + static PrometheusEmitter of(PrometheusEmitterConfig config) { + return new PrometheusEmitter(config); + } + + public PrometheusEmitter(PrometheusEmitterConfig config) { + metrics = new Metrics(config.getNamespace()); + } + + + @Override + public void start() { + } + + @Override + public void emit(Event event) { + if (event instanceof ServiceMetricEvent) { + emitMetric((ServiceMetricEvent) event); + } + } + + void emitMetric(ServiceMetricEvent metricEvent) { + String host = metricEvent.getHost(); + String service = metricEvent.getService(); + String metric = metricEvent.getMetric(); + Map userDims = metricEvent.getUserDims(); + Number value = metricEvent.getValue(); + + Metrics.Metric byName = metrics.getByName(metric); + String[] labelValues = new String[byName.getDimensions().length]; + String[] labelNames = byName.getDimensions(); + for (int i = 0; i < labelValues.length; i++) { + String labelName = labelNames[i]; + Object userDim = userDims.get(labelName); + labelValues[i] = userDim.toString(); + } + + if (byName.getCollector() instanceof Counter) { + ((Counter) byName.getCollector()).labels(labelValues).inc(value.doubleValue()); + } else if (byName.getCollector() instanceof Gauge) { + ((Gauge) byName.getCollector()).labels(labelValues).set(value.doubleValue()); + } else if (byName.getCollector() instanceof Histogram) { + ((Histogram) byName.getCollector()).labels(labelValues).observe(value.doubleValue()); + } else { + //TODO + } + + } + + @Override + public void flush() { + } + + @Override + public void close() { + } +} diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java new file mode 100644 index 000000000000..aed6c282243f --- /dev/null +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.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.druid.emitter.prometheus; + +import javax.annotation.Nullable; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + */ +public class PrometheusEmitterConfig +{ + + @JsonProperty + private final String namespace; + + @JsonCreator + public PrometheusEmitterConfig(@JsonProperty("namespace") @Nullable String namespace) + { + this.namespace = namespace != null ? namespace : ""; + } + + public String getNamespace() { + return namespace; + } +} diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java new file mode 100644 index 000000000000..19ef691eb084 --- /dev/null +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.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.druid.emitter.prometheus; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Binder; +import com.google.inject.Provides; +import com.google.inject.name.Named; +import org.apache.druid.guice.JsonConfigProvider; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.initialization.DruidModule; +import org.apache.druid.java.util.emitter.core.Emitter; + +import java.util.Collections; +import java.util.List; + +/** + */ +public class PrometheusEmitterModule implements DruidModule +{ + private static final String EMITTER_TYPE = "prometheus"; + + @Override + public List getJacksonModules() + { + return Collections.EMPTY_LIST; + } + + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.emitter." + EMITTER_TYPE, PrometheusEmitterConfig.class); + } + + @Provides + @ManageLifecycle + @Named(EMITTER_TYPE) + public Emitter getEmitter(PrometheusEmitterConfig config) + { + return PrometheusEmitter.of(config); + } +} diff --git a/extensions-contrib/prometheus-emitter/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/prometheus-emitter/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule new file mode 100644 index 000000000000..da92fc068bf8 --- /dev/null +++ b/extensions-contrib/prometheus-emitter/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule @@ -0,0 +1,16 @@ +# 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. + +org.apache.druid.emitter.prometheus.PrometheusEmitterModule diff --git a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java new file mode 100644 index 000000000000..666bd1419515 --- /dev/null +++ b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java @@ -0,0 +1,24 @@ +/* + * 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.druid.emitter.prometheus; + +public class PrometheusEmitterTest +{ +} From bb8eb39996da6b907b31771f5ce7160c89daf078 Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Wed, 2 Oct 2019 13:25:40 -0700 Subject: [PATCH 02/42] use existing jetty server to expose prometheus collection endpoint --- extensions-contrib/prometheus-emitter/pom.xml | 13 ++ .../druid/emitter/prometheus/Metrics.java | 165 +++++++++++------- .../emitter/prometheus/PrometheusEmitter.java | 34 ++-- .../prometheus/PrometheusEmitterConfig.java | 24 ++- .../prometheus/PrometheusEmitterModule.java | 14 +- pom.xml | 1 + 6 files changed, 170 insertions(+), 81 deletions(-) diff --git a/extensions-contrib/prometheus-emitter/pom.xml b/extensions-contrib/prometheus-emitter/pom.xml index 112c5ccc7983..c909308c7024 100644 --- a/extensions-contrib/prometheus-emitter/pom.xml +++ b/extensions-contrib/prometheus-emitter/pom.xml @@ -118,5 +118,18 @@ io.dropwizard.metrics metrics-core + + org.eclipse.jetty + jetty-server + + + org.eclipse.jetty + jetty-servlet + + + io.prometheus + simpleclient_servlet + 0.4.0 + diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java index ab586109f8cf..69fc55b804bc 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java @@ -1,79 +1,122 @@ -package org.apache.druid.emitter.prometheus; +/* + * 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. + */ -import java.util.HashMap; -import java.util.Map; +package org.apache.druid.emitter.prometheus; import io.prometheus.client.Counter; import io.prometheus.client.Histogram; import io.prometheus.client.SimpleCollector; -public class Metrics { +import java.util.HashMap; +import java.util.Map; - private Map map = new HashMap<>(); +public class Metrics +{ - public Metric getByName(String name) { - return map.get(name); - } + private Map map = new HashMap<>(); - public Metrics(String namespace) { + public Metric getByName(String name) + { + return map.get(name); + } - map.put("query/time", new Metric( - new String[] { "dataSource", "type" }, - new Histogram.Builder() - .namespace(namespace) - .name("query_time") - .labelNames("dataSource", "type") - .buckets(new double[] { .1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300 }) - .register())); - map.put("query/bytes", - new Metric(new String[] { "dataSource", "type" }, - new Counter.Builder() - .namespace(namespace) - .name("query_bytes_total") - .labelNames("dataSource", "type") - .register())); - map.put("query/node/time", - new Metric(new String[] { "server" }, - new Histogram.Builder() - .namespace(namespace) - .name("query_node_time") - .labelNames("server") - .buckets(new double[] { .1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300 }) - .register())); - map.put("query/node/ttfb", - new Metric(new String[] { "server" }, - new Histogram.Builder() - .namespace(namespace) - .name("query_node_ttfb_time") - .labelNames("server") - .buckets(new double[] { .1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300 }) - .register())); - map.put("query/node/bytes", - new Metric(new String[] { "server" }, - new Counter.Builder() - .namespace(namespace) - .name("query_node_bytes_total") - .labelNames("server") - .register())); - } + public Metrics(String namespace) + { + map.put("query/time", new Metric( + new String[]{"dataSource", "type"}, + new Histogram.Builder() + .namespace(namespace) + .name("query_time") + .labelNames("dataSource", "type") + .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) + .register() + )); + map.put( + "query/bytes", + new Metric( + new String[]{"dataSource", "type"}, + new Counter.Builder() + .namespace(namespace) + .name("query_bytes_total") + .labelNames("dataSource", "type") + .register() + ) + ); + map.put( + "query/node/time", + new Metric( + new String[]{"server"}, + new Histogram.Builder() + .namespace(namespace) + .name("query_node_time") + .labelNames("server") + .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) + .register() + ) + ); + map.put( + "query/node/ttfb", + new Metric( + new String[]{"server"}, + new Histogram.Builder() + .namespace(namespace) + .name("query_node_ttfb_time") + .labelNames("server") + .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) + .register() + ) + ); + map.put( + "query/node/bytes", + new Metric( + new String[]{"server"}, + new Counter.Builder() + .namespace(namespace) + .name("query_node_bytes_total") + .labelNames("server") + .register() + ) + ); + } - public static class Metric { - private final String[] dimensions; - private final SimpleCollector collector; - Metric(String[] dimensions, SimpleCollector collector) { - this.dimensions = dimensions; - this.collector = collector; - } + public static class Metric + { + private final String[] dimensions; + private final SimpleCollector collector; - public String[] getDimensions() { - return dimensions; - } + Metric(String[] dimensions, SimpleCollector collector) + { + this.dimensions = dimensions; + this.collector = collector; + } + + public String[] getDimensions() + { + return dimensions; + } - public SimpleCollector getCollector() { - return collector; - } + public SimpleCollector getCollector() + { + return collector; } + } } diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java index ca6c42341841..d1b726e0b054 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -19,45 +19,51 @@ package org.apache.druid.emitter.prometheus; -import java.util.Map; +import io.prometheus.client.Counter; +import io.prometheus.client.Gauge; +import io.prometheus.client.Histogram; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.core.Emitter; import org.apache.druid.java.util.emitter.core.Event; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; - -import io.prometheus.client.Counter; -import io.prometheus.client.Gauge; -import io.prometheus.client.Histogram; +import java.util.Map; /** + * */ -public class PrometheusEmitter implements Emitter { +public class PrometheusEmitter implements Emitter +{ private static final Logger log = new Logger(PrometheusEmitter.class); private final Metrics metrics; - static PrometheusEmitter of(PrometheusEmitterConfig config) { + static PrometheusEmitter of(PrometheusEmitterConfig config) + { return new PrometheusEmitter(config); } - public PrometheusEmitter(PrometheusEmitterConfig config) { + public PrometheusEmitter(PrometheusEmitterConfig config) + { metrics = new Metrics(config.getNamespace()); } @Override - public void start() { + public void start() + { } @Override - public void emit(Event event) { + public void emit(Event event) + { if (event instanceof ServiceMetricEvent) { emitMetric((ServiceMetricEvent) event); } } - void emitMetric(ServiceMetricEvent metricEvent) { + void emitMetric(ServiceMetricEvent metricEvent) + { String host = metricEvent.getHost(); String service = metricEvent.getService(); String metric = metricEvent.getMetric(); @@ -86,10 +92,12 @@ void emitMetric(ServiceMetricEvent metricEvent) { } @Override - public void flush() { + public void flush() + { } @Override - public void close() { + public void close() + { } } diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java index aed6c282243f..c5065192b135 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java @@ -19,12 +19,13 @@ package org.apache.druid.emitter.prometheus; -import javax.annotation.Nullable; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import javax.annotation.Nullable; + /** + * */ public class PrometheusEmitterConfig { @@ -32,13 +33,26 @@ public class PrometheusEmitterConfig @JsonProperty private final String namespace; + @JsonProperty + private final String path; + @JsonCreator - public PrometheusEmitterConfig(@JsonProperty("namespace") @Nullable String namespace) + public PrometheusEmitterConfig( + @JsonProperty("namespace") @Nullable String namespace, + @JsonProperty("path") @Nullable String path + ) { - this.namespace = namespace != null ? namespace : ""; + this.namespace = namespace != null ? namespace : "druid"; + this.path = path != null ? path : "/prometheus"; } - public String getNamespace() { + public String getNamespace() + { return namespace; } + + public String getPath() + { + return path; + } } diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java index 19ef691eb084..42ff7e7b2435 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java @@ -19,20 +19,26 @@ package org.apache.druid.emitter.prometheus; + import com.fasterxml.jackson.databind.Module; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Binder; import com.google.inject.Provides; import com.google.inject.name.Named; +import io.prometheus.client.exporter.MetricsServlet; import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.initialization.DruidModule; import org.apache.druid.java.util.emitter.core.Emitter; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; import java.util.Collections; import java.util.List; + /** + * */ public class PrometheusEmitterModule implements DruidModule { @@ -53,8 +59,12 @@ public void configure(Binder binder) @Provides @ManageLifecycle @Named(EMITTER_TYPE) - public Emitter getEmitter(PrometheusEmitterConfig config) + public Emitter getEmitter(PrometheusEmitterConfig config, Server server) { + ServletContextHandler context = new ServletContextHandler(); + context.setContextPath("/"); + server.setHandler(context); + context.addServlet(new ServletHolder(new MetricsServlet()), config.getPath()); return PrometheusEmitter.of(config); } } diff --git a/pom.xml b/pom.xml index 2101f8394c8a..f873c199eb69 100644 --- a/pom.xml +++ b/pom.xml @@ -198,6 +198,7 @@ extensions-contrib/influxdb-emitter extensions-contrib/gce-extensions extensions-contrib/aliyun-oss-extensions + extensions-contrib/prometheus-emitter distribution From ad4346675308091b1eb593de3252eaed1540976b Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Wed, 2 Oct 2019 14:13:14 -0700 Subject: [PATCH 03/42] unused variables --- .../org/apache/druid/emitter/prometheus/PrometheusEmitter.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java index d1b726e0b054..a77a0a6b5222 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -64,8 +64,6 @@ public void emit(Event event) void emitMetric(ServiceMetricEvent metricEvent) { - String host = metricEvent.getHost(); - String service = metricEvent.getService(); String metric = metricEvent.getMetric(); Map userDims = metricEvent.getUserDims(); Number value = metricEvent.getValue(); From b04eaba556ea8fd4013b1908482df947bf1445f4 Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Wed, 2 Oct 2019 14:49:56 -0700 Subject: [PATCH 04/42] better variable names --- .../emitter/prometheus/PrometheusEmitter.java | 39 ++++++++++--------- 1 file changed, 20 insertions(+), 19 deletions(-) diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java index a77a0a6b5222..530cd94b2c79 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -64,29 +64,30 @@ public void emit(Event event) void emitMetric(ServiceMetricEvent metricEvent) { - String metric = metricEvent.getMetric(); + String name = metricEvent.getMetric(); Map userDims = metricEvent.getUserDims(); Number value = metricEvent.getValue(); - Metrics.Metric byName = metrics.getByName(metric); - String[] labelValues = new String[byName.getDimensions().length]; - String[] labelNames = byName.getDimensions(); - for (int i = 0; i < labelValues.length; i++) { - String labelName = labelNames[i]; - Object userDim = userDims.get(labelName); - labelValues[i] = userDim.toString(); + Metrics.Metric metric = metrics.getByName(name); + if (metric != null) { + String[] labelValues = new String[metric.getDimensions().length]; + String[] labelNames = metric.getDimensions(); + for (int i = 0; i < labelValues.length; i++) { + String labelName = labelNames[i]; + Object userDim = userDims.get(labelName); + labelValues[i] = userDim.toString(); + } + + if (metric.getCollector() instanceof Counter) { + ((Counter) metric.getCollector()).labels(labelValues).inc(value.doubleValue()); + } else if (metric.getCollector() instanceof Gauge) { + ((Gauge) metric.getCollector()).labels(labelValues).set(value.doubleValue()); + } else if (metric.getCollector() instanceof Histogram) { + ((Histogram) metric.getCollector()).labels(labelValues).observe(value.doubleValue()); + } else { + //TODO + } } - - if (byName.getCollector() instanceof Counter) { - ((Counter) byName.getCollector()).labels(labelValues).inc(value.doubleValue()); - } else if (byName.getCollector() instanceof Gauge) { - ((Gauge) byName.getCollector()).labels(labelValues).set(value.doubleValue()); - } else if (byName.getCollector() instanceof Histogram) { - ((Histogram) byName.getCollector()).labels(labelValues).observe(value.doubleValue()); - } else { - //TODO - } - } @Override From 82f408bc82d439aa279b975f64b27cfa5b09b527 Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Wed, 2 Oct 2019 14:56:34 -0700 Subject: [PATCH 05/42] removed unused dependencies --- extensions-contrib/prometheus-emitter/pom.xml | 9 --------- 1 file changed, 9 deletions(-) diff --git a/extensions-contrib/prometheus-emitter/pom.xml b/extensions-contrib/prometheus-emitter/pom.xml index c909308c7024..9be9aba4bb45 100644 --- a/extensions-contrib/prometheus-emitter/pom.xml +++ b/extensions-contrib/prometheus-emitter/pom.xml @@ -45,11 +45,6 @@ simpleclient 0.7.0 - - io.prometheus - simpleclient_httpserver - 0.7.0 - com.google.code.findbugs jsr305 @@ -114,10 +109,6 @@ test-jar test - - io.dropwizard.metrics - metrics-core - org.eclipse.jetty jetty-server From b6750ccb030863c4938a63b097ccca6c3b4e76c5 Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Thu, 3 Oct 2019 08:35:47 -0700 Subject: [PATCH 06/42] more metric definitions --- .../druid/emitter/prometheus/Metrics.java | 539 ++++++++++++++++++ 1 file changed, 539 insertions(+) diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java index 69fc55b804bc..a0ae8af32141 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java @@ -20,6 +20,7 @@ package org.apache.druid.emitter.prometheus; import io.prometheus.client.Counter; +import io.prometheus.client.Gauge; import io.prometheus.client.Histogram; import io.prometheus.client.SimpleCollector; @@ -36,6 +37,8 @@ public Metric getByName(String name) return map.get(name); } + //TODO: revise metric types + //TODO: revise Histogram bucket values public Metrics(String namespace) { @@ -94,6 +97,542 @@ public Metrics(String namespace) .register() ) ); + map.put( + "query/node/backpressure", + new Metric( + new String[]{"server"}, + new Histogram.Builder() + .namespace(namespace) + .name("query_node_backpressure_time") + .labelNames("server") + .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) + .register() + ) + ); + map.put( + "query/intervalChunk/time", + new Metric( + new String[]{}, + new Histogram.Builder() + .namespace(namespace) + .name("query_interval_chunk_time") + .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) + .register() + ) + ); + map.put( + "query/segment/time", + new Metric( + new String[]{}, + new Histogram.Builder() + .namespace(namespace) + .name("query_segment_time") + .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) + .register() + ) + ); + map.put( + "query/wait/time", + new Metric( + new String[]{}, + new Histogram.Builder() + .namespace(namespace) + .name("query_wait_time") + .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) + .register() + ) + ); + map.put( + "segment/scan/pending", + new Metric( + new String[]{}, + new Gauge.Builder() + .namespace(namespace) + .name("segment_scan_pending") + .register() + ) + ); + map.put( + "query/segmentAndCache/time", + new Metric( + new String[]{}, + new Histogram.Builder() + .namespace(namespace) + .name("query_segment_and_cache_time") + .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) + .register() + ) + ); + map.put( + "query/cpu/time", + new Metric( + new String[]{"dataSource", "type"}, + new Histogram.Builder() + .namespace(namespace) + .name("query_cpu_time") + .labelNames("dataSource", "type") + .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) + .register() + ) + ); + map.put( + "query/count", + new Metric( + new String[]{}, + new Counter.Builder() + .namespace(namespace) + .name("query_count_total") + .register() + ) + ); + map.put( + "query/success/count", + new Metric( + new String[]{}, + new Counter.Builder() + .namespace(namespace) + .name("query_success_count_total") + .register() + ) + ); + map.put( + "query/failed/count", + new Metric( + new String[]{}, + new Counter.Builder() + .namespace(namespace) + .name("query_failed_count_total") + .register() + ) + ); + map.put( + "query/interrupted/count", + new Metric( + new String[]{}, + new Counter.Builder() + .namespace(namespace) + .name("query_interrupted_count_total") + .register() + ) + ); + map.put( + "query/cache/delta/numEntries", + new Metric( + new String[]{}, + new Counter.Builder() + .namespace(namespace) + .name("query_cache_delta_numentries_total") + .register() + ) + ); + map.put( + "query/cache/delta/sizeBytes", + new Metric( + new String[]{}, + new Counter.Builder() + .namespace(namespace) + .name("query_cache_delta_sizebytes_total") + .register() + ) + ); + map.put( + "query/cache/delta/hits", + new Metric( + new String[]{}, + new Counter.Builder() + .namespace(namespace) + .name("query_cache_delta_hits_total") + .register() + ) + ); + map.put( + "query/cache/delta/misses", + new Metric( + new String[]{}, + new Counter.Builder() + .namespace(namespace) + .name("query_cache_delta_misses_total") + .register() + ) + ); + map.put( + "query/cache/delta/evictions", + new Metric( + new String[]{}, + new Counter.Builder() + .namespace(namespace) + .name("query_cache_delta_evictions_total") + .register() + ) + ); + // Leaning toward ignoring this since it should be derived from delta/hits above + // "query/cache/delta/hitRate" : { "dimensions" : [], "type" : "count", "convertRange" : true }, + + map.put( + "query/cache/delta/averageBytes", + new Metric( + new String[]{}, + new Counter.Builder() + .namespace(namespace) + .name("query_cache_delta_averagebytes_total") + .register() + ) + ); + map.put( + "query/cache/delta/timeouts", + new Metric( + new String[]{}, + new Counter.Builder() + .namespace(namespace) + .name("query_cache_delta_timeouts_total") + .register() + ) + ); + map.put( + "query/cache/delta/errors", + new Metric( + new String[]{}, + new Counter.Builder() + .namespace(namespace) + .name("query_cache_delta_errors_total") + .register() + ) + ); + + map.put( + "query/cache/total/numentries", + new Metric( + new String[]{}, + new Gauge.Builder() + .namespace(namespace) + .name("query_cache_total_numentries") + .register() + ) + ); + map.put( + "query/cache/total/sizeBytes", + new Metric( + new String[]{}, + new Gauge.Builder() + .namespace(namespace) + .name("query_cache_total_sizebytes") + .register() + ) + ); + map.put( + "query/cache/total/hits", + new Metric( + new String[]{}, + new Gauge.Builder() + .namespace(namespace) + .name("query_cache_total_hits") + .register() + ) + ); + map.put( + "query/cache/total/misses", + new Metric( + new String[]{}, + new Gauge.Builder() + .namespace(namespace) + .name("query_cache_total_misses") + .register() + ) + ); + map.put( + "query/cache/total/evictions", + new Metric( + new String[]{}, + new Gauge.Builder() + .namespace(namespace) + .name("query_cache_total_evictions") + .register() + ) + ); + map.put( + "query/cache/total/hitRate", + new Metric( + new String[]{}, + new Gauge.Builder() + .namespace(namespace) + .name("query_cache_total_hitrate") + .register() + ) + ); + map.put( + "query/cache/total/averageBytes", + new Metric( + new String[]{}, + new Gauge.Builder() + .namespace(namespace) + .name("query_cache_total_averagebytes") + .register() + ) + ); + map.put( + "query/cache/total/timeouts", + new Metric( + new String[]{}, + new Gauge.Builder() + .namespace(namespace) + .name("query_cache_total_timeouts") + .register() + ) + ); + map.put( + "query/cache/total/errors", + new Metric( + new String[]{}, + new Gauge.Builder() + .namespace(namespace) + .name("query_cache_total_errors") + .register() + ) + ); + map.put( + "ingest/events/thrownAway", + new Metric( + new String[]{"dataSource"}, + new Counter.Builder() + .namespace(namespace) + .name("ingest_events_thrownaway_total") + .labelNames("dataSource") + .register() + ) + ); + map.put( + "ingest/events/unparseable", + new Metric( + new String[]{"dataSource"}, + new Counter.Builder() + .namespace(namespace) + .name("ingest_events_unparseable_total") + .labelNames("dataSource") + .register() + ) + ); + map.put( + "ingest/events/duplicate", + new Metric( + new String[]{"dataSource"}, + new Counter.Builder() + .namespace(namespace) + .name("ingest_events_duplicate_total") + .labelNames("dataSource") + .register() + ) + ); + map.put( + "ingest/events/processed", + new Metric( + new String[]{"dataSource"}, + new Counter.Builder() + .namespace(namespace) + .name("ingest_events_processed_total") + .labelNames("dataSource") + .register() + ) + ); + map.put( + "ingest/events/messageGap", + new Metric( + new String[]{"dataSource"}, + new Counter.Builder() + .namespace(namespace) + .name("ingest_events_messagegap_total") + .labelNames("dataSource") + .register() + ) + ); + map.put( + "ingest/rows/output", + new Metric( + new String[]{"dataSource"}, + new Counter.Builder() + .namespace(namespace) + .name("ingest_rows_output_total") + .labelNames("dataSource") + .register() + ) + ); + map.put( + "ingest/persists/count", + new Metric( + new String[]{"dataSource"}, + new Counter.Builder() + .namespace(namespace) + .name("ingest_persists_count_total") + .labelNames("dataSource") + .register() + ) + ); + map.put( + "ingest/persists/time", + new Metric( + new String[]{"dataSource"}, + new Histogram.Builder() + .namespace(namespace) + .name("ingest_persists_time") + .labelNames("dataSource") + .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) + .register() + ) + ); + map.put( + "ingest/persists/cpu", + new Metric( + new String[]{"dataSource"}, + new Histogram.Builder() + .namespace(namespace) + .name("ingest_persists_cpu_time") + .labelNames("dataSource") + .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) + .register() + ) + ); + map.put( + "ingest/persists/backPressure", + new Metric( + new String[]{"dataSource"}, + new Gauge.Builder() + .namespace(namespace) + .name("ingest_persists_backpressure") + .labelNames("dataSource") + .register() + ) + ); + map.put( + "ingest/persists/failed", + new Metric( + new String[]{"dataSource"}, + new Counter.Builder() + .namespace(namespace) + .name("ingest_persists_failed_total") + .labelNames("dataSource") + .register() + ) + ); + map.put( + "ingest/handoff/failed", + new Metric( + new String[]{"dataSource"}, + new Counter.Builder() + .namespace(namespace) + .name("ingest_handoff_failed_total") + .labelNames("dataSource") + .register() + ) + ); + map.put( + "ingest/merge/time", + new Metric( + new String[]{"dataSource"}, + new Histogram.Builder() + .namespace(namespace) + .name("ingest_merge_time") + .labelNames("dataSource") + .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) + .register() + ) + ); + map.put( + "ingest/merge/cpu", + new Metric( + new String[]{"dataSource"}, + new Histogram.Builder() + .namespace(namespace) + .name("ingest_merge_cpu_time") + .labelNames("dataSource") + .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) + .register() + ) + ); + map.put( + "ingest/kafka/lag", + new Metric( + new String[]{"dataSource"}, + new Gauge.Builder() + .namespace(namespace) + .name("ingest_kafka_lag") + .labelNames("dataSource") + .register() + ) + ); + + +// "ingest/kafka/lag" : { "dimensions" : ["dataSource"], "type" : "gauge" }, +// "ingest/kafka/maxLag" : { "dimensions" : ["dataSource"], "type" : "gauge" }, +// "ingest/kafka/avgLag" : { "dimensions" : ["dataSource"], "type" : "gauge" }, +// +// "task/success/count" : { "dimensions" : ["dataSource"], "type" : "count" }, +// "task/failed/count" : { "dimensions" : ["dataSource"], "type" : "count" }, +// "task/running/count" : { "dimensions" : ["dataSource"], "type" : "count" }, +// "task/pending/count" : { "dimensions" : ["dataSource"], "type" : "count" }, +// "task/waiting/count" : { "dimensions" : ["dataSource"], "type" : "count" }, +// +// "task/run/time" : { "dimensions" : ["dataSource", "taskType"], "type" : "timer" }, +// "segment/added/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count" }, +// "segment/moved/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count" }, +// "segment/nuked/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count" }, +// +// "segment/assigned/count" : { "dimensions" : ["tier"], "type" : "count" }, +// "segment/moved/count" : { "dimensions" : ["tier"], "type" : "count" }, +// "segment/dropped/count" : { "dimensions" : ["tier"], "type" : "count" }, +// "segment/deleted/count" : { "dimensions" : ["tier"], "type" : "count" }, +// "segment/unneeded/count" : { "dimensions" : ["tier"], "type" : "count" }, +// "segment/unavailable/count" : { "dimensions" : ["dataSource"], "type" : "count" }, +// "segment/underReplicated/count" : { "dimensions" : ["dataSource", "tier"], "type" : "count" }, +// "segment/cost/raw" : { "dimensions" : ["tier"], "type" : "count" }, +// "segment/cost/normalization" : { "dimensions" : ["tier"], "type" : "count" }, +// "segment/cost/normalized" : { "dimensions" : ["tier"], "type" : "count" }, +// "segment/loadQueue/size" : { "dimensions" : ["server"], "type" : "gauge" }, +// "segment/loadQueue/failed" : { "dimensions" : ["server"], "type" : "gauge" }, +// "segment/loadQueue/count" : { "dimensions" : ["server"], "type" : "gauge" }, +// "segment/dropQueue/count" : { "dimensions" : ["server"], "type" : "gauge" }, +// "segment/size" : { "dimensions" : ["dataSource"], "type" : "gauge" }, +// "segment/overShadowed/count" : { "dimensions" : [], "type" : "gauge" }, +// +// "segment/max" : { "dimensions" : [], "type" : "gauge"}, +// "segment/used" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge" }, +// "segment/usedPercent" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge", "convertRange" : true }, +// "segment/pendingDelete" : { "dimensions" : [], "type" : "gauge"}, +// +// "jvm/pool/committed" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge" }, +// "jvm/pool/init" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge" }, +// "jvm/pool/max" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge" }, +// "jvm/pool/used" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge" }, +// "jvm/bufferpool/count" : { "dimensions" : ["bufferPoolName"], "type" : "gauge" }, +// "jvm/bufferpool/used" : { "dimensions" : ["bufferPoolName"], "type" : "gauge" }, +// "jvm/bufferpool/capacity" : { "dimensions" : ["bufferPoolName"], "type" : "gauge" }, +// "jvm/mem/init" : { "dimensions" : ["memKind"], "type" : "gauge" }, +// "jvm/mem/max" : { "dimensions" : ["memKind"], "type" : "gauge" }, +// "jvm/mem/used" : { "dimensions" : ["memKind"], "type" : "gauge" }, +// "jvm/mem/committed" : { "dimensions" : ["memKind"], "type" : "gauge" }, +// "jvm/gc/count" : { "dimensions" : ["gcName"], "type" : "count" }, +// "jvm/gc/cpu" : { "dimensions" : ["gcName"], "type" : "timer" }, +// +// "ingest/events/buffered" : { "dimensions" : ["serviceName, bufferCapacity"], "type" : "gauge"}, +// +// "sys/swap/free" : { "dimensions" : [], "type" : "gauge"}, +// "sys/swap/max" : { "dimensions" : [], "type" : "gauge"}, +// "sys/swap/pageIn" : { "dimensions" : [], "type" : "gauge"}, +// "sys/swap/pageOut" : { "dimensions" : [], "type" : "gauge"}, +// "sys/disk/write/count" : { "dimensions" : ["fsDevName"], "type" : "count"}, +// "sys/disk/read/count" : { "dimensions" : ["fsDevName"], "type" : "count"}, +// "sys/disk/write/size" : { "dimensions" : ["fsDevName"], "type" : "count"}, +// "sys/disk/read/size" : { "dimensions" : ["fsDevName"], "type" : "count"}, +// "sys/net/write/size" : { "dimensions" : [], "type" : "count"}, +// "sys/net/read/size" : { "dimensions" : [], "type" : "count"}, +// "sys/fs/used" : { "dimensions" : ["fsDevName", "fsDirName", "fsTypeName", "fsSysTypeName", "fsOptions"], "type" : "gauge"}, +// "sys/fs/max" : { "dimensions" : ["fsDevName", "fsDirName", "fsTypeName", "fsSysTypeName", "fsOptions"], "type" : "gauge"}, +// "sys/mem/used" : { "dimensions" : [], "type" : "gauge"}, +// "sys/mem/max" : { "dimensions" : [], "type" : "gauge"}, +// "sys/storage/used" : { "dimensions" : ["fsDirName"], "type" : "gauge"}, +// "sys/cpu" : { "dimensions" : ["cpuName", "cpuTime"], "type" : "gauge"}, +// +// "coordinator-segment/count" : { "dimensions" : ["dataSource"], "type" : "gauge" }, +// "historical-segment/count" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge" } + } From 2dc2ced4e88d5747afc3266f4ab9d23564c8a40b Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Thu, 3 Oct 2019 13:40:59 -0700 Subject: [PATCH 07/42] reorganize --- .../prometheus/DimensionsAndCollector.java | 44 ++ .../druid/emitter/prometheus/Metrics.java | 691 +++--------------- .../emitter/prometheus/PrometheusEmitter.java | 4 +- .../prometheus/PrometheusEmitterConfig.java | 13 +- .../src/main/resources/defaultMetrics.json | 128 ++++ 5 files changed, 268 insertions(+), 612 deletions(-) create mode 100644 extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/DimensionsAndCollector.java create mode 100644 extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/DimensionsAndCollector.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/DimensionsAndCollector.java new file mode 100644 index 000000000000..64279d9d0939 --- /dev/null +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/DimensionsAndCollector.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.druid.emitter.prometheus; + +import io.prometheus.client.SimpleCollector; + +public class DimensionsAndCollector +{ + private final String[] dimensions; + private final SimpleCollector collector; + + DimensionsAndCollector(String[] dimensions, SimpleCollector collector) + { + this.dimensions = dimensions; + this.collector = collector; + } + + public String[] getDimensions() + { + return dimensions; + } + + public SimpleCollector getCollector() + { + return collector; + } +} diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java index a0ae8af32141..7d8e58688795 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java @@ -19,643 +19,116 @@ package org.apache.druid.emitter.prometheus; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Strings; import io.prometheus.client.Counter; import io.prometheus.client.Gauge; import io.prometheus.client.Histogram; import io.prometheus.client.SimpleCollector; - +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; import java.util.HashMap; import java.util.Map; +import java.util.SortedSet; public class Metrics { - private Map map = new HashMap<>(); + private static final Logger log = new Logger(Metrics.class); + private final Map map = new HashMap<>(); + private final ObjectMapper mapper = new ObjectMapper(); - public Metric getByName(String name) + public DimensionsAndCollector getByName(String name) { return map.get(name); } - //TODO: revise metric types - //TODO: revise Histogram bucket values - public Metrics(String namespace) + public Metrics(String namespace, String path) { - - map.put("query/time", new Metric( - new String[]{"dataSource", "type"}, - new Histogram.Builder() + Map metrics = readMap(path); + for (String name : metrics.keySet()) { + Metric metric = metrics.get(name); + Metric.Type type = metric.type; + String[] dimensions = metric.dimensions.toArray(new String[0]); + String formattedName = StringUtils.replaceChar(StringUtils.toLowerCase(name), '/', "_"); + SimpleCollector collector = null; + if (Metric.Type.count.equals(type)) { + collector = new Counter.Builder() .namespace(namespace) - .name("query_time") - .labelNames("dataSource", "type") - .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) - .register() - )); - map.put( - "query/bytes", - new Metric( - new String[]{"dataSource", "type"}, - new Counter.Builder() - .namespace(namespace) - .name("query_bytes_total") - .labelNames("dataSource", "type") - .register() - ) - ); - map.put( - "query/node/time", - new Metric( - new String[]{"server"}, - new Histogram.Builder() - .namespace(namespace) - .name("query_node_time") - .labelNames("server") - .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) - .register() - ) - ); - map.put( - "query/node/ttfb", - new Metric( - new String[]{"server"}, - new Histogram.Builder() - .namespace(namespace) - .name("query_node_ttfb_time") - .labelNames("server") - .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) - .register() - ) - ); - map.put( - "query/node/bytes", - new Metric( - new String[]{"server"}, - new Counter.Builder() - .namespace(namespace) - .name("query_node_bytes_total") - .labelNames("server") - .register() - ) - ); - map.put( - "query/node/backpressure", - new Metric( - new String[]{"server"}, - new Histogram.Builder() - .namespace(namespace) - .name("query_node_backpressure_time") - .labelNames("server") - .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) - .register() - ) - ); - map.put( - "query/intervalChunk/time", - new Metric( - new String[]{}, - new Histogram.Builder() - .namespace(namespace) - .name("query_interval_chunk_time") - .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) - .register() - ) - ); - map.put( - "query/segment/time", - new Metric( - new String[]{}, - new Histogram.Builder() - .namespace(namespace) - .name("query_segment_time") - .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) - .register() - ) - ); - map.put( - "query/wait/time", - new Metric( - new String[]{}, - new Histogram.Builder() - .namespace(namespace) - .name("query_wait_time") - .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) - .register() - ) - ); - map.put( - "segment/scan/pending", - new Metric( - new String[]{}, - new Gauge.Builder() - .namespace(namespace) - .name("segment_scan_pending") - .register() - ) - ); - map.put( - "query/segmentAndCache/time", - new Metric( - new String[]{}, - new Histogram.Builder() - .namespace(namespace) - .name("query_segment_and_cache_time") - .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) - .register() - ) - ); - map.put( - "query/cpu/time", - new Metric( - new String[]{"dataSource", "type"}, - new Histogram.Builder() - .namespace(namespace) - .name("query_cpu_time") - .labelNames("dataSource", "type") - .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) - .register() - ) - ); - map.put( - "query/count", - new Metric( - new String[]{}, - new Counter.Builder() - .namespace(namespace) - .name("query_count_total") - .register() - ) - ); - map.put( - "query/success/count", - new Metric( - new String[]{}, - new Counter.Builder() - .namespace(namespace) - .name("query_success_count_total") - .register() - ) - ); - map.put( - "query/failed/count", - new Metric( - new String[]{}, - new Counter.Builder() - .namespace(namespace) - .name("query_failed_count_total") - .register() - ) - ); - map.put( - "query/interrupted/count", - new Metric( - new String[]{}, - new Counter.Builder() - .namespace(namespace) - .name("query_interrupted_count_total") - .register() - ) - ); - map.put( - "query/cache/delta/numEntries", - new Metric( - new String[]{}, - new Counter.Builder() - .namespace(namespace) - .name("query_cache_delta_numentries_total") - .register() - ) - ); - map.put( - "query/cache/delta/sizeBytes", - new Metric( - new String[]{}, - new Counter.Builder() - .namespace(namespace) - .name("query_cache_delta_sizebytes_total") - .register() - ) - ); - map.put( - "query/cache/delta/hits", - new Metric( - new String[]{}, - new Counter.Builder() - .namespace(namespace) - .name("query_cache_delta_hits_total") - .register() - ) - ); - map.put( - "query/cache/delta/misses", - new Metric( - new String[]{}, - new Counter.Builder() - .namespace(namespace) - .name("query_cache_delta_misses_total") - .register() - ) - ); - map.put( - "query/cache/delta/evictions", - new Metric( - new String[]{}, - new Counter.Builder() - .namespace(namespace) - .name("query_cache_delta_evictions_total") - .register() - ) - ); - // Leaning toward ignoring this since it should be derived from delta/hits above - // "query/cache/delta/hitRate" : { "dimensions" : [], "type" : "count", "convertRange" : true }, - - map.put( - "query/cache/delta/averageBytes", - new Metric( - new String[]{}, - new Counter.Builder() - .namespace(namespace) - .name("query_cache_delta_averagebytes_total") - .register() - ) - ); - map.put( - "query/cache/delta/timeouts", - new Metric( - new String[]{}, - new Counter.Builder() - .namespace(namespace) - .name("query_cache_delta_timeouts_total") - .register() - ) - ); - map.put( - "query/cache/delta/errors", - new Metric( - new String[]{}, - new Counter.Builder() - .namespace(namespace) - .name("query_cache_delta_errors_total") - .register() - ) - ); - - map.put( - "query/cache/total/numentries", - new Metric( - new String[]{}, - new Gauge.Builder() - .namespace(namespace) - .name("query_cache_total_numentries") - .register() - ) - ); - map.put( - "query/cache/total/sizeBytes", - new Metric( - new String[]{}, - new Gauge.Builder() - .namespace(namespace) - .name("query_cache_total_sizebytes") - .register() - ) - ); - map.put( - "query/cache/total/hits", - new Metric( - new String[]{}, - new Gauge.Builder() - .namespace(namespace) - .name("query_cache_total_hits") - .register() - ) - ); - map.put( - "query/cache/total/misses", - new Metric( - new String[]{}, - new Gauge.Builder() - .namespace(namespace) - .name("query_cache_total_misses") - .register() - ) - ); - map.put( - "query/cache/total/evictions", - new Metric( - new String[]{}, - new Gauge.Builder() - .namespace(namespace) - .name("query_cache_total_evictions") - .register() - ) - ); - map.put( - "query/cache/total/hitRate", - new Metric( - new String[]{}, - new Gauge.Builder() - .namespace(namespace) - .name("query_cache_total_hitrate") - .register() - ) - ); - map.put( - "query/cache/total/averageBytes", - new Metric( - new String[]{}, - new Gauge.Builder() - .namespace(namespace) - .name("query_cache_total_averagebytes") - .register() - ) - ); - map.put( - "query/cache/total/timeouts", - new Metric( - new String[]{}, - new Gauge.Builder() - .namespace(namespace) - .name("query_cache_total_timeouts") - .register() - ) - ); - map.put( - "query/cache/total/errors", - new Metric( - new String[]{}, - new Gauge.Builder() - .namespace(namespace) - .name("query_cache_total_errors") - .register() - ) - ); - map.put( - "ingest/events/thrownAway", - new Metric( - new String[]{"dataSource"}, - new Counter.Builder() - .namespace(namespace) - .name("ingest_events_thrownaway_total") - .labelNames("dataSource") - .register() - ) - ); - map.put( - "ingest/events/unparseable", - new Metric( - new String[]{"dataSource"}, - new Counter.Builder() - .namespace(namespace) - .name("ingest_events_unparseable_total") - .labelNames("dataSource") - .register() - ) - ); - map.put( - "ingest/events/duplicate", - new Metric( - new String[]{"dataSource"}, - new Counter.Builder() - .namespace(namespace) - .name("ingest_events_duplicate_total") - .labelNames("dataSource") - .register() - ) - ); - map.put( - "ingest/events/processed", - new Metric( - new String[]{"dataSource"}, - new Counter.Builder() - .namespace(namespace) - .name("ingest_events_processed_total") - .labelNames("dataSource") - .register() - ) - ); - map.put( - "ingest/events/messageGap", - new Metric( - new String[]{"dataSource"}, - new Counter.Builder() - .namespace(namespace) - .name("ingest_events_messagegap_total") - .labelNames("dataSource") - .register() - ) - ); - map.put( - "ingest/rows/output", - new Metric( - new String[]{"dataSource"}, - new Counter.Builder() - .namespace(namespace) - .name("ingest_rows_output_total") - .labelNames("dataSource") - .register() - ) - ); - map.put( - "ingest/persists/count", - new Metric( - new String[]{"dataSource"}, - new Counter.Builder() - .namespace(namespace) - .name("ingest_persists_count_total") - .labelNames("dataSource") - .register() - ) - ); - map.put( - "ingest/persists/time", - new Metric( - new String[]{"dataSource"}, - new Histogram.Builder() - .namespace(namespace) - .name("ingest_persists_time") - .labelNames("dataSource") - .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) - .register() - ) - ); - map.put( - "ingest/persists/cpu", - new Metric( - new String[]{"dataSource"}, - new Histogram.Builder() - .namespace(namespace) - .name("ingest_persists_cpu_time") - .labelNames("dataSource") - .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) - .register() - ) - ); - map.put( - "ingest/persists/backPressure", - new Metric( - new String[]{"dataSource"}, - new Gauge.Builder() - .namespace(namespace) - .name("ingest_persists_backpressure") - .labelNames("dataSource") - .register() - ) - ); - map.put( - "ingest/persists/failed", - new Metric( - new String[]{"dataSource"}, - new Counter.Builder() - .namespace(namespace) - .name("ingest_persists_failed_total") - .labelNames("dataSource") - .register() - ) - ); - map.put( - "ingest/handoff/failed", - new Metric( - new String[]{"dataSource"}, - new Counter.Builder() - .namespace(namespace) - .name("ingest_handoff_failed_total") - .labelNames("dataSource") - .register() - ) - ); - map.put( - "ingest/merge/time", - new Metric( - new String[]{"dataSource"}, - new Histogram.Builder() - .namespace(namespace) - .name("ingest_merge_time") - .labelNames("dataSource") - .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) - .register() - ) - ); - map.put( - "ingest/merge/cpu", - new Metric( - new String[]{"dataSource"}, - new Histogram.Builder() - .namespace(namespace) - .name("ingest_merge_cpu_time") - .labelNames("dataSource") - .buckets(new double[]{.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300}) - .register() - ) - ); - map.put( - "ingest/kafka/lag", - new Metric( - new String[]{"dataSource"}, - new Gauge.Builder() - .namespace(namespace) - .name("ingest_kafka_lag") - .labelNames("dataSource") - .register() - ) - ); - - -// "ingest/kafka/lag" : { "dimensions" : ["dataSource"], "type" : "gauge" }, -// "ingest/kafka/maxLag" : { "dimensions" : ["dataSource"], "type" : "gauge" }, -// "ingest/kafka/avgLag" : { "dimensions" : ["dataSource"], "type" : "gauge" }, -// -// "task/success/count" : { "dimensions" : ["dataSource"], "type" : "count" }, -// "task/failed/count" : { "dimensions" : ["dataSource"], "type" : "count" }, -// "task/running/count" : { "dimensions" : ["dataSource"], "type" : "count" }, -// "task/pending/count" : { "dimensions" : ["dataSource"], "type" : "count" }, -// "task/waiting/count" : { "dimensions" : ["dataSource"], "type" : "count" }, -// -// "task/run/time" : { "dimensions" : ["dataSource", "taskType"], "type" : "timer" }, -// "segment/added/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count" }, -// "segment/moved/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count" }, -// "segment/nuked/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count" }, -// -// "segment/assigned/count" : { "dimensions" : ["tier"], "type" : "count" }, -// "segment/moved/count" : { "dimensions" : ["tier"], "type" : "count" }, -// "segment/dropped/count" : { "dimensions" : ["tier"], "type" : "count" }, -// "segment/deleted/count" : { "dimensions" : ["tier"], "type" : "count" }, -// "segment/unneeded/count" : { "dimensions" : ["tier"], "type" : "count" }, -// "segment/unavailable/count" : { "dimensions" : ["dataSource"], "type" : "count" }, -// "segment/underReplicated/count" : { "dimensions" : ["dataSource", "tier"], "type" : "count" }, -// "segment/cost/raw" : { "dimensions" : ["tier"], "type" : "count" }, -// "segment/cost/normalization" : { "dimensions" : ["tier"], "type" : "count" }, -// "segment/cost/normalized" : { "dimensions" : ["tier"], "type" : "count" }, -// "segment/loadQueue/size" : { "dimensions" : ["server"], "type" : "gauge" }, -// "segment/loadQueue/failed" : { "dimensions" : ["server"], "type" : "gauge" }, -// "segment/loadQueue/count" : { "dimensions" : ["server"], "type" : "gauge" }, -// "segment/dropQueue/count" : { "dimensions" : ["server"], "type" : "gauge" }, -// "segment/size" : { "dimensions" : ["dataSource"], "type" : "gauge" }, -// "segment/overShadowed/count" : { "dimensions" : [], "type" : "gauge" }, -// -// "segment/max" : { "dimensions" : [], "type" : "gauge"}, -// "segment/used" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge" }, -// "segment/usedPercent" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge", "convertRange" : true }, -// "segment/pendingDelete" : { "dimensions" : [], "type" : "gauge"}, -// -// "jvm/pool/committed" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge" }, -// "jvm/pool/init" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge" }, -// "jvm/pool/max" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge" }, -// "jvm/pool/used" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge" }, -// "jvm/bufferpool/count" : { "dimensions" : ["bufferPoolName"], "type" : "gauge" }, -// "jvm/bufferpool/used" : { "dimensions" : ["bufferPoolName"], "type" : "gauge" }, -// "jvm/bufferpool/capacity" : { "dimensions" : ["bufferPoolName"], "type" : "gauge" }, -// "jvm/mem/init" : { "dimensions" : ["memKind"], "type" : "gauge" }, -// "jvm/mem/max" : { "dimensions" : ["memKind"], "type" : "gauge" }, -// "jvm/mem/used" : { "dimensions" : ["memKind"], "type" : "gauge" }, -// "jvm/mem/committed" : { "dimensions" : ["memKind"], "type" : "gauge" }, -// "jvm/gc/count" : { "dimensions" : ["gcName"], "type" : "count" }, -// "jvm/gc/cpu" : { "dimensions" : ["gcName"], "type" : "timer" }, -// -// "ingest/events/buffered" : { "dimensions" : ["serviceName, bufferCapacity"], "type" : "gauge"}, -// -// "sys/swap/free" : { "dimensions" : [], "type" : "gauge"}, -// "sys/swap/max" : { "dimensions" : [], "type" : "gauge"}, -// "sys/swap/pageIn" : { "dimensions" : [], "type" : "gauge"}, -// "sys/swap/pageOut" : { "dimensions" : [], "type" : "gauge"}, -// "sys/disk/write/count" : { "dimensions" : ["fsDevName"], "type" : "count"}, -// "sys/disk/read/count" : { "dimensions" : ["fsDevName"], "type" : "count"}, -// "sys/disk/write/size" : { "dimensions" : ["fsDevName"], "type" : "count"}, -// "sys/disk/read/size" : { "dimensions" : ["fsDevName"], "type" : "count"}, -// "sys/net/write/size" : { "dimensions" : [], "type" : "count"}, -// "sys/net/read/size" : { "dimensions" : [], "type" : "count"}, -// "sys/fs/used" : { "dimensions" : ["fsDevName", "fsDirName", "fsTypeName", "fsSysTypeName", "fsOptions"], "type" : "gauge"}, -// "sys/fs/max" : { "dimensions" : ["fsDevName", "fsDirName", "fsTypeName", "fsSysTypeName", "fsOptions"], "type" : "gauge"}, -// "sys/mem/used" : { "dimensions" : [], "type" : "gauge"}, -// "sys/mem/max" : { "dimensions" : [], "type" : "gauge"}, -// "sys/storage/used" : { "dimensions" : ["fsDirName"], "type" : "gauge"}, -// "sys/cpu" : { "dimensions" : ["cpuName", "cpuTime"], "type" : "gauge"}, -// -// "coordinator-segment/count" : { "dimensions" : ["dataSource"], "type" : "gauge" }, -// "historical-segment/count" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge" } + .name(formattedName) + .labelNames(dimensions) + .register(); + } else if (Metric.Type.gauge.equals(type)) { + collector = new Gauge.Builder() + .namespace(namespace) + .name(formattedName) + .labelNames(dimensions) + .register(); + } else if (Metric.Type.timer.equals(type)) { + collector = new Histogram.Builder() + .namespace(namespace) + .name(formattedName) + .labelNames(dimensions) + .buckets(.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300) + .register(); + } else { + log.error("Unrecognized metric type [%s]", type); + } + + if (collector != null) { + map.put(name, new DimensionsAndCollector(dimensions, collector)); + } + } } + private Map readMap(String path) + { + try { + InputStream is; + if (Strings.isNullOrEmpty(path)) { + log.info("Using default metric dimension and types"); + is = this.getClass().getClassLoader().getResourceAsStream("defaultMetrics.json"); + } else { + log.info("Using metric dimensions at types at [%s]", path); + is = new FileInputStream(new File(path)); + } + return mapper.readerFor(new TypeReference>() + { + }).readValue(is); + } + catch (IOException e) { + throw new ISE(e, "Failed to parse metric dimensions and types"); + } + } public static class Metric { - private final String[] dimensions; - private final SimpleCollector collector; - - Metric(String[] dimensions, SimpleCollector collector) + public final SortedSet dimensions; + public final Type type; + + @JsonCreator + public Metric( + @JsonProperty("dimensions") SortedSet dimensions, + @JsonProperty("type") Type type + ) { this.dimensions = dimensions; - this.collector = collector; + this.type = type; } - public String[] getDimensions() + public enum Type { - return dimensions; - } - - public SimpleCollector getCollector() - { - return collector; + count, gauge, timer } } - } diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java index 530cd94b2c79..7748c5eb6e5c 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -45,7 +45,7 @@ static PrometheusEmitter of(PrometheusEmitterConfig config) public PrometheusEmitter(PrometheusEmitterConfig config) { - metrics = new Metrics(config.getNamespace()); + metrics = new Metrics(config.getNamespace(), config.getDimensionMapPath()); } @@ -68,7 +68,7 @@ void emitMetric(ServiceMetricEvent metricEvent) Map userDims = metricEvent.getUserDims(); Number value = metricEvent.getValue(); - Metrics.Metric metric = metrics.getByName(name); + DimensionsAndCollector metric = metrics.getByName(name); if (metric != null) { String[] labelValues = new String[metric.getDimensions().length]; String[] labelNames = metric.getDimensions(); diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java index c5065192b135..83315e79dd17 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java @@ -36,14 +36,20 @@ public class PrometheusEmitterConfig @JsonProperty private final String path; + @JsonProperty + @Nullable + private final String dimensionMapPath; + @JsonCreator public PrometheusEmitterConfig( @JsonProperty("namespace") @Nullable String namespace, - @JsonProperty("path") @Nullable String path + @JsonProperty("path") @Nullable String path, + @JsonProperty("dimensionMapPath") @Nullable String dimensionMapPath ) { this.namespace = namespace != null ? namespace : "druid"; this.path = path != null ? path : "/prometheus"; + this.dimensionMapPath = dimensionMapPath; } public String getNamespace() @@ -55,4 +61,9 @@ public String getPath() { return path; } + + public String getDimensionMapPath() + { + return dimensionMapPath; + } } diff --git a/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json new file mode 100644 index 000000000000..ac5077656cca --- /dev/null +++ b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json @@ -0,0 +1,128 @@ +{ + "query/time" : { "dimensions" : ["dataSource", "type"], "type" : "timer"}, + "query/bytes" : { "dimensions" : ["dataSource", "type"], "type" : "count"}, + "query/node/time" : { "dimensions" : ["server"], "type" : "timer"}, + "query/node/ttfb" : { "dimensions" : ["server"], "type" : "timer"}, + "query/node/bytes" : { "dimensions" : ["server"], "type" : "count"}, + "query/node/backpressure": { "dimensions" : ["server"], "type" : "timer"}, + "query/intervalChunk/time" : { "dimensions" : [], "type" : "timer"}, + + "query/segment/time" : { "dimensions" : [], "type" : "timer"}, + "query/wait/time" : { "dimensions" : [], "type" : "timer"}, + "segment/scan/pending" : { "dimensions" : [], "type" : "gauge"}, + "query/segmentAndCache/time" : { "dimensions" : [], "type" : "timer" }, + "query/cpu/time" : { "dimensions" : ["dataSource", "type"], "type" : "timer" }, + + "query/count" : { "dimensions" : [], "type" : "count" }, + "query/success/count" : { "dimensions" : [], "type" : "count" }, + "query/failed/count" : { "dimensions" : [], "type" : "count" }, + "query/interrupted/count" : { "dimensions" : [], "type" : "count" }, + + "query/cache/delta/numEntries" : { "dimensions" : [], "type" : "count" }, + "query/cache/delta/sizeBytes" : { "dimensions" : [], "type" : "count" }, + "query/cache/delta/hits" : { "dimensions" : [], "type" : "count" }, + "query/cache/delta/misses" : { "dimensions" : [], "type" : "count" }, + "query/cache/delta/evictions" : { "dimensions" : [], "type" : "count" }, + "query/cache/delta/hitRate" : { "dimensions" : [], "type" : "count"}, + "query/cache/delta/averageBytes" : { "dimensions" : [], "type" : "count" }, + "query/cache/delta/timeouts" : { "dimensions" : [], "type" : "count" }, + "query/cache/delta/errors" : { "dimensions" : [], "type" : "count" }, + + "query/cache/total/numEntries" : { "dimensions" : [], "type" : "gauge" }, + "query/cache/total/sizeBytes" : { "dimensions" : [], "type" : "gauge" }, + "query/cache/total/hits" : { "dimensions" : [], "type" : "gauge" }, + "query/cache/total/misses" : { "dimensions" : [], "type" : "gauge" }, + "query/cache/total/evictions" : { "dimensions" : [], "type" : "gauge" }, + "query/cache/total/hitRate" : { "dimensions" : [], "type" : "gauge"}, + "query/cache/total/averageBytes" : { "dimensions" : [], "type" : "gauge" }, + "query/cache/total/timeouts" : { "dimensions" : [], "type" : "gauge" }, + "query/cache/total/errors" : { "dimensions" : [], "type" : "gauge" }, + + "ingest/events/thrownAway" : { "dimensions" : ["dataSource"], "type" : "count" }, + "ingest/events/unparseable" : { "dimensions" : ["dataSource"], "type" : "count" }, + "ingest/events/duplicate" : { "dimensions" : ["dataSource"], "type" : "count" }, + "ingest/events/processed" : { "dimensions" : ["dataSource"], "type" : "count" }, + "ingest/events/messageGap" : { "dimensions" : ["dataSource"], "type" : "gauge" }, + "ingest/rows/output" : { "dimensions" : ["dataSource"], "type" : "count" }, + "ingest/persists/count" : { "dimensions" : ["dataSource"], "type" : "count" }, + "ingest/persists/time" : { "dimensions" : ["dataSource"], "type" : "timer" }, + "ingest/persists/cpu" : { "dimensions" : ["dataSource"], "type" : "timer" }, + "ingest/persists/backPressure" : { "dimensions" : ["dataSource"], "type" : "gauge" }, + "ingest/persists/failed" : { "dimensions" : ["dataSource"], "type" : "count" }, + "ingest/handoff/failed" : { "dimensions" : ["dataSource"], "type" : "count" }, + "ingest/merge/time" : { "dimensions" : ["dataSource"], "type" : "timer" }, + "ingest/merge/cpu" : { "dimensions" : ["dataSource"], "type" : "timer" }, + + "ingest/kafka/lag" : { "dimensions" : ["dataSource"], "type" : "gauge" }, + "ingest/kafka/maxLag" : { "dimensions" : ["dataSource"], "type" : "gauge" }, + "ingest/kafka/avgLag" : { "dimensions" : ["dataSource"], "type" : "gauge" }, + + "task/success/count" : { "dimensions" : ["dataSource"], "type" : "count" }, + "task/failed/count" : { "dimensions" : ["dataSource"], "type" : "count" }, + "task/running/count" : { "dimensions" : ["dataSource"], "type" : "count" }, + "task/pending/count" : { "dimensions" : ["dataSource"], "type" : "count" }, + "task/waiting/count" : { "dimensions" : ["dataSource"], "type" : "count" }, + + "task/run/time" : { "dimensions" : ["dataSource", "taskType"], "type" : "timer" }, + "segment/added/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count" }, + "segment/moved/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count" }, + "segment/nuked/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count" }, + + "segment/assigned/count" : { "dimensions" : ["tier"], "type" : "count" }, + "segment/moved/count" : { "dimensions" : ["tier"], "type" : "count" }, + "segment/dropped/count" : { "dimensions" : ["tier"], "type" : "count" }, + "segment/deleted/count" : { "dimensions" : ["tier"], "type" : "count" }, + "segment/unneeded/count" : { "dimensions" : ["tier"], "type" : "count" }, + "segment/unavailable/count" : { "dimensions" : ["dataSource"], "type" : "count" }, + "segment/underReplicated/count" : { "dimensions" : ["dataSource", "tier"], "type" : "count" }, + "segment/cost/raw" : { "dimensions" : ["tier"], "type" : "count" }, + "segment/cost/normalization" : { "dimensions" : ["tier"], "type" : "count" }, + "segment/cost/normalized" : { "dimensions" : ["tier"], "type" : "count" }, + "segment/loadQueue/size" : { "dimensions" : ["server"], "type" : "gauge" }, + "segment/loadQueue/failed" : { "dimensions" : ["server"], "type" : "gauge" }, + "segment/loadQueue/count" : { "dimensions" : ["server"], "type" : "gauge" }, + "segment/dropQueue/count" : { "dimensions" : ["server"], "type" : "gauge" }, + "segment/size" : { "dimensions" : ["dataSource"], "type" : "gauge" }, + "segment/overShadowed/count" : { "dimensions" : [], "type" : "gauge" }, + + "segment/max" : { "dimensions" : [], "type" : "gauge"}, + "segment/used" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge" }, + "segment/usedPercent" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge"}, + "segment/pendingDelete" : { "dimensions" : [], "type" : "gauge"}, + + "jvm/pool/committed" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge" }, + "jvm/pool/init" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge" }, + "jvm/pool/max" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge" }, + "jvm/pool/used" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge" }, + "jvm/bufferpool/count" : { "dimensions" : ["bufferPoolName"], "type" : "gauge" }, + "jvm/bufferpool/used" : { "dimensions" : ["bufferPoolName"], "type" : "gauge" }, + "jvm/bufferpool/capacity" : { "dimensions" : ["bufferPoolName"], "type" : "gauge" }, + "jvm/mem/init" : { "dimensions" : ["memKind"], "type" : "gauge" }, + "jvm/mem/max" : { "dimensions" : ["memKind"], "type" : "gauge" }, + "jvm/mem/used" : { "dimensions" : ["memKind"], "type" : "gauge" }, + "jvm/mem/committed" : { "dimensions" : ["memKind"], "type" : "gauge" }, + "jvm/gc/count" : { "dimensions" : ["gcName"], "type" : "count" }, + "jvm/gc/cpu" : { "dimensions" : ["gcName"], "type" : "timer" }, + + "ingest/events/buffered" : { "dimensions" : ["serviceName, bufferCapacity"], "type" : "gauge"}, + + "sys/swap/free" : { "dimensions" : [], "type" : "gauge"}, + "sys/swap/max" : { "dimensions" : [], "type" : "gauge"}, + "sys/swap/pageIn" : { "dimensions" : [], "type" : "gauge"}, + "sys/swap/pageOut" : { "dimensions" : [], "type" : "gauge"}, + "sys/disk/write/count" : { "dimensions" : ["fsDevName"], "type" : "count"}, + "sys/disk/read/count" : { "dimensions" : ["fsDevName"], "type" : "count"}, + "sys/disk/write/size" : { "dimensions" : ["fsDevName"], "type" : "count"}, + "sys/disk/read/size" : { "dimensions" : ["fsDevName"], "type" : "count"}, + "sys/net/write/size" : { "dimensions" : [], "type" : "count"}, + "sys/net/read/size" : { "dimensions" : [], "type" : "count"}, + "sys/fs/used" : { "dimensions" : ["fsDevName", "fsDirName", "fsTypeName", "fsSysTypeName", "fsOptions"], "type" : "gauge"}, + "sys/fs/max" : { "dimensions" : ["fsDevName", "fsDirName", "fsTypeName", "fsSysTypeName", "fsOptions"], "type" : "gauge"}, + "sys/mem/used" : { "dimensions" : [], "type" : "gauge"}, + "sys/mem/max" : { "dimensions" : [], "type" : "gauge"}, + "sys/storage/used" : { "dimensions" : ["fsDirName"], "type" : "gauge"}, + "sys/cpu" : { "dimensions" : ["cpuName", "cpuTime"], "type" : "gauge"}, + + "coordinator-segment/count" : { "dimensions" : ["dataSource"], "type" : "gauge" }, + "historical-segment/count" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge" } +} From c2a71319eefe72e33571762f0b0a65bf85b99e49 Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Thu, 3 Oct 2019 14:44:29 -0700 Subject: [PATCH 08/42] use prometheus HTTPServer instead of hooking into Jetty server --- extensions-contrib/prometheus-emitter/pom.xml | 6 ++++++ .../druid/emitter/prometheus/Metrics.java | 5 +++-- .../emitter/prometheus/PrometheusEmitter.java | 16 ++++++++++++++ .../prometheus/PrometheusEmitterConfig.java | 21 ++++++++++--------- .../prometheus/PrometheusEmitterModule.java | 11 ++-------- 5 files changed, 38 insertions(+), 21 deletions(-) diff --git a/extensions-contrib/prometheus-emitter/pom.xml b/extensions-contrib/prometheus-emitter/pom.xml index 9be9aba4bb45..963d2f358aea 100644 --- a/extensions-contrib/prometheus-emitter/pom.xml +++ b/extensions-contrib/prometheus-emitter/pom.xml @@ -45,6 +45,12 @@ simpleclient 0.7.0 + + io.prometheus + simpleclient_httpserver + 0.7.0 + + com.google.code.findbugs jsr305 diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java index 7d8e58688795..f6a04bc407ea 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java @@ -55,8 +55,9 @@ public DimensionsAndCollector getByName(String name) public Metrics(String namespace, String path) { Map metrics = readMap(path); - for (String name : metrics.keySet()) { - Metric metric = metrics.get(name); + for (Map.Entry entry : metrics.entrySet()) { + String name = entry.getKey(); + Metric metric = entry.getValue(); Metric.Type type = metric.type; String[] dimensions = metric.dimensions.toArray(new String[0]); String formattedName = StringUtils.replaceChar(StringUtils.toLowerCase(name), '/', "_"); diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java index 7748c5eb6e5c..f37af98f80e7 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -23,10 +23,13 @@ import io.prometheus.client.Counter; import io.prometheus.client.Gauge; import io.prometheus.client.Histogram; +import io.prometheus.client.exporter.HTTPServer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.core.Emitter; import org.apache.druid.java.util.emitter.core.Event; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; + +import java.io.IOException; import java.util.Map; /** @@ -37,6 +40,9 @@ public class PrometheusEmitter implements Emitter private static final Logger log = new Logger(PrometheusEmitter.class); private final Metrics metrics; + private final PrometheusEmitterConfig config; + + private HTTPServer server; static PrometheusEmitter of(PrometheusEmitterConfig config) { @@ -45,6 +51,7 @@ static PrometheusEmitter of(PrometheusEmitterConfig config) public PrometheusEmitter(PrometheusEmitterConfig config) { + this.config = config; metrics = new Metrics(config.getNamespace(), config.getDimensionMapPath()); } @@ -52,6 +59,12 @@ public PrometheusEmitter(PrometheusEmitterConfig config) @Override public void start() { + try { + server = new HTTPServer(config.getPort()); + } + catch (IOException e) { + log.error(e, "Unable to start prometheus HTTPServer"); + } } @Override @@ -98,5 +111,8 @@ public void flush() @Override public void close() { + if (server != null) { + server.stop(); + } } } diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java index 83315e79dd17..e988b951782d 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; import javax.annotation.Nullable; @@ -33,23 +34,23 @@ public class PrometheusEmitterConfig @JsonProperty private final String namespace; - @JsonProperty - private final String path; - @JsonProperty @Nullable private final String dimensionMapPath; + @JsonProperty + private final int port; + @JsonCreator public PrometheusEmitterConfig( @JsonProperty("namespace") @Nullable String namespace, - @JsonProperty("path") @Nullable String path, - @JsonProperty("dimensionMapPath") @Nullable String dimensionMapPath + @JsonProperty("dimensionMapPath") @Nullable String dimensionMapPath, + @JsonProperty("port") int port ) { this.namespace = namespace != null ? namespace : "druid"; - this.path = path != null ? path : "/prometheus"; this.dimensionMapPath = dimensionMapPath; + this.port = Preconditions.checkNotNull(port, "Prometheus server port cannot be null."); } public String getNamespace() @@ -57,13 +58,13 @@ public String getNamespace() return namespace; } - public String getPath() + public String getDimensionMapPath() { - return path; + return dimensionMapPath; } - public String getDimensionMapPath() + public int getPort() { - return dimensionMapPath; + return port; } } diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java index 42ff7e7b2435..6cc2b06c9ead 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java @@ -24,15 +24,12 @@ import com.google.inject.Binder; import com.google.inject.Provides; import com.google.inject.name.Named; -import io.prometheus.client.exporter.MetricsServlet; import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.initialization.DruidModule; import org.apache.druid.java.util.emitter.core.Emitter; -import org.eclipse.jetty.server.Server; -import org.eclipse.jetty.servlet.ServletContextHandler; -import org.eclipse.jetty.servlet.ServletHolder; +import java.io.IOException; import java.util.Collections; import java.util.List; @@ -59,12 +56,8 @@ public void configure(Binder binder) @Provides @ManageLifecycle @Named(EMITTER_TYPE) - public Emitter getEmitter(PrometheusEmitterConfig config, Server server) + public Emitter getEmitter(PrometheusEmitterConfig config) throws IOException { - ServletContextHandler context = new ServletContextHandler(); - context.setContextPath("/"); - server.setHandler(context); - context.addServlet(new ServletHolder(new MetricsServlet()), config.getPath()); return PrometheusEmitter.of(config); } } From 5a91cc0befcf8de3a580d6b63da4a63e724af912 Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Thu, 3 Oct 2019 15:09:20 -0700 Subject: [PATCH 09/42] temporary empty help string --- .../main/java/org/apache/druid/emitter/prometheus/Metrics.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java index f6a04bc407ea..bc987a134f71 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java @@ -67,12 +67,14 @@ public Metrics(String namespace, String path) .namespace(namespace) .name(formattedName) .labelNames(dimensions) + .help("") .register(); } else if (Metric.Type.gauge.equals(type)) { collector = new Gauge.Builder() .namespace(namespace) .name(formattedName) .labelNames(dimensions) + .help("") .register(); } else if (Metric.Type.timer.equals(type)) { collector = new Histogram.Builder() @@ -80,6 +82,7 @@ public Metrics(String namespace, String path) .name(formattedName) .labelNames(dimensions) .buckets(.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300) + .help("") .register(); } else { log.error("Unrecognized metric type [%s]", type); From a7851af284c28d3d13daf62b7fd1788220e696ca Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Thu, 3 Oct 2019 15:25:21 -0700 Subject: [PATCH 10/42] temporary non-empty help. fix incorrect dimension value in JSON (also updated statsd json) --- .../java/org/apache/druid/emitter/prometheus/Metrics.java | 6 +++--- .../src/main/resources/defaultMetrics.json | 2 +- .../src/main/resources/defaultMetricDimensions.json | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java index bc987a134f71..a4f0ce4e4899 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java @@ -67,14 +67,14 @@ public Metrics(String namespace, String path) .namespace(namespace) .name(formattedName) .labelNames(dimensions) - .help("") + .help("todo") .register(); } else if (Metric.Type.gauge.equals(type)) { collector = new Gauge.Builder() .namespace(namespace) .name(formattedName) .labelNames(dimensions) - .help("") + .help("todo") .register(); } else if (Metric.Type.timer.equals(type)) { collector = new Histogram.Builder() @@ -82,7 +82,7 @@ public Metrics(String namespace, String path) .name(formattedName) .labelNames(dimensions) .buckets(.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300) - .help("") + .help("todo") .register(); } else { log.error("Unrecognized metric type [%s]", type); diff --git a/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json index ac5077656cca..2697ce3359f2 100644 --- a/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json +++ b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json @@ -104,7 +104,7 @@ "jvm/gc/count" : { "dimensions" : ["gcName"], "type" : "count" }, "jvm/gc/cpu" : { "dimensions" : ["gcName"], "type" : "timer" }, - "ingest/events/buffered" : { "dimensions" : ["serviceName, bufferCapacity"], "type" : "gauge"}, + "ingest/events/buffered" : { "dimensions" : ["serviceName", "bufferCapacity"], "type" : "gauge"}, "sys/swap/free" : { "dimensions" : [], "type" : "gauge"}, "sys/swap/max" : { "dimensions" : [], "type" : "gauge"}, diff --git a/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json b/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json index 1b17a4c1b6d4..298b440e5209 100644 --- a/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json +++ b/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json @@ -110,7 +110,7 @@ "jvm/gc/count" : { "dimensions" : ["gcName", "gcGen"], "type" : "count" }, "jvm/gc/cpu" : { "dimensions" : ["gcName", "gcGen"], "type" : "count" }, - "ingest/events/buffered" : { "dimensions" : ["serviceName, bufferCapacity"], "type" : "gauge"}, + "ingest/events/buffered" : { "dimensions" : ["serviceName", "bufferCapacity"], "type" : "gauge"}, "sys/swap/free" : { "dimensions" : [], "type" : "gauge"}, "sys/swap/max" : { "dimensions" : [], "type" : "gauge"}, From a2136385527aaa6b61f6e61221a2327da628cc48 Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Thu, 3 Oct 2019 16:44:19 -0700 Subject: [PATCH 11/42] added full help text. added metric conversion factor for timers that are not using seconds. Correct metric dimension name in documentation --- docs/operations/metrics.md | 2 +- .../prometheus/DimensionsAndCollector.java | 9 +- .../druid/emitter/prometheus/Metrics.java | 26 ++- .../emitter/prometheus/PrometheusEmitter.java | 7 +- .../src/main/resources/defaultMetrics.json | 220 +++++++++--------- 5 files changed, 142 insertions(+), 122 deletions(-) diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md index 9a1d2c7e4983..cf7f00754aba 100644 --- a/docs/operations/metrics.md +++ b/docs/operations/metrics.md @@ -286,7 +286,7 @@ These metrics are only available if the JVMMonitor module is included. |`jvm/pool/used`|Pool used.|poolKind, poolName.|< max pool| |`jvm/bufferpool/count`|Bufferpool count.|bufferpoolName.|Varies.| |`jvm/bufferpool/used`|Bufferpool used.|bufferpoolName.|close to capacity| -|`jvm/bufferpool/capacity`|Bufferpool capacity.|bufferpoolName.|Varies.| +|`jvm/bufferpool/capacity`|Bufferpool capacity.|bufferPoolName.|Varies.| |`jvm/mem/init`|Initial memory.|memKind.|Varies.| |`jvm/mem/max`|Max memory.|memKind.|Varies.| |`jvm/mem/used`|Used memory.|memKind.|< max memory| diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/DimensionsAndCollector.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/DimensionsAndCollector.java index 64279d9d0939..ede4977aeee6 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/DimensionsAndCollector.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/DimensionsAndCollector.java @@ -25,11 +25,13 @@ public class DimensionsAndCollector { private final String[] dimensions; private final SimpleCollector collector; + private final double conversionFactor; - DimensionsAndCollector(String[] dimensions, SimpleCollector collector) + DimensionsAndCollector(String[] dimensions, SimpleCollector collector, double conversionFactor) { this.dimensions = dimensions; this.collector = collector; + this.conversionFactor = conversionFactor; } public String[] getDimensions() @@ -41,4 +43,9 @@ public SimpleCollector getCollector() { return collector; } + + public double getConversionFactor() + { + return conversionFactor; + } } diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java index a4f0ce4e4899..61fd2e1384cf 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java @@ -47,9 +47,15 @@ public class Metrics private final Map map = new HashMap<>(); private final ObjectMapper mapper = new ObjectMapper(); - public DimensionsAndCollector getByName(String name) + public DimensionsAndCollector getByName(String name, String service) { - return map.get(name); + if (map.containsKey(name)) { + return map.get(name); + } else if (map.containsKey(service + "_" + name)) { + return map.get(service + "_" + name); + } else { + return null; + } } public Metrics(String namespace, String path) @@ -67,14 +73,14 @@ public Metrics(String namespace, String path) .namespace(namespace) .name(formattedName) .labelNames(dimensions) - .help("todo") + .help(metric.help) .register(); } else if (Metric.Type.gauge.equals(type)) { collector = new Gauge.Builder() .namespace(namespace) .name(formattedName) .labelNames(dimensions) - .help("todo") + .help(metric.help) .register(); } else if (Metric.Type.timer.equals(type)) { collector = new Histogram.Builder() @@ -82,14 +88,14 @@ public Metrics(String namespace, String path) .name(formattedName) .labelNames(dimensions) .buckets(.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300) - .help("todo") + .help(metric.help) .register(); } else { log.error("Unrecognized metric type [%s]", type); } if (collector != null) { - map.put(name, new DimensionsAndCollector(dimensions, collector)); + map.put(name, new DimensionsAndCollector(dimensions, collector, metric.conversionFactor)); } } @@ -119,15 +125,21 @@ public static class Metric { public final SortedSet dimensions; public final Type type; + public final String help; + public final double conversionFactor; @JsonCreator public Metric( @JsonProperty("dimensions") SortedSet dimensions, - @JsonProperty("type") Type type + @JsonProperty("type") Type type, + @JsonProperty("help") String help, + @JsonProperty("conversionFactor") double conversionFactor ) { this.dimensions = dimensions; this.type = type; + this.help = help; + this.conversionFactor = conversionFactor; } public enum Type diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java index f37af98f80e7..c7fe221a2ee6 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -78,10 +78,11 @@ public void emit(Event event) void emitMetric(ServiceMetricEvent metricEvent) { String name = metricEvent.getMetric(); + String service = metricEvent.getService(); Map userDims = metricEvent.getUserDims(); Number value = metricEvent.getValue(); - DimensionsAndCollector metric = metrics.getByName(name); + DimensionsAndCollector metric = metrics.getByName(name, service); if (metric != null) { String[] labelValues = new String[metric.getDimensions().length]; String[] labelNames = metric.getDimensions(); @@ -96,9 +97,9 @@ void emitMetric(ServiceMetricEvent metricEvent) } else if (metric.getCollector() instanceof Gauge) { ((Gauge) metric.getCollector()).labels(labelValues).set(value.doubleValue()); } else if (metric.getCollector() instanceof Histogram) { - ((Histogram) metric.getCollector()).labels(labelValues).observe(value.doubleValue()); + ((Histogram) metric.getCollector()).labels(labelValues).observe(value.doubleValue() / metric.getConversionFactor()); } else { - //TODO + log.error("Unrecognized metric type [%s]", metric.getCollector().getClass()); } } } diff --git a/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json index 2697ce3359f2..51468a0d764f 100644 --- a/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json +++ b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json @@ -1,128 +1,128 @@ { - "query/time" : { "dimensions" : ["dataSource", "type"], "type" : "timer"}, - "query/bytes" : { "dimensions" : ["dataSource", "type"], "type" : "count"}, - "query/node/time" : { "dimensions" : ["server"], "type" : "timer"}, - "query/node/ttfb" : { "dimensions" : ["server"], "type" : "timer"}, - "query/node/bytes" : { "dimensions" : ["server"], "type" : "count"}, - "query/node/backpressure": { "dimensions" : ["server"], "type" : "timer"}, - "query/intervalChunk/time" : { "dimensions" : [], "type" : "timer"}, + "query/time" : { "dimensions" : ["dataSource", "type"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to complete a query."}, + "query/bytes" : { "dimensions" : ["dataSource", "type"], "type" : "count", "help": "Number of bytes returned in query response."}, + "query/node/time" : { "dimensions" : ["server"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to query individual historical/realtime processes."}, + "query/node/ttfb" : { "dimensions" : ["server"], "type" : "timer", "help": "Time to first byte. Seconds elapsed until Broker starts receiving the response from individual historical/realtime processes."}, + "query/node/bytes" : { "dimensions" : ["server"], "type" : "count", "help": "Number of bytes returned from querying individual historical/realtime processes."}, + "query/node/backpressure": { "dimensions" : ["server"], "type" : "timer", "help": "Seconds that the channel to this process has spent suspended due to backpressure."}, + "query/intervalChunk/time" : { "dimensions" : [], "type" : "timer", "conversionFactor": 1000.0, "help": "Only emitted if interval chunking is enabled. Milliseconds required to query an interval chunk. This metric is deprecated and will be removed in the future because interval chunking is deprecated."}, - "query/segment/time" : { "dimensions" : [], "type" : "timer"}, - "query/wait/time" : { "dimensions" : [], "type" : "timer"}, - "segment/scan/pending" : { "dimensions" : [], "type" : "gauge"}, - "query/segmentAndCache/time" : { "dimensions" : [], "type" : "timer" }, - "query/cpu/time" : { "dimensions" : ["dataSource", "type"], "type" : "timer" }, + "query/segment/time" : { "dimensions" : [], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to query individual segment. Includes time to page in the segment from disk."}, + "query/wait/time" : { "dimensions" : [], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds spent waiting for a segment to be scanned."}, + "segment/scan/pending" : { "dimensions" : [], "type" : "gauge", "help": "Number of segments in queue waiting to be scanned."}, + "query/segmentAndCache/time" : { "dimensions" : [], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to query individual segment or hit the cache (if it is enabled on the Historical process)."}, + "query/cpu/time" : { "dimensions" : ["dataSource", "type"], "type" : "timer", "conversionFactor": "1000000", "help": "Seconds of CPU time taken to complete a query"}, - "query/count" : { "dimensions" : [], "type" : "count" }, - "query/success/count" : { "dimensions" : [], "type" : "count" }, - "query/failed/count" : { "dimensions" : [], "type" : "count" }, - "query/interrupted/count" : { "dimensions" : [], "type" : "count" }, + "query/count" : { "dimensions" : [], "type" : "count", "help": "Number of total queries" }, + "query/success/count" : { "dimensions" : [], "type" : "count", "help": "Number of queries successfully processed"}, + "query/failed/count" : { "dimensions" : [], "type" : "count", "help": "Number of failed queries"}, + "query/interrupted/count" : { "dimensions" : [], "type" : "count", "help": "Number of queries interrupted due to cancellation or timeout"}, - "query/cache/delta/numEntries" : { "dimensions" : [], "type" : "count" }, - "query/cache/delta/sizeBytes" : { "dimensions" : [], "type" : "count" }, - "query/cache/delta/hits" : { "dimensions" : [], "type" : "count" }, - "query/cache/delta/misses" : { "dimensions" : [], "type" : "count" }, - "query/cache/delta/evictions" : { "dimensions" : [], "type" : "count" }, - "query/cache/delta/hitRate" : { "dimensions" : [], "type" : "count"}, - "query/cache/delta/averageBytes" : { "dimensions" : [], "type" : "count" }, - "query/cache/delta/timeouts" : { "dimensions" : [], "type" : "count" }, - "query/cache/delta/errors" : { "dimensions" : [], "type" : "count" }, + "query/cache/delta/numEntries" : { "dimensions" : [], "type" : "count", "help": "Number of entries in cache"}, + "query/cache/delta/sizeBytes" : { "dimensions" : [], "type" : "count", "help": "Size of cache in bytes."}, + "query/cache/delta/hits" : { "dimensions" : [], "type" : "count", "help": "Number of cache hits."}, + "query/cache/delta/misses" : { "dimensions" : [], "type" : "count", "help": "Number of cache misses."}, + "query/cache/delta/evictions" : { "dimensions" : [], "type" : "count", "help": "Number of cache evictions."}, + "query/cache/delta/hitRate" : { "dimensions" : [], "type" : "count","help": "Cache hit rate."}, + "query/cache/delta/averageBytes" : { "dimensions" : [], "type" : "count", "help": "Average size of record in bytes"}, + "query/cache/delta/timeouts" : { "dimensions" : [], "type" : "count", "help": "Number of cache timeouts"}, + "query/cache/delta/errors" : { "dimensions" : [], "type" : "count", "help": "Number of cache errors."}, - "query/cache/total/numEntries" : { "dimensions" : [], "type" : "gauge" }, - "query/cache/total/sizeBytes" : { "dimensions" : [], "type" : "gauge" }, - "query/cache/total/hits" : { "dimensions" : [], "type" : "gauge" }, - "query/cache/total/misses" : { "dimensions" : [], "type" : "gauge" }, - "query/cache/total/evictions" : { "dimensions" : [], "type" : "gauge" }, - "query/cache/total/hitRate" : { "dimensions" : [], "type" : "gauge"}, - "query/cache/total/averageBytes" : { "dimensions" : [], "type" : "gauge" }, - "query/cache/total/timeouts" : { "dimensions" : [], "type" : "gauge" }, - "query/cache/total/errors" : { "dimensions" : [], "type" : "gauge" }, + "query/cache/total/numEntries" : { "dimensions" : [], "type" : "gauge","help": "Total number of entries in cache" }, + "query/cache/total/sizeBytes" : { "dimensions" : [], "type" : "gauge", "help": "Total size of cache in bytes."}, + "query/cache/total/hits" : { "dimensions" : [], "type" : "gauge", "help": "Total number of cache hits."}, + "query/cache/total/misses" : { "dimensions" : [], "type" : "gauge", "help": "Total number of cache misses." }, + "query/cache/total/evictions" : { "dimensions" : [], "type" : "gauge", "help": "Total number of cache evictions."}, + "query/cache/total/hitRate" : { "dimensions" : [], "type" : "gauge", "help": "Total cache hit rate"}, + "query/cache/total/averageBytes" : { "dimensions" : [], "type" : "gauge", "help": "Total average record size in bytes"}, + "query/cache/total/timeouts" : { "dimensions" : [], "type" : "gauge", "help": "Total number of cache timeouts"}, + "query/cache/total/errors" : { "dimensions" : [], "type" : "gauge", "help": "Total number of cache errors" }, - "ingest/events/thrownAway" : { "dimensions" : ["dataSource"], "type" : "count" }, - "ingest/events/unparseable" : { "dimensions" : ["dataSource"], "type" : "count" }, - "ingest/events/duplicate" : { "dimensions" : ["dataSource"], "type" : "count" }, - "ingest/events/processed" : { "dimensions" : ["dataSource"], "type" : "count" }, - "ingest/events/messageGap" : { "dimensions" : ["dataSource"], "type" : "gauge" }, - "ingest/rows/output" : { "dimensions" : ["dataSource"], "type" : "count" }, - "ingest/persists/count" : { "dimensions" : ["dataSource"], "type" : "count" }, - "ingest/persists/time" : { "dimensions" : ["dataSource"], "type" : "timer" }, - "ingest/persists/cpu" : { "dimensions" : ["dataSource"], "type" : "timer" }, - "ingest/persists/backPressure" : { "dimensions" : ["dataSource"], "type" : "gauge" }, - "ingest/persists/failed" : { "dimensions" : ["dataSource"], "type" : "count" }, - "ingest/handoff/failed" : { "dimensions" : ["dataSource"], "type" : "count" }, - "ingest/merge/time" : { "dimensions" : ["dataSource"], "type" : "timer" }, - "ingest/merge/cpu" : { "dimensions" : ["dataSource"], "type" : "timer" }, + "ingest/events/thrownAway" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events rejected because they are outside the windowPeriod."}, + "ingest/events/unparseable" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events rejected because the events are unparseable." }, + "ingest/events/duplicate" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events rejected because the events are duplicated."}, + "ingest/events/processed" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events successfully processed per emission period." }, + "ingest/events/messageGap" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Time gap between the data time in event and current system time."}, + "ingest/rows/output" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of Druid rows persisted."}, + "ingest/persists/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of times persist occurred." }, + "ingest/persists/time" : { "dimensions" : ["dataSource"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds spent doing intermediate persist."}, + "ingest/persists/cpu" : { "dimensions" : ["dataSource"], "type" : "timer", "conversionFactor": 1000000000.0, "help": "Cpu time in Seconds spent on doing intermediate persist." }, + "ingest/persists/backPressure" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Seconds spent creating persist tasks and blocking waiting for them to finish." }, + "ingest/persists/failed" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of persists that failed." }, + "ingest/handoff/failed" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of handoffs that failed." }, + "ingest/merge/time" : { "dimensions" : ["dataSource"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds spent merging intermediate segments" }, + "ingest/merge/cpu" : { "dimensions" : ["dataSource"], "type" : "timer", "conversionFactor": 1000000000.0, "help": "Cpu time in Seconds spent on merging intermediate segments."}, - "ingest/kafka/lag" : { "dimensions" : ["dataSource"], "type" : "gauge" }, - "ingest/kafka/maxLag" : { "dimensions" : ["dataSource"], "type" : "gauge" }, - "ingest/kafka/avgLag" : { "dimensions" : ["dataSource"], "type" : "gauge" }, + "ingest/kafka/lag" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Total lag between the offsets consumed by the Kafka indexing tasks and latest offsets in Kafka brokers across all partitions. Minimum emission period for this metric is a minute."}, + "ingest/kafka/maxLag" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Max lag between the offsets consumed by the Kafka indexing tasks and latest offsets in Kafka brokers across all partitions. Minimum emission period for this metric is a minute."}, + "ingest/kafka/avgLag" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Average lag between the offsets consumed by the Kafka indexing tasks and latest offsets in Kafka brokers across all partitions. Minimum emission period for this metric is a minute."}, - "task/success/count" : { "dimensions" : ["dataSource"], "type" : "count" }, - "task/failed/count" : { "dimensions" : ["dataSource"], "type" : "count" }, - "task/running/count" : { "dimensions" : ["dataSource"], "type" : "count" }, - "task/pending/count" : { "dimensions" : ["dataSource"], "type" : "count" }, - "task/waiting/count" : { "dimensions" : ["dataSource"], "type" : "count" }, + "task/success/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of successful tasks per emission period. This metric is only available if the TaskCountStatsMonitor module is included."}, + "task/failed/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of failed tasks per emission period. This metric is only available if the TaskCountStatsMonitor module is included."}, + "task/running/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of current running tasks. This metric is only available if the TaskCountStatsMonitor module is included."}, + "task/pending/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of current pending tasks. This metric is only available if the TaskCountStatsMonitor module is included."}, + "task/waiting/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of current waiting tasks. This metric is only available if the TaskCountStatsMonitor module is included."}, + "task/run/time" : { "dimensions" : ["dataSource", "taskType"], "type" : "timer", "conversionFactor": 1000.0, "help": "Milliseconds taken to run a task."}, - "task/run/time" : { "dimensions" : ["dataSource", "taskType"], "type" : "timer" }, - "segment/added/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count" }, - "segment/moved/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count" }, - "segment/nuked/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count" }, + "segment/added/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count", "help": "Size in bytes of new segments created." }, + "segment/moved/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count", "help": "Size in bytes of segments moved/archived via the Move Task." }, + "segment/nuked/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count", "help": "Size in bytes of segments deleted via the Kill Task." }, - "segment/assigned/count" : { "dimensions" : ["tier"], "type" : "count" }, - "segment/moved/count" : { "dimensions" : ["tier"], "type" : "count" }, - "segment/dropped/count" : { "dimensions" : ["tier"], "type" : "count" }, - "segment/deleted/count" : { "dimensions" : ["tier"], "type" : "count" }, - "segment/unneeded/count" : { "dimensions" : ["tier"], "type" : "count" }, - "segment/unavailable/count" : { "dimensions" : ["dataSource"], "type" : "count" }, - "segment/underReplicated/count" : { "dimensions" : ["dataSource", "tier"], "type" : "count" }, - "segment/cost/raw" : { "dimensions" : ["tier"], "type" : "count" }, - "segment/cost/normalization" : { "dimensions" : ["tier"], "type" : "count" }, - "segment/cost/normalized" : { "dimensions" : ["tier"], "type" : "count" }, - "segment/loadQueue/size" : { "dimensions" : ["server"], "type" : "gauge" }, - "segment/loadQueue/failed" : { "dimensions" : ["server"], "type" : "gauge" }, - "segment/loadQueue/count" : { "dimensions" : ["server"], "type" : "gauge" }, - "segment/dropQueue/count" : { "dimensions" : ["server"], "type" : "gauge" }, - "segment/size" : { "dimensions" : ["dataSource"], "type" : "gauge" }, - "segment/overShadowed/count" : { "dimensions" : [], "type" : "gauge" }, + "segment/assigned/count" : { "dimensions" : ["tier"], "type" : "count", "help": "Number of segments assigned to be loaded in the cluster."}, + "segment/moved/count" : { "dimensions" : ["tier"], "type" : "count", "help": "Number of segments moved in the cluster." }, + "segment/dropped/count" : { "dimensions" : ["tier"], "type" : "count", "help": "Number of segments dropped due to being overshadowed." }, + "segment/deleted/count" : { "dimensions" : ["tier"], "type" : "count", "help": "Number of segments dropped due to rules."}, + "segment/unneeded/count" : { "dimensions" : ["tier"], "type" : "count", "help": "Number of segments dropped due to being marked as unused."}, + "segment/unavailable/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of segments (not including replicas) left to load until segments that should be loaded in the cluster are available for queries."}, + "segment/underReplicated/count" : { "dimensions" : ["dataSource", "tier"], "type" : "count", "help": "Number of segments (including replicas) left to load until segments that should be loaded in the cluster are available for queries."}, + "segment/cost/raw" : { "dimensions" : ["tier"], "type" : "count", "help": "Used in cost balancing. The raw cost of hosting segments."}, + "segment/cost/normalization" : { "dimensions" : ["tier"], "type" : "count", "help": "Used in cost balancing. The normalization of hosting segments."}, + "segment/cost/normalized" : { "dimensions" : ["tier"], "type" : "count", "help": "Used in cost balancing. The normalized cost of hosting segments."}, + "segment/loadQueue/size" : { "dimensions" : ["server"], "type" : "gauge", "help": "Size in bytes of segments to load."}, + "segment/loadQueue/failed" : { "dimensions" : ["server"], "type" : "gauge", "help": "Number of segments that failed to load."}, + "segment/loadQueue/count" : { "dimensions" : ["server"], "type" : "gauge", "help": "Number of segments to load."}, + "segment/dropQueue/count" : { "dimensions" : ["server"], "type" : "gauge", "help": "Number of segments to drop."}, + "segment/size" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Size in bytes of available segments."}, + "segment/overShadowed/count" : { "dimensions" : [], "type" : "gauge", "help": "Number of overShadowed segments."}, - "segment/max" : { "dimensions" : [], "type" : "gauge"}, - "segment/used" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge" }, - "segment/usedPercent" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge"}, - "segment/pendingDelete" : { "dimensions" : [], "type" : "gauge"}, + "segment/max" : { "dimensions" : [], "type" : "gauge", "help": "Maximum byte limit available for segments."}, + "segment/used" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge", "help": "Bytes used for served segments."}, + "segment/usedPercent" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge", "help": "Percentage of space used by served segments."}, + "segment/pendingDelete" : { "dimensions" : [], "type" : "gauge", "help": "On-disk size in bytes of segments that are waiting to be cleared out"}, - "jvm/pool/committed" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge" }, - "jvm/pool/init" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge" }, - "jvm/pool/max" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge" }, - "jvm/pool/used" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge" }, - "jvm/bufferpool/count" : { "dimensions" : ["bufferPoolName"], "type" : "gauge" }, - "jvm/bufferpool/used" : { "dimensions" : ["bufferPoolName"], "type" : "gauge" }, - "jvm/bufferpool/capacity" : { "dimensions" : ["bufferPoolName"], "type" : "gauge" }, - "jvm/mem/init" : { "dimensions" : ["memKind"], "type" : "gauge" }, - "jvm/mem/max" : { "dimensions" : ["memKind"], "type" : "gauge" }, - "jvm/mem/used" : { "dimensions" : ["memKind"], "type" : "gauge" }, - "jvm/mem/committed" : { "dimensions" : ["memKind"], "type" : "gauge" }, - "jvm/gc/count" : { "dimensions" : ["gcName"], "type" : "count" }, - "jvm/gc/cpu" : { "dimensions" : ["gcName"], "type" : "timer" }, + "jvm/pool/committed" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge", "help": "Committed pool."}, + "jvm/pool/init" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge", "help": "Initial pool."}, + "jvm/pool/max" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge", "help": "Max pool."}, + "jvm/pool/used" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge", "help": "Pool used."}, + "jvm/bufferpool/count" : { "dimensions" : ["bufferpoolName"], "type" : "gauge", "help": "bufferpool count"}, + "jvm/bufferpool/used" : { "dimensions" : ["bufferpoolName"], "type" : "gauge", "help": "bufferpool used"}, + "jvm/bufferpool/capacity" : { "dimensions" : ["bufferpoolName"], "type" : "gauge", "help": "bufferpool capacity"}, + "jvm/mem/init" : { "dimensions" : ["memKind"], "type" : "gauge", "help": "Initial memory"}, + "jvm/mem/max" : { "dimensions" : ["memKind"], "type" : "gauge", "help": "Max memory"}, + "jvm/mem/used" : { "dimensions" : ["memKind"], "type" : "gauge", "help": "Used memory"}, + "jvm/mem/committed" : { "dimensions" : ["memKind"], "type" : "gauge", "help": "Committed memory"}, + "jvm/gc/count" : { "dimensions" : ["gcName"], "type" : "count", "help": "Garbage collection count"}, + "jvm/gc/cpu" : { "dimensions" : ["gcName"], "type" : "timer", "help": "Cpu time in Nanoseconds spent on garbage collection."}, - "ingest/events/buffered" : { "dimensions" : ["serviceName", "bufferCapacity"], "type" : "gauge"}, + "ingest/events/buffered" : { "dimensions" : ["serviceName", "bufferCapacity"], "type" : "gauge", "help": "Number of events queued in the EventReceiverFirehose's buffer"}, - "sys/swap/free" : { "dimensions" : [], "type" : "gauge"}, - "sys/swap/max" : { "dimensions" : [], "type" : "gauge"}, - "sys/swap/pageIn" : { "dimensions" : [], "type" : "gauge"}, - "sys/swap/pageOut" : { "dimensions" : [], "type" : "gauge"}, + "sys/swap/free" : { "dimensions" : [], "type" : "gauge", "help": "Free swap"}, + "sys/swap/max" : { "dimensions" : [], "type" : "gauge", "help": "Max swap"}, + "sys/swap/pageIn" : { "dimensions" : [], "type" : "gauge", "help": "Paged in swap"}, + "sys/swap/pageOut" : { "dimensions" : [], "type" : "gauge", "help": "Paged out swap"}, "sys/disk/write/count" : { "dimensions" : ["fsDevName"], "type" : "count"}, "sys/disk/read/count" : { "dimensions" : ["fsDevName"], "type" : "count"}, - "sys/disk/write/size" : { "dimensions" : ["fsDevName"], "type" : "count"}, - "sys/disk/read/size" : { "dimensions" : ["fsDevName"], "type" : "count"}, - "sys/net/write/size" : { "dimensions" : [], "type" : "count"}, - "sys/net/read/size" : { "dimensions" : [], "type" : "count"}, - "sys/fs/used" : { "dimensions" : ["fsDevName", "fsDirName", "fsTypeName", "fsSysTypeName", "fsOptions"], "type" : "gauge"}, - "sys/fs/max" : { "dimensions" : ["fsDevName", "fsDirName", "fsTypeName", "fsSysTypeName", "fsOptions"], "type" : "gauge"}, - "sys/mem/used" : { "dimensions" : [], "type" : "gauge"}, - "sys/mem/max" : { "dimensions" : [], "type" : "gauge"}, - "sys/storage/used" : { "dimensions" : ["fsDirName"], "type" : "gauge"}, - "sys/cpu" : { "dimensions" : ["cpuName", "cpuTime"], "type" : "gauge"}, + "sys/disk/write/size" : { "dimensions" : ["fsDevName"], "type" : "count", "help": "Bytes written to disk. Can we used to determine how much paging is occurring with regards to segments."}, + "sys/disk/read/size" : { "dimensions" : ["fsDevName"], "type" : "count", "help": "Bytes read from disk. Can we used to determine how much paging is occurring with regards to segments."}, + "sys/net/write/size" : { "dimensions" : [], "type" : "count", "help": "Bytes written to the network."}, + "sys/net/read/size" : { "dimensions" : [], "type" : "count", "help": "Bytes read from the network."}, + "sys/fs/used" : { "dimensions" : ["fsDevName", "fsDirName", "fsTypeName", "fsSysTypeName", "fsOptions"], "type" : "gauge", "help": "Filesystem bytes used."}, + "sys/fs/max" : { "dimensions" : ["fsDevName", "fsDirName", "fsTypeName", "fsSysTypeName", "fsOptions"], "type" : "gauge", "help": "Filesystesm bytes max."}, + "sys/mem/used" : { "dimensions" : [], "type" : "gauge", "help": "Memory used."}, + "sys/mem/max" : { "dimensions" : [], "type" : "gauge", "help": "Memory max"}, + "sys/storage/used" : { "dimensions" : ["fsDirName"], "type" : "gauge", "help": "Disk space used."}, + "sys/cpu" : { "dimensions" : ["cpuName", "cpuTime"], "type" : "gauge", "help": "CPU used"}, - "coordinator-segment/count" : { "dimensions" : ["dataSource"], "type" : "gauge" }, - "historical-segment/count" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge" } + "coordinator_segment/count" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Number of served segments."}, + "historical_segment/count" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge", "help": "Number of served segments."} } From 2ec1a9b8a90a3391f1d9f440c7eed7e7cd93d657 Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Thu, 3 Oct 2019 16:53:38 -0700 Subject: [PATCH 12/42] added documentation for prometheus emitter --- .../extensions-contrib/prometheus.md | 61 +++++++++++++++++++ .../emitter/prometheus/PrometheusEmitter.java | 2 + 2 files changed, 63 insertions(+) create mode 100644 docs/development/extensions-contrib/prometheus.md diff --git a/docs/development/extensions-contrib/prometheus.md b/docs/development/extensions-contrib/prometheus.md new file mode 100644 index 000000000000..bb42ad894f9f --- /dev/null +++ b/docs/development/extensions-contrib/prometheus.md @@ -0,0 +1,61 @@ +--- +id: prometheus +title: "Prometheus Emitter" +--- + + + + +To use this Apache Druid (incubating) extension, make sure to [include](../../development/extensions.md#loading-extensions) `prometheus-emitter` extension. + +## Introduction + +This extension exposes Druid metrics for collection by a Prometheus server +(https://prometheus.io/) + +## Configuration + +All the configuration parameters for the Prometheus emitter are under `druid.emitter.prometheus`. + +|property|description|required?|default| +|--------|-----------|---------|-------| +|`druid.emitter.prometheus.port`|The port on which to expose the prometheus HTTPServer.|yes|none| +|`druid.emitter.prometheus.namespace`|Optional metric namespace.|no|"druid"| +|`druid.emitter.prometheus.dimensionMapPath`|JSON file defining the Prometheus metric type, desired dimensions, help text, and conversionFactor for every Druid metric.|no|Default mapping provided. See below.| + +### Metric mapping + +Each metric to be collected by Prometheus must specify a type, one of `[timer, counter, guage]`. Prometheus Emitter expects this mapping to +be provided as a JSON file. Additionally, this mapping specifies which dimensions should be included for each metric. Prometheus expects +histogram timers to use Seconds as the base unit. Timers which do not use seconds as a base unit can use the `conversionFactor` to set +the base time unit. If the user does not specify their own JSON file, a default mapping is used. All +metrics are expected to be mapped. Metrics which are not mapped will not be tracked. +Prometheus metric path is organized using the following schema: +` : { "dimensions" : , "type" : , conversionFactor: , "help" : ,}` +e.g. +`query/time" : { "dimensions" : ["dataSource", "type"], "conversionFactor": 1000.0, "type" : "timer", "help": "Seconds taken to complete a query."}` + +For metrics which are emitted from multiple services with different dimensions, the metric name is prefixed with +the service name. +e.g. +`"coordinator-segment/count" : { "dimensions" : ["dataSource"], "type" : "gauge" }, + "historical-segment/count" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge" }` + +For most use-cases, the default mapping is sufficient. diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java index c7fe221a2ee6..0b756c2e2eae 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -101,6 +101,8 @@ void emitMetric(ServiceMetricEvent metricEvent) } else { log.error("Unrecognized metric type [%s]", metric.getCollector().getClass()); } + } else { + log.debug("Unmapped metric [%s]", name); } } From 13d3fc3ac4f990ab21a507dc9562d3e973833e78 Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Thu, 3 Oct 2019 17:00:27 -0700 Subject: [PATCH 13/42] safety for invalid labelNames --- .../org/apache/druid/emitter/prometheus/PrometheusEmitter.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java index 0b756c2e2eae..9aeaf3b6b29a 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -88,8 +88,9 @@ void emitMetric(ServiceMetricEvent metricEvent) String[] labelNames = metric.getDimensions(); for (int i = 0; i < labelValues.length; i++) { String labelName = labelNames[i]; + //labelName is controlled by the user. Instead of potential NPE on invalid labelName we use "unknown" as the dimension value Object userDim = userDims.get(labelName); - labelValues[i] = userDim.toString(); + labelValues[i] = userDim != null ? userDim.toString() : "unknown"; } if (metric.getCollector() instanceof Counter) { From cd392eb803e0154c8db828880fe0b6c040ca7a90 Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Fri, 4 Oct 2019 11:31:20 -0700 Subject: [PATCH 14/42] fix travis checks --- extensions-contrib/prometheus-emitter/pom.xml | 14 -------------- .../prometheus/PrometheusEmitterConfig.java | 4 ++-- 2 files changed, 2 insertions(+), 16 deletions(-) diff --git a/extensions-contrib/prometheus-emitter/pom.xml b/extensions-contrib/prometheus-emitter/pom.xml index 963d2f358aea..2b83426a5066 100644 --- a/extensions-contrib/prometheus-emitter/pom.xml +++ b/extensions-contrib/prometheus-emitter/pom.xml @@ -50,7 +50,6 @@ simpleclient_httpserver 0.7.0 - com.google.code.findbugs jsr305 @@ -115,18 +114,5 @@ test-jar test - - org.eclipse.jetty - jetty-server - - - org.eclipse.jetty - jetty-servlet - - - io.prometheus - simpleclient_servlet - 0.4.0 - diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java index e988b951782d..7c81603a57ad 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java @@ -39,13 +39,13 @@ public class PrometheusEmitterConfig private final String dimensionMapPath; @JsonProperty - private final int port; + private final Integer port; @JsonCreator public PrometheusEmitterConfig( @JsonProperty("namespace") @Nullable String namespace, @JsonProperty("dimensionMapPath") @Nullable String dimensionMapPath, - @JsonProperty("port") int port + @JsonProperty("port") Integer port ) { this.namespace = namespace != null ? namespace : "druid"; From 8246c3c18bbcd7acd835ab1ebdf1b8cbeb7e098e Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Fri, 4 Oct 2019 13:39:08 -0700 Subject: [PATCH 15/42] Unit test and better sanitization of metrics names and label values --- .../druid/emitter/prometheus/Metrics.java | 2 +- .../emitter/prometheus/PrometheusEmitter.java | 2 +- .../src/main/resources/defaultMetrics.json | 4 ++-- ...theusEmitterTest.java => MetricsTest.java} | 21 ++++++++++++++++++- sshuttle.pid | 1 + 5 files changed, 25 insertions(+), 5 deletions(-) rename extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/{PrometheusEmitterTest.java => MetricsTest.java} (52%) create mode 100644 sshuttle.pid diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java index 61fd2e1384cf..c373f5991b50 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java @@ -66,7 +66,7 @@ public Metrics(String namespace, String path) Metric metric = entry.getValue(); Metric.Type type = metric.type; String[] dimensions = metric.dimensions.toArray(new String[0]); - String formattedName = StringUtils.replaceChar(StringUtils.toLowerCase(name), '/', "_"); + String formattedName = StringUtils.toLowerCase(name).replaceAll("[^a-zA-Z_:][^a-zA-Z0-9_:]*", "_"); SimpleCollector collector = null; if (Metric.Type.count.equals(type)) { collector = new Counter.Builder() diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java index 9aeaf3b6b29a..32eb2165e484 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -90,7 +90,7 @@ void emitMetric(ServiceMetricEvent metricEvent) String labelName = labelNames[i]; //labelName is controlled by the user. Instead of potential NPE on invalid labelName we use "unknown" as the dimension value Object userDim = userDims.get(labelName); - labelValues[i] = userDim != null ? userDim.toString() : "unknown"; + labelValues[i] = userDim != null ? userDim.toString().replaceAll("[^a-zA-Z_][^a-zA-Z0-9_]*", "_") : "unknown"; } if (metric.getCollector() instanceof Counter) { diff --git a/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json index 51468a0d764f..4d1fa5d13ab5 100644 --- a/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json +++ b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json @@ -110,8 +110,8 @@ "sys/swap/max" : { "dimensions" : [], "type" : "gauge", "help": "Max swap"}, "sys/swap/pageIn" : { "dimensions" : [], "type" : "gauge", "help": "Paged in swap"}, "sys/swap/pageOut" : { "dimensions" : [], "type" : "gauge", "help": "Paged out swap"}, - "sys/disk/write/count" : { "dimensions" : ["fsDevName"], "type" : "count"}, - "sys/disk/read/count" : { "dimensions" : ["fsDevName"], "type" : "count"}, + "sys/disk/write/count" : { "dimensions" : ["fsDevName"], "type" : "count", "help": "Writes to disk."}, + "sys/disk/read/count" : { "dimensions" : ["fsDevName"], "type" : "count", "help": "Reads from disk."}, "sys/disk/write/size" : { "dimensions" : ["fsDevName"], "type" : "count", "help": "Bytes written to disk. Can we used to determine how much paging is occurring with regards to segments."}, "sys/disk/read/size" : { "dimensions" : ["fsDevName"], "type" : "count", "help": "Bytes read from disk. Can we used to determine how much paging is occurring with regards to segments."}, "sys/net/write/size" : { "dimensions" : [], "type" : "count", "help": "Bytes written to the network."}, diff --git a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java similarity index 52% rename from extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java rename to extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java index 666bd1419515..6d87780360fe 100644 --- a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java +++ b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java @@ -19,6 +19,25 @@ package org.apache.druid.emitter.prometheus; -public class PrometheusEmitterTest +import io.prometheus.client.Histogram; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +public class MetricsTest { + @Test + public void testMetricsConfiguration() + { + Metrics metrics = new Metrics("test", null); + DimensionsAndCollector dimensionsAndCollector = metrics.getByName("query/time", "historical"); + assertNotNull(dimensionsAndCollector); + String[] dimensions = dimensionsAndCollector.getDimensions(); + assertEquals("dataSource", dimensions[0]); + assertEquals("type", dimensions[1]); + assertEquals(1000.0, dimensionsAndCollector.getConversionFactor(), 0.0); + assertTrue(dimensionsAndCollector.getCollector() instanceof Histogram); + } } diff --git a/sshuttle.pid b/sshuttle.pid new file mode 100644 index 000000000000..9d84d6806e4d --- /dev/null +++ b/sshuttle.pid @@ -0,0 +1 @@ +77782 From 8ab89f28ae6136a98648de51290e334b82bbf448 Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Fri, 4 Oct 2019 13:42:28 -0700 Subject: [PATCH 16/42] add precondition to check namespace against regex --- .../apache/druid/emitter/prometheus/PrometheusEmitterConfig.java | 1 + 1 file changed, 1 insertion(+) diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java index 7c81603a57ad..937bccf5e00a 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java @@ -49,6 +49,7 @@ public PrometheusEmitterConfig( ) { this.namespace = namespace != null ? namespace : "druid"; + Preconditions.checkArgument(namespace.matches("[a-zA-Z_:][a-zA-Z0-9_:]*"), "Invalid namespace " + namespace); this.dimensionMapPath = dimensionMapPath; this.port = Preconditions.checkNotNull(port, "Prometheus server port cannot be null."); } From 9b4c70d3920c26924ad1c5801c04854698d4baa2 Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Fri, 4 Oct 2019 14:19:23 -0700 Subject: [PATCH 17/42] use precompiled regex --- .../java/org/apache/druid/emitter/prometheus/Metrics.java | 4 +++- .../apache/druid/emitter/prometheus/PrometheusEmitter.java | 4 +++- .../druid/emitter/prometheus/PrometheusEmitterConfig.java | 5 ++++- sshuttle.pid | 1 - 4 files changed, 10 insertions(+), 4 deletions(-) delete mode 100644 sshuttle.pid diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java index c373f5991b50..7a3487179832 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java @@ -39,6 +39,7 @@ import java.util.HashMap; import java.util.Map; import java.util.SortedSet; +import java.util.regex.Pattern; public class Metrics { @@ -46,6 +47,7 @@ public class Metrics private static final Logger log = new Logger(Metrics.class); private final Map map = new HashMap<>(); private final ObjectMapper mapper = new ObjectMapper(); + private final Pattern pattern = Pattern.compile("[^a-zA-Z_:][^a-zA-Z0-9_:]*"); public DimensionsAndCollector getByName(String name, String service) { @@ -66,7 +68,7 @@ public Metrics(String namespace, String path) Metric metric = entry.getValue(); Metric.Type type = metric.type; String[] dimensions = metric.dimensions.toArray(new String[0]); - String formattedName = StringUtils.toLowerCase(name).replaceAll("[^a-zA-Z_:][^a-zA-Z0-9_:]*", "_"); + String formattedName = pattern.matcher(StringUtils.toLowerCase(name)).replaceAll("_"); SimpleCollector collector = null; if (Metric.Type.count.equals(type)) { collector = new Counter.Builder() diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java index 32eb2165e484..5a69c8996f4c 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -31,6 +31,7 @@ import java.io.IOException; import java.util.Map; +import java.util.regex.Pattern; /** * @@ -41,6 +42,7 @@ public class PrometheusEmitter implements Emitter private static final Logger log = new Logger(PrometheusEmitter.class); private final Metrics metrics; private final PrometheusEmitterConfig config; + private final Pattern pattern = Pattern.compile("[^a-zA-Z_][^a-zA-Z0-9_]*"); private HTTPServer server; @@ -90,7 +92,7 @@ void emitMetric(ServiceMetricEvent metricEvent) String labelName = labelNames[i]; //labelName is controlled by the user. Instead of potential NPE on invalid labelName we use "unknown" as the dimension value Object userDim = userDims.get(labelName); - labelValues[i] = userDim != null ? userDim.toString().replaceAll("[^a-zA-Z_][^a-zA-Z0-9_]*", "_") : "unknown"; + labelValues[i] = userDim != null ? pattern.matcher(userDim.toString()).replaceAll("_") : "unknown"; } if (metric.getCollector() instanceof Counter) { diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java index 937bccf5e00a..6a89562e813d 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java @@ -24,6 +24,7 @@ import com.google.common.base.Preconditions; import javax.annotation.Nullable; +import java.util.regex.Pattern; /** * @@ -31,6 +32,8 @@ public class PrometheusEmitterConfig { + Pattern pattern = Pattern.compile("[a-zA-Z_:][a-zA-Z0-9_:]*"); + @JsonProperty private final String namespace; @@ -49,7 +52,7 @@ public PrometheusEmitterConfig( ) { this.namespace = namespace != null ? namespace : "druid"; - Preconditions.checkArgument(namespace.matches("[a-zA-Z_:][a-zA-Z0-9_:]*"), "Invalid namespace " + namespace); + Preconditions.checkArgument(pattern.matcher(namespace).matches(), "Invalid namespace " + namespace); this.dimensionMapPath = dimensionMapPath; this.port = Preconditions.checkNotNull(port, "Prometheus server port cannot be null."); } diff --git a/sshuttle.pid b/sshuttle.pid deleted file mode 100644 index 9d84d6806e4d..000000000000 --- a/sshuttle.pid +++ /dev/null @@ -1 +0,0 @@ -77782 From 5221655a60f8e8623f32ccca9f95cee327af497d Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Mon, 7 Oct 2019 10:59:50 -0700 Subject: [PATCH 18/42] remove static imports. fix metric types --- .../src/main/resources/defaultMetrics.json | 6 +++--- .../druid/emitter/prometheus/MetricsTest.java | 15 ++++++--------- 2 files changed, 9 insertions(+), 12 deletions(-) diff --git a/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json index 4d1fa5d13ab5..90770d7b4fc9 100644 --- a/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json +++ b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json @@ -73,8 +73,8 @@ "segment/dropped/count" : { "dimensions" : ["tier"], "type" : "count", "help": "Number of segments dropped due to being overshadowed." }, "segment/deleted/count" : { "dimensions" : ["tier"], "type" : "count", "help": "Number of segments dropped due to rules."}, "segment/unneeded/count" : { "dimensions" : ["tier"], "type" : "count", "help": "Number of segments dropped due to being marked as unused."}, - "segment/unavailable/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of segments (not including replicas) left to load until segments that should be loaded in the cluster are available for queries."}, - "segment/underReplicated/count" : { "dimensions" : ["dataSource", "tier"], "type" : "count", "help": "Number of segments (including replicas) left to load until segments that should be loaded in the cluster are available for queries."}, + "segment/unavailable/count" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Number of segments (not including replicas) left to load until segments that should be loaded in the cluster are available for queries."}, + "segment/underReplicated/count" : { "dimensions" : ["dataSource", "tier"], "type" : "gauge", "help": "Number of segments (including replicas) left to load until segments that should be loaded in the cluster are available for queries."}, "segment/cost/raw" : { "dimensions" : ["tier"], "type" : "count", "help": "Used in cost balancing. The raw cost of hosting segments."}, "segment/cost/normalization" : { "dimensions" : ["tier"], "type" : "count", "help": "Used in cost balancing. The normalization of hosting segments."}, "segment/cost/normalized" : { "dimensions" : ["tier"], "type" : "count", "help": "Used in cost balancing. The normalized cost of hosting segments."}, @@ -102,7 +102,7 @@ "jvm/mem/used" : { "dimensions" : ["memKind"], "type" : "gauge", "help": "Used memory"}, "jvm/mem/committed" : { "dimensions" : ["memKind"], "type" : "gauge", "help": "Committed memory"}, "jvm/gc/count" : { "dimensions" : ["gcName"], "type" : "count", "help": "Garbage collection count"}, - "jvm/gc/cpu" : { "dimensions" : ["gcName"], "type" : "timer", "help": "Cpu time in Nanoseconds spent on garbage collection."}, + "jvm/gc/cpu" : { "dimensions" : ["gcName"], "type" : "count", "help": "Count of CPU time in Nanoseconds spent on garbage collection. Note: `jvm/gc/cpu` represents the total time over multiple GC cycles; divide by `jvm/gc/count` to get the mean GC time per cycle."}, "ingest/events/buffered" : { "dimensions" : ["serviceName", "bufferCapacity"], "type" : "gauge", "help": "Number of events queued in the EventReceiverFirehose's buffer"}, diff --git a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java index 6d87780360fe..128f9efe7f95 100644 --- a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java +++ b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java @@ -20,12 +20,9 @@ package org.apache.druid.emitter.prometheus; import io.prometheus.client.Histogram; +import org.junit.Assert; import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; - public class MetricsTest { @Test @@ -33,11 +30,11 @@ public void testMetricsConfiguration() { Metrics metrics = new Metrics("test", null); DimensionsAndCollector dimensionsAndCollector = metrics.getByName("query/time", "historical"); - assertNotNull(dimensionsAndCollector); + Assert.assertNotNull(dimensionsAndCollector); String[] dimensions = dimensionsAndCollector.getDimensions(); - assertEquals("dataSource", dimensions[0]); - assertEquals("type", dimensions[1]); - assertEquals(1000.0, dimensionsAndCollector.getConversionFactor(), 0.0); - assertTrue(dimensionsAndCollector.getCollector() instanceof Histogram); + Assert.assertEquals("dataSource", dimensions[0]); + Assert.assertEquals("type", dimensions[1]); + Assert.assertEquals(1000.0, dimensionsAndCollector.getConversionFactor(), 0.0); + Assert.assertTrue(dimensionsAndCollector.getCollector() instanceof Histogram); } } From abae5c02aa8c2840b24b254f6a921a796ae8ec13 Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Thu, 10 Oct 2019 01:03:34 -0700 Subject: [PATCH 19/42] better docs. fix possible NPE in PrometheusEmitterConfig. Guard against multiple calls to PrometheusEmitter.start() --- docs/development/extensions-contrib/prometheus.md | 8 +++++++- .../emitter/prometheus/PrometheusEmitter.java | 14 +++++++++----- .../prometheus/PrometheusEmitterConfig.java | 3 ++- 3 files changed, 18 insertions(+), 7 deletions(-) diff --git a/docs/development/extensions-contrib/prometheus.md b/docs/development/extensions-contrib/prometheus.md index bb42ad894f9f..560941e5b6ef 100644 --- a/docs/development/extensions-contrib/prometheus.md +++ b/docs/development/extensions-contrib/prometheus.md @@ -37,9 +37,15 @@ All the configuration parameters for the Prometheus emitter are under `druid.emi |property|description|required?|default| |--------|-----------|---------|-------| |`druid.emitter.prometheus.port`|The port on which to expose the prometheus HTTPServer.|yes|none| -|`druid.emitter.prometheus.namespace`|Optional metric namespace.|no|"druid"| +|`druid.emitter.prometheus.namespace`|Optional metric namespace. Must match the regex `[a-zA-Z_:][a-zA-Z0-9_:]*`|no|"druid"| |`druid.emitter.prometheus.dimensionMapPath`|JSON file defining the Prometheus metric type, desired dimensions, help text, and conversionFactor for every Druid metric.|no|Default mapping provided. See below.| +### Metric names + +All metric names and labels are reformatted to match Prometheus standards. +- For names: all characters which are not alphanumeric, underscores, or colons (matching `[^a-zA-Z_:][^a-zA-Z0-9_:]*`) are replaced with `_` +- For labels: all characters which are not alphanumeric or underscores (matching `[^a-zA-Z_][^a-zA-Z0-9_]*`) are replaced with `_` + ### Metric mapping Each metric to be collected by Prometheus must specify a type, one of `[timer, counter, guage]`. Prometheus Emitter expects this mapping to diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java index 5a69c8996f4c..518b270748d1 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -61,11 +61,15 @@ public PrometheusEmitter(PrometheusEmitterConfig config) @Override public void start() { - try { - server = new HTTPServer(config.getPort()); - } - catch (IOException e) { - log.error(e, "Unable to start prometheus HTTPServer"); + if (server == null) { + try { + server = new HTTPServer(config.getPort()); + } + catch (IOException e) { + log.error(e, "Unable to start prometheus HTTPServer"); + } + } else { + log.error("HTTPServer is already started"); } } diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java index 6a89562e813d..39c94e2af8a1 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java @@ -35,6 +35,7 @@ public class PrometheusEmitterConfig Pattern pattern = Pattern.compile("[a-zA-Z_:][a-zA-Z0-9_:]*"); @JsonProperty + @Nullable private final String namespace; @JsonProperty @@ -52,7 +53,7 @@ public PrometheusEmitterConfig( ) { this.namespace = namespace != null ? namespace : "druid"; - Preconditions.checkArgument(pattern.matcher(namespace).matches(), "Invalid namespace " + namespace); + Preconditions.checkArgument(pattern.matcher(this.namespace).matches(), "Invalid namespace " + this.namespace); this.dimensionMapPath = dimensionMapPath; this.port = Preconditions.checkNotNull(port, "Prometheus server port cannot be null."); } From fa4a985c9d9d9e390d555e86822110200e4ad49d Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Wed, 27 Nov 2019 10:18:06 -0800 Subject: [PATCH 20/42] Update regex for label-value replacements to allow internal numeric values. Additional tests --- .../extensions-contrib/prometheus.md | 2 +- .../emitter/prometheus/PrometheusEmitter.java | 2 +- .../druid/emitter/prometheus/MetricsTest.java | 2 ++ .../prometheus/PrometheusEmitterTest.java | 30 +++++++++++++++++++ 4 files changed, 34 insertions(+), 2 deletions(-) create mode 100644 extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java diff --git a/docs/development/extensions-contrib/prometheus.md b/docs/development/extensions-contrib/prometheus.md index 560941e5b6ef..eec0bd308b05 100644 --- a/docs/development/extensions-contrib/prometheus.md +++ b/docs/development/extensions-contrib/prometheus.md @@ -44,7 +44,7 @@ All the configuration parameters for the Prometheus emitter are under `druid.emi All metric names and labels are reformatted to match Prometheus standards. - For names: all characters which are not alphanumeric, underscores, or colons (matching `[^a-zA-Z_:][^a-zA-Z0-9_:]*`) are replaced with `_` -- For labels: all characters which are not alphanumeric or underscores (matching `[^a-zA-Z_][^a-zA-Z0-9_]*`) are replaced with `_` +- For labels: all characters which are not alphanumeric or underscores (matching `[^a-zA-Z0-9_][^a-zA-Z0-9_]*`) are replaced with `_` ### Metric mapping diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java index 518b270748d1..6080c871fc80 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -42,7 +42,7 @@ public class PrometheusEmitter implements Emitter private static final Logger log = new Logger(PrometheusEmitter.class); private final Metrics metrics; private final PrometheusEmitterConfig config; - private final Pattern pattern = Pattern.compile("[^a-zA-Z_][^a-zA-Z0-9_]*"); + private final Pattern pattern = Pattern.compile("[^a-zA-Z0-9_][^a-zA-Z0-9_]*"); private HTTPServer server; diff --git a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java index 128f9efe7f95..add6f72c8582 100644 --- a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java +++ b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java @@ -19,6 +19,7 @@ package org.apache.druid.emitter.prometheus; +import io.prometheus.client.Gauge; import io.prometheus.client.Histogram; import org.junit.Assert; import org.junit.Test; @@ -30,6 +31,7 @@ public void testMetricsConfiguration() { Metrics metrics = new Metrics("test", null); DimensionsAndCollector dimensionsAndCollector = metrics.getByName("query/time", "historical"); + DimensionsAndCollector d = metrics.getByName("segment/loadQueue/count", "historical"); Assert.assertNotNull(dimensionsAndCollector); String[] dimensions = dimensionsAndCollector.getDimensions(); Assert.assertEquals("dataSource", dimensions[0]); diff --git a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java new file mode 100644 index 000000000000..2d8572e5d57f --- /dev/null +++ b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java @@ -0,0 +1,30 @@ +package org.apache.druid.emitter.prometheus; + +import com.google.common.collect.ImmutableMap; +import io.prometheus.client.CollectorRegistry; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.joda.time.DateTime; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +public class PrometheusEmitterTest +{ + @Test + public void testEmitter() { + PrometheusEmitterConfig config = new PrometheusEmitterConfig(null, null, 0); + PrometheusEmitter emitter = new PrometheusEmitter(config); + ServiceMetricEvent build = ServiceMetricEvent.builder() + .setDimension("server", "druid-data01.vpc.region") + .build("segment/loadQueue/count", 10) + .build(ImmutableMap.of("service", "historical")); + assertEquals("historical", build.getService()); + assertFalse(build.getUserDims().isEmpty()); + emitter.emit(build); + Double count = CollectorRegistry.defaultRegistry.getSampleValue( + "druid_segment_loadqueue_count", new String[]{"server"}, new String[]{"druid_data01_vpc_region"} + ); + assertEquals(10, count.intValue()); + } +} From 5689f444b291bdc702969a6fb68d6c5c9707789e Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Tue, 28 Jan 2020 18:58:06 -0800 Subject: [PATCH 21/42] Adds missing license header updates website/.spelling to add words used in prometheus-emitter docs. updates docs/operations/metrics.md to correct the spelling of bufferPoolName --- docs/operations/metrics.md | 4 ++-- .../prometheus/PrometheusEmitterTest.java | 19 +++++++++++++++++++ website/.spelling | 3 +++ 3 files changed, 24 insertions(+), 2 deletions(-) diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md index cf7f00754aba..db7d22f73bf2 100644 --- a/docs/operations/metrics.md +++ b/docs/operations/metrics.md @@ -284,8 +284,8 @@ These metrics are only available if the JVMMonitor module is included. |`jvm/pool/init`|Initial pool.|poolKind, poolName.|Varies.| |`jvm/pool/max`|Max pool.|poolKind, poolName.|Varies.| |`jvm/pool/used`|Pool used.|poolKind, poolName.|< max pool| -|`jvm/bufferpool/count`|Bufferpool count.|bufferpoolName.|Varies.| -|`jvm/bufferpool/used`|Bufferpool used.|bufferpoolName.|close to capacity| +|`jvm/bufferpool/count`|Bufferpool count.|bufferPoolName.|Varies.| +|`jvm/bufferpool/used`|Bufferpool used.|bufferPoolName.|close to capacity| |`jvm/bufferpool/capacity`|Bufferpool capacity.|bufferPoolName.|Varies.| |`jvm/mem/init`|Initial memory.|memKind.|Varies.| |`jvm/mem/max`|Max memory.|memKind.|Varies.| diff --git a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java index 2d8572e5d57f..65cecdc01646 100644 --- a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java +++ b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java @@ -1,3 +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. + */ + package org.apache.druid.emitter.prometheus; import com.google.common.collect.ImmutableMap; diff --git a/website/.spelling b/website/.spelling index 4184c674101c..38a9f06e025a 100644 --- a/website/.spelling +++ b/website/.spelling @@ -582,6 +582,9 @@ com.microsoft.sqlserver.jdbc.SQLServerDriver sqljdbc - ../docs/development/extensions-contrib/statsd.md convertRange +- ../docs/development/extensions-contrib/prometheus.md +HTTPServer +conversionFactor - ../docs/development/extensions-contrib/tdigestsketch-quantiles.md postAggregator quantileFromTDigestSketch From 1c355951470522470a98a0924b605829260479e0 Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Tue, 28 Jan 2020 19:59:28 -0800 Subject: [PATCH 22/42] fixes version in extensions-contrib/prometheus-emitter --- extensions-contrib/prometheus-emitter/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-contrib/prometheus-emitter/pom.xml b/extensions-contrib/prometheus-emitter/pom.xml index 2b83426a5066..569a7e20df78 100644 --- a/extensions-contrib/prometheus-emitter/pom.xml +++ b/extensions-contrib/prometheus-emitter/pom.xml @@ -23,7 +23,7 @@ druid org.apache.druid - 0.17.0-incubating-SNAPSHOT + 0.18.0-SNAPSHOT ../../pom.xml 4.0.0 From 456486aa197ad2295d546d37455f005f41be90a1 Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Tue, 28 Jan 2020 21:37:11 -0800 Subject: [PATCH 23/42] fix style guide errors --- .../druid/emitter/prometheus/MetricsTest.java | 1 - .../emitter/prometheus/PrometheusEmitterTest.java | 13 ++++++------- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java index add6f72c8582..dfd71ba1c95a 100644 --- a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java +++ b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java @@ -19,7 +19,6 @@ package org.apache.druid.emitter.prometheus; -import io.prometheus.client.Gauge; import io.prometheus.client.Histogram; import org.junit.Assert; import org.junit.Test; diff --git a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java index 65cecdc01646..97d6692dfd65 100644 --- a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java +++ b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java @@ -22,28 +22,27 @@ import com.google.common.collect.ImmutableMap; import io.prometheus.client.CollectorRegistry; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; -import org.joda.time.DateTime; import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; +import org.junit.Assert; public class PrometheusEmitterTest { @Test - public void testEmitter() { + public void testEmitter() + { PrometheusEmitterConfig config = new PrometheusEmitterConfig(null, null, 0); PrometheusEmitter emitter = new PrometheusEmitter(config); ServiceMetricEvent build = ServiceMetricEvent.builder() .setDimension("server", "druid-data01.vpc.region") .build("segment/loadQueue/count", 10) .build(ImmutableMap.of("service", "historical")); - assertEquals("historical", build.getService()); - assertFalse(build.getUserDims().isEmpty()); + Assert.assertEquals("historical", build.getService()); + Assert.assertFalse(build.getUserDims().isEmpty()); emitter.emit(build); Double count = CollectorRegistry.defaultRegistry.getSampleValue( "druid_segment_loadqueue_count", new String[]{"server"}, new String[]{"druid_data01_vpc_region"} ); - assertEquals(10, count.intValue()); + Assert.assertEquals(10, count.intValue()); } } From 8827fbf78bc2cd3fc8f193e52284b88c742616da Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Tue, 28 Jan 2020 23:25:33 -0800 Subject: [PATCH 24/42] update import ordering --- .../apache/druid/emitter/prometheus/PrometheusEmitterTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java index 97d6692dfd65..560c9cfebe86 100644 --- a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java +++ b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java @@ -22,9 +22,9 @@ import com.google.common.collect.ImmutableMap; import io.prometheus.client.CollectorRegistry; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.junit.Assert; import org.junit.Test; -import org.junit.Assert; public class PrometheusEmitterTest { From ba4a67242ad8f2b087b5cf2c672556158ee29d78 Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Wed, 29 Jan 2020 00:35:55 -0800 Subject: [PATCH 25/42] add another word to website/.spelling --- website/.spelling | 1 + 1 file changed, 1 insertion(+) diff --git a/website/.spelling b/website/.spelling index 38a9f06e025a..31245fb8f4ac 100644 --- a/website/.spelling +++ b/website/.spelling @@ -585,6 +585,7 @@ convertRange - ../docs/development/extensions-contrib/prometheus.md HTTPServer conversionFactor +prometheus - ../docs/development/extensions-contrib/tdigestsketch-quantiles.md postAggregator quantileFromTDigestSketch From 7aed5e1b54d484520a0cbec6b810e0458b1463a2 Mon Sep 17 00:00:00 2001 From: Tianxin Zhao Date: Sun, 29 Mar 2020 15:15:22 -0700 Subject: [PATCH 26/42] remove unthrown declared exception --- .../druid/emitter/prometheus/PrometheusEmitterModule.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java index 6cc2b06c9ead..b56e56c49f2a 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java @@ -56,7 +56,7 @@ public void configure(Binder binder) @Provides @ManageLifecycle @Named(EMITTER_TYPE) - public Emitter getEmitter(PrometheusEmitterConfig config) throws IOException + public Emitter getEmitter(PrometheusEmitterConfig config) { return PrometheusEmitter.of(config); } From f079d67c30e19bce9e4356037f745f31f4042d04 Mon Sep 17 00:00:00 2001 From: Tianxin Zhao Date: Sun, 29 Mar 2020 16:56:04 -0700 Subject: [PATCH 27/42] remove unused import --- .../apache/druid/emitter/prometheus/PrometheusEmitterModule.java | 1 - 1 file changed, 1 deletion(-) diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java index b56e56c49f2a..bb26c4d37d58 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java @@ -29,7 +29,6 @@ import org.apache.druid.initialization.DruidModule; import org.apache.druid.java.util.emitter.core.Emitter; -import java.io.IOException; import java.util.Collections; import java.util.List; From 76278f102baf0caf1244b5083eb1497db7866dcc Mon Sep 17 00:00:00 2001 From: Tianxin Zhao Date: Sun, 5 Apr 2020 22:15:11 -0700 Subject: [PATCH 28/42] Pushgateway strategy for metrics --- .../extensions-contrib/prometheus.md | 5 ++- extensions-contrib/prometheus-emitter/pom.xml | 5 +++ .../druid/emitter/prometheus/Metrics.java | 4 ++ .../emitter/prometheus/PrometheusEmitter.java | 44 ++++++++++++++----- .../prometheus/PrometheusEmitterConfig.java | 31 ++++++++++++- .../prometheus/PrometheusEmitterTest.java | 2 +- 6 files changed, 77 insertions(+), 14 deletions(-) diff --git a/docs/development/extensions-contrib/prometheus.md b/docs/development/extensions-contrib/prometheus.md index eec0bd308b05..6ede7923a469 100644 --- a/docs/development/extensions-contrib/prometheus.md +++ b/docs/development/extensions-contrib/prometheus.md @@ -36,9 +36,12 @@ All the configuration parameters for the Prometheus emitter are under `druid.emi |property|description|required?|default| |--------|-----------|---------|-------| -|`druid.emitter.prometheus.port`|The port on which to expose the prometheus HTTPServer.|yes|none| +|`druid.emitter.prometheus.strategy`|The strategy to expose prometheus metrics. Default strategy `exporter` would expose metrics for scraping purpose. Only peon task (short-lived jobs) need to use `pushgateway` strategy.|yes|scraping| +|`druid.emitter.prometheus.port`|The port on which to expose the prometheus HTTPServer. Required if using exporter strategy.|no|none| |`druid.emitter.prometheus.namespace`|Optional metric namespace. Must match the regex `[a-zA-Z_:][a-zA-Z0-9_:]*`|no|"druid"| |`druid.emitter.prometheus.dimensionMapPath`|JSON file defining the Prometheus metric type, desired dimensions, help text, and conversionFactor for every Druid metric.|no|Default mapping provided. See below.| +|`druid.emitter.prometheus.pushGatewayAddress`|Pushgateway address. Required if using pushgateway strategy|no|none| + ### Metric names diff --git a/extensions-contrib/prometheus-emitter/pom.xml b/extensions-contrib/prometheus-emitter/pom.xml index 569a7e20df78..ea6864398e5b 100644 --- a/extensions-contrib/prometheus-emitter/pom.xml +++ b/extensions-contrib/prometheus-emitter/pom.xml @@ -50,6 +50,11 @@ simpleclient_httpserver 0.7.0 + + io.prometheus + simpleclient_pushgateway + 0.7.0 + com.google.code.findbugs jsr305 diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java index 7a3487179832..a5d1861a2e16 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java @@ -123,6 +123,10 @@ private Map readMap(String path) } } + public Map getMap(){ + return map; + } + public static class Metric { public final SortedSet dimensions; diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java index 6080c871fc80..1292567f1006 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -20,10 +20,12 @@ package org.apache.druid.emitter.prometheus; +import com.google.common.collect.ImmutableMap; import io.prometheus.client.Counter; import io.prometheus.client.Gauge; import io.prometheus.client.Histogram; import io.prometheus.client.exporter.HTTPServer; +import io.prometheus.client.exporter.PushGateway; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.core.Emitter; import org.apache.druid.java.util.emitter.core.Event; @@ -42,9 +44,12 @@ public class PrometheusEmitter implements Emitter private static final Logger log = new Logger(PrometheusEmitter.class); private final Metrics metrics; private final PrometheusEmitterConfig config; + private final PrometheusEmitterConfig.Strategy strategy; private final Pattern pattern = Pattern.compile("[^a-zA-Z0-9_][^a-zA-Z0-9_]*"); private HTTPServer server; + private PushGateway pushGateway; + private String identifier; static PrometheusEmitter of(PrometheusEmitterConfig config) { @@ -54,6 +59,7 @@ static PrometheusEmitter of(PrometheusEmitterConfig config) public PrometheusEmitter(PrometheusEmitterConfig config) { this.config = config; + this.strategy = config.getStrategy(); metrics = new Metrics(config.getNamespace(), config.getDimensionMapPath()); } @@ -61,16 +67,21 @@ public PrometheusEmitter(PrometheusEmitterConfig config) @Override public void start() { - if (server == null) { - try { - server = new HTTPServer(config.getPort()); - } - catch (IOException e) { - log.error(e, "Unable to start prometheus HTTPServer"); + if(strategy.equals(PrometheusEmitterConfig.Strategy.exporter)){ + if (server == null) { + try { + server = new HTTPServer(config.getPort()); + } + catch (IOException e) { + log.error(e, "Unable to start prometheus HTTPServer"); + } + } else { + log.error("HTTPServer is already started"); } - } else { - log.error("HTTPServer is already started"); + } else if (strategy.equals(PrometheusEmitterConfig.Strategy.pushgateway)){ + pushGateway = new PushGateway(config.getPushGatewayAddress()); } + } @Override @@ -86,6 +97,7 @@ void emitMetric(ServiceMetricEvent metricEvent) String name = metricEvent.getMetric(); String service = metricEvent.getService(); Map userDims = metricEvent.getUserDims(); + identifier = (userDims.get("task") == null ? metricEvent.getHost() : (String) userDims.get("task")); Number value = metricEvent.getValue(); DimensionsAndCollector metric = metrics.getByName(name, service); @@ -116,13 +128,25 @@ void emitMetric(ServiceMetricEvent metricEvent) @Override public void flush() { + Map map = metrics.getMap(); + try { + for (DimensionsAndCollector collector : map.values()) { + pushGateway.push(collector.getCollector(), config.getNamespace(), ImmutableMap.of(config.getNamespace(), identifier)); + } + } catch(IOException e){ + log.error(e, "Unable to push prometheus metrics to pushGateway"); + } } @Override public void close() { - if (server != null) { - server.stop(); + if(strategy.equals(PrometheusEmitterConfig.Strategy.exporter)){ + if (server != null) { + server.stop(); + } + } else { + flush(); } } } diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java index 39c94e2af8a1..f713e85ab7eb 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java @@ -24,6 +24,7 @@ import com.google.common.base.Preconditions; import javax.annotation.Nullable; +import javax.validation.constraints.Null; import java.util.regex.Pattern; /** @@ -34,6 +35,9 @@ public class PrometheusEmitterConfig Pattern pattern = Pattern.compile("[a-zA-Z_:][a-zA-Z0-9_:]*"); + @JsonProperty + private final Strategy strategy; + @JsonProperty @Nullable private final String namespace; @@ -43,19 +47,29 @@ public class PrometheusEmitterConfig private final String dimensionMapPath; @JsonProperty + @Nullable private final Integer port; + @JsonProperty + @Nullable + private final String pushGatewayAddress; + @JsonCreator public PrometheusEmitterConfig( + @JsonProperty("strategy") Strategy strategy, @JsonProperty("namespace") @Nullable String namespace, @JsonProperty("dimensionMapPath") @Nullable String dimensionMapPath, - @JsonProperty("port") Integer port + @JsonProperty("port") @Nullable Integer port, + @JsonProperty("pushGatewayAddress") @Nullable String pushGatewayAddress ) { + + this.strategy = Preconditions.checkNotNull(strategy, "Prometheus metrics expose strategy cannot be null"); this.namespace = namespace != null ? namespace : "druid"; Preconditions.checkArgument(pattern.matcher(this.namespace).matches(), "Invalid namespace " + this.namespace); this.dimensionMapPath = dimensionMapPath; - this.port = Preconditions.checkNotNull(port, "Prometheus server port cannot be null."); + this.port = port; + this.pushGatewayAddress = pushGatewayAddress; } public String getNamespace() @@ -72,4 +86,17 @@ public int getPort() { return port; } + + public String getPushGatewayAddress() { + return pushGatewayAddress; + } + + public Strategy getStrategy(){ + return strategy; + } + + public enum Strategy + { + exporter, pushgateway + } } diff --git a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java index 560c9cfebe86..fc2fdd5f7600 100644 --- a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java +++ b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java @@ -31,7 +31,7 @@ public class PrometheusEmitterTest @Test public void testEmitter() { - PrometheusEmitterConfig config = new PrometheusEmitterConfig(null, null, 0); + PrometheusEmitterConfig config = new PrometheusEmitterConfig(PrometheusEmitterConfig.Strategy.exporter, null, null, 0, null); PrometheusEmitter emitter = new PrometheusEmitter(config); ServiceMetricEvent build = ServiceMetricEvent.builder() .setDimension("server", "druid-data01.vpc.region") From 1e2e78ced09ca617d97df7105c912cb47f146ed7 Mon Sep 17 00:00:00 2001 From: Tianxin Zhao Date: Sun, 5 Apr 2020 22:18:53 -0700 Subject: [PATCH 29/42] typo --- docs/development/extensions-contrib/prometheus.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/development/extensions-contrib/prometheus.md b/docs/development/extensions-contrib/prometheus.md index 6ede7923a469..5f695f23c5cd 100644 --- a/docs/development/extensions-contrib/prometheus.md +++ b/docs/development/extensions-contrib/prometheus.md @@ -36,7 +36,7 @@ All the configuration parameters for the Prometheus emitter are under `druid.emi |property|description|required?|default| |--------|-----------|---------|-------| -|`druid.emitter.prometheus.strategy`|The strategy to expose prometheus metrics. Default strategy `exporter` would expose metrics for scraping purpose. Only peon task (short-lived jobs) need to use `pushgateway` strategy.|yes|scraping| +|`druid.emitter.prometheus.strategy`|The strategy to expose prometheus metrics. Default strategy `exporter` would expose metrics for scraping purpose. Only peon task (short-lived jobs) need to use `pushgateway` strategy.|yes|exporter| |`druid.emitter.prometheus.port`|The port on which to expose the prometheus HTTPServer. Required if using exporter strategy.|no|none| |`druid.emitter.prometheus.namespace`|Optional metric namespace. Must match the regex `[a-zA-Z_:][a-zA-Z0-9_:]*`|no|"druid"| |`druid.emitter.prometheus.dimensionMapPath`|JSON file defining the Prometheus metric type, desired dimensions, help text, and conversionFactor for every Druid metric.|no|Default mapping provided. See below.| From 45412cc55dad7c84397f9bac1c5aba2e475a04f7 Mon Sep 17 00:00:00 2001 From: Tianxin Zhao Date: Mon, 8 Jun 2020 17:53:39 -0700 Subject: [PATCH 30/42] Format fix and nullable strategy --- .../apache/druid/emitter/prometheus/Metrics.java | 3 ++- .../emitter/prometheus/PrometheusEmitter.java | 11 ++++++----- .../prometheus/PrometheusEmitterConfig.java | 14 ++++++++------ 3 files changed, 16 insertions(+), 12 deletions(-) diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java index a5d1861a2e16..fd62a7b35ccc 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java @@ -123,7 +123,8 @@ private Map readMap(String path) } } - public Map getMap(){ + public Map getMap() + { return map; } diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java index 1292567f1006..945f46100ed9 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -67,7 +67,7 @@ public PrometheusEmitter(PrometheusEmitterConfig config) @Override public void start() { - if(strategy.equals(PrometheusEmitterConfig.Strategy.exporter)){ + if (strategy.equals(PrometheusEmitterConfig.Strategy.exporter)) { if (server == null) { try { server = new HTTPServer(config.getPort()); @@ -78,7 +78,7 @@ public void start() } else { log.error("HTTPServer is already started"); } - } else if (strategy.equals(PrometheusEmitterConfig.Strategy.pushgateway)){ + } else if (strategy.equals(PrometheusEmitterConfig.Strategy.pushgateway)) { pushGateway = new PushGateway(config.getPushGatewayAddress()); } @@ -131,9 +131,10 @@ public void flush() Map map = metrics.getMap(); try { for (DimensionsAndCollector collector : map.values()) { - pushGateway.push(collector.getCollector(), config.getNamespace(), ImmutableMap.of(config.getNamespace(), identifier)); + pushGateway.push(collector.getCollector(), config.getNamespace(), ImmutableMap.of(config.getNamespace(), identifier)); } - } catch(IOException e){ + } + catch(IOException e){ log.error(e, "Unable to push prometheus metrics to pushGateway"); } } @@ -141,7 +142,7 @@ public void flush() @Override public void close() { - if(strategy.equals(PrometheusEmitterConfig.Strategy.exporter)){ + if (strategy.equals(PrometheusEmitterConfig.Strategy.exporter)) { if (server != null) { server.stop(); } diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java index f713e85ab7eb..32439f395cb6 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java @@ -24,7 +24,7 @@ import com.google.common.base.Preconditions; import javax.annotation.Nullable; -import javax.validation.constraints.Null; +import javax.xml.ws.BindingType; import java.util.regex.Pattern; /** @@ -56,15 +56,15 @@ public class PrometheusEmitterConfig @JsonCreator public PrometheusEmitterConfig( - @JsonProperty("strategy") Strategy strategy, + @JsonProperty("strategy") @Nullable Strategy strategy, @JsonProperty("namespace") @Nullable String namespace, @JsonProperty("dimensionMapPath") @Nullable String dimensionMapPath, - @JsonProperty("port") @Nullable Integer port, + @JsonProperty("port") @Nullable Integer port, @JsonProperty("pushGatewayAddress") @Nullable String pushGatewayAddress ) { - this.strategy = Preconditions.checkNotNull(strategy, "Prometheus metrics expose strategy cannot be null"); + this.strategy = strategy != null ? strategy : Strategy.exporter; this.namespace = namespace != null ? namespace : "druid"; Preconditions.checkArgument(pattern.matcher(this.namespace).matches(), "Invalid namespace " + this.namespace); this.dimensionMapPath = dimensionMapPath; @@ -87,11 +87,13 @@ public int getPort() return port; } - public String getPushGatewayAddress() { + public String getPushGatewayAddress() + { return pushGatewayAddress; } - public Strategy getStrategy(){ + public Strategy getStrategy() + { return strategy; } From 6c248ce6f863413077b22de7a25030c9f57caccf Mon Sep 17 00:00:00 2001 From: Tianxin Zhao Date: Sun, 20 Sep 2020 23:12:26 -0700 Subject: [PATCH 31/42] Update pom file for prometheus-emitter --- extensions-contrib/prometheus-emitter/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-contrib/prometheus-emitter/pom.xml b/extensions-contrib/prometheus-emitter/pom.xml index ea6864398e5b..a44e9fe5321f 100644 --- a/extensions-contrib/prometheus-emitter/pom.xml +++ b/extensions-contrib/prometheus-emitter/pom.xml @@ -23,7 +23,7 @@ druid org.apache.druid - 0.18.0-SNAPSHOT + 0.20.0-SNAPSHOT ../../pom.xml 4.0.0 From 7d4f7656799fdeaabb94e0fb199d2af9c4489de0 Mon Sep 17 00:00:00 2001 From: Tianxin Zhao Date: Sun, 1 Nov 2020 15:15:35 -0800 Subject: [PATCH 32/42] code review comments. Counter to gauge for cache metrics, periodical task to pushGateway --- .../druid/emitter/prometheus/Metrics.java | 30 ++++++++--------- .../emitter/prometheus/PrometheusEmitter.java | 33 ++++++++++++------- .../prometheus/PrometheusEmitterConfig.java | 2 +- .../src/main/resources/defaultMetrics.json | 18 +++++----- 4 files changed, 45 insertions(+), 38 deletions(-) diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java index fd62a7b35ccc..0a6395b45482 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java @@ -45,24 +45,20 @@ public class Metrics { private static final Logger log = new Logger(Metrics.class); - private final Map map = new HashMap<>(); + private final Map registeredMetrics = new HashMap<>(); private final ObjectMapper mapper = new ObjectMapper(); - private final Pattern pattern = Pattern.compile("[^a-zA-Z_:][^a-zA-Z0-9_:]*"); + public static final Pattern pattern = Pattern.compile("[^a-zA-Z_:][^a-zA-Z0-9_:]*"); public DimensionsAndCollector getByName(String name, String service) { - if (map.containsKey(name)) { - return map.get(name); - } else if (map.containsKey(service + "_" + name)) { - return map.get(service + "_" + name); - } else { - return null; - } + if (registeredMetrics.containsKey(name)) { + return registeredMetrics.get(name); + } else return registeredMetrics.getOrDefault(service + "_" + name, null); } public Metrics(String namespace, String path) { - Map metrics = readMap(path); + Map metrics = readConfig(path); for (Map.Entry entry : metrics.entrySet()) { String name = entry.getKey(); Metric metric = entry.getValue(); @@ -97,21 +93,21 @@ public Metrics(String namespace, String path) } if (collector != null) { - map.put(name, new DimensionsAndCollector(dimensions, collector, metric.conversionFactor)); + registeredMetrics.put(name, new DimensionsAndCollector(dimensions, collector, metric.conversionFactor)); } } } - private Map readMap(String path) + private Map readConfig(String path) { try { InputStream is; if (Strings.isNullOrEmpty(path)) { - log.info("Using default metric dimension and types"); + log.info("Using default metric configuration"); is = this.getClass().getClassLoader().getResourceAsStream("defaultMetrics.json"); } else { - log.info("Using metric dimensions at types at [%s]", path); + log.info("Using metric configuration at [%s]", path); is = new FileInputStream(new File(path)); } return mapper.readerFor(new TypeReference>() @@ -119,13 +115,13 @@ private Map readMap(String path) }).readValue(is); } catch (IOException e) { - throw new ISE(e, "Failed to parse metric dimensions and types"); + throw new ISE(e, "Failed to parse metric configuration"); } } - public Map getMap() + public Map getRegisteredMetrics() { - return map; + return registeredMetrics; } public static class Metric diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java index 945f46100ed9..86fba72d656c 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -26,6 +26,7 @@ import io.prometheus.client.Histogram; import io.prometheus.client.exporter.HTTPServer; import io.prometheus.client.exporter.PushGateway; +import org.apache.commons.lang3.concurrent.BasicThreadFactory; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.core.Emitter; import org.apache.druid.java.util.emitter.core.Event; @@ -33,24 +34,28 @@ import java.io.IOException; import java.util.Map; -import java.util.regex.Pattern; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; /** * */ public class PrometheusEmitter implements Emitter { - private static final Logger log = new Logger(PrometheusEmitter.class); private final Metrics metrics; private final PrometheusEmitterConfig config; private final PrometheusEmitterConfig.Strategy strategy; - private final Pattern pattern = Pattern.compile("[^a-zA-Z0-9_][^a-zA-Z0-9_]*"); private HTTPServer server; private PushGateway pushGateway; private String identifier; + private static ScheduledExecutorService PUSHGATE_WAY_EXECUTOR_SERVICE = + Executors.newSingleThreadScheduledExecutor( + new BasicThreadFactory.Builder().namingPattern("Metric-Pusher-%d").daemon(true).build()); + static PrometheusEmitter of(PrometheusEmitterConfig config) { return new PrometheusEmitter(config); @@ -80,6 +85,7 @@ public void start() } } else if (strategy.equals(PrometheusEmitterConfig.Strategy.pushgateway)) { pushGateway = new PushGateway(config.getPushGatewayAddress()); + PUSHGATE_WAY_EXECUTOR_SERVICE.scheduleAtFixedRate(this::pushMetric, 1L, 5L, TimeUnit.MINUTES); } } @@ -92,7 +98,7 @@ public void emit(Event event) } } - void emitMetric(ServiceMetricEvent metricEvent) + private void emitMetric(ServiceMetricEvent metricEvent) { String name = metricEvent.getMetric(); String service = metricEvent.getService(); @@ -108,7 +114,7 @@ void emitMetric(ServiceMetricEvent metricEvent) String labelName = labelNames[i]; //labelName is controlled by the user. Instead of potential NPE on invalid labelName we use "unknown" as the dimension value Object userDim = userDims.get(labelName); - labelValues[i] = userDim != null ? pattern.matcher(userDim.toString()).replaceAll("_") : "unknown"; + labelValues[i] = userDim != null ? Metrics.pattern.matcher(userDim.toString()).replaceAll("_") : "unknown"; } if (metric.getCollector() instanceof Counter) { @@ -125,20 +131,25 @@ void emitMetric(ServiceMetricEvent metricEvent) } } - @Override - public void flush() - { - Map map = metrics.getMap(); + private void pushMetric(){ + Map map = metrics.getRegisteredMetrics(); try { for (DimensionsAndCollector collector : map.values()) { pushGateway.push(collector.getCollector(), config.getNamespace(), ImmutableMap.of(config.getNamespace(), identifier)); } - } - catch(IOException e){ + } catch (IOException e) { log.error(e, "Unable to push prometheus metrics to pushGateway"); } } + @Override + public void flush() + { + if(pushGateway != null) { + pushMetric(); + } + } + @Override public void close() { diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java index 32439f395cb6..3a7000be3da6 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java @@ -33,7 +33,7 @@ public class PrometheusEmitterConfig { - Pattern pattern = Pattern.compile("[a-zA-Z_:][a-zA-Z0-9_:]*"); + final static Pattern pattern = Pattern.compile("[a-zA-Z_:][a-zA-Z0-9_:]*"); @JsonProperty private final Strategy strategy; diff --git a/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json index 90770d7b4fc9..c89d41e68eac 100644 --- a/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json +++ b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json @@ -18,15 +18,15 @@ "query/failed/count" : { "dimensions" : [], "type" : "count", "help": "Number of failed queries"}, "query/interrupted/count" : { "dimensions" : [], "type" : "count", "help": "Number of queries interrupted due to cancellation or timeout"}, - "query/cache/delta/numEntries" : { "dimensions" : [], "type" : "count", "help": "Number of entries in cache"}, - "query/cache/delta/sizeBytes" : { "dimensions" : [], "type" : "count", "help": "Size of cache in bytes."}, - "query/cache/delta/hits" : { "dimensions" : [], "type" : "count", "help": "Number of cache hits."}, - "query/cache/delta/misses" : { "dimensions" : [], "type" : "count", "help": "Number of cache misses."}, - "query/cache/delta/evictions" : { "dimensions" : [], "type" : "count", "help": "Number of cache evictions."}, - "query/cache/delta/hitRate" : { "dimensions" : [], "type" : "count","help": "Cache hit rate."}, - "query/cache/delta/averageBytes" : { "dimensions" : [], "type" : "count", "help": "Average size of record in bytes"}, - "query/cache/delta/timeouts" : { "dimensions" : [], "type" : "count", "help": "Number of cache timeouts"}, - "query/cache/delta/errors" : { "dimensions" : [], "type" : "count", "help": "Number of cache errors."}, + "query/cache/delta/numEntries" : { "dimensions" : [], "type" : "gauge", "help": "Number of entries in cache"}, + "query/cache/delta/sizeBytes" : { "dimensions" : [], "type" : "gauge", "help": "Size of cache in bytes."}, + "query/cache/delta/hits" : { "dimensions" : [], "type" : "gauge", "help": "Number of cache hits."}, + "query/cache/delta/misses" : { "dimensions" : [], "type" : "gauge", "help": "Number of cache misses."}, + "query/cache/delta/evictions" : { "dimensions" : [], "type" : "gauge", "help": "Number of cache evictions."}, + "query/cache/delta/hitRate" : { "dimensions" : [], "type" : "gauge","help": "Cache hit rate."}, + "query/cache/delta/averageBytes" : { "dimensions" : [], "type" : "gauge", "help": "Average size of record in bytes"}, + "query/cache/delta/timeouts" : { "dimensions" : [], "type" : "gauge", "help": "Number of cache timeouts"}, + "query/cache/delta/errors" : { "dimensions" : [], "type" : "gauge", "help": "Number of cache errors."}, "query/cache/total/numEntries" : { "dimensions" : [], "type" : "gauge","help": "Total number of entries in cache" }, "query/cache/total/sizeBytes" : { "dimensions" : [], "type" : "gauge", "help": "Total size of cache in bytes."}, From 4d90a419d93c2a929d7f2ae38a72f40fc16567ea Mon Sep 17 00:00:00 2001 From: Tianxin Zhao Date: Mon, 4 Jan 2021 18:18:32 -0800 Subject: [PATCH 33/42] Syntax fix --- extensions-contrib/prometheus-emitter/pom.xml | 2 +- .../org/apache/druid/emitter/prometheus/Metrics.java | 8 +++++--- .../druid/emitter/prometheus/PrometheusEmitter.java | 10 ++++++---- .../emitter/prometheus/PrometheusEmitterConfig.java | 5 ++--- 4 files changed, 14 insertions(+), 11 deletions(-) diff --git a/extensions-contrib/prometheus-emitter/pom.xml b/extensions-contrib/prometheus-emitter/pom.xml index a44e9fe5321f..9345fcc135ce 100644 --- a/extensions-contrib/prometheus-emitter/pom.xml +++ b/extensions-contrib/prometheus-emitter/pom.xml @@ -23,7 +23,7 @@ druid org.apache.druid - 0.20.0-SNAPSHOT + 0.21.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java index 0a6395b45482..7006da3ca8a7 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java @@ -47,13 +47,15 @@ public class Metrics private static final Logger log = new Logger(Metrics.class); private final Map registeredMetrics = new HashMap<>(); private final ObjectMapper mapper = new ObjectMapper(); - public static final Pattern pattern = Pattern.compile("[^a-zA-Z_:][^a-zA-Z0-9_:]*"); + public static final Pattern PATTERN = Pattern.compile("[^a-zA-Z_:][^a-zA-Z0-9_:]*"); public DimensionsAndCollector getByName(String name, String service) { if (registeredMetrics.containsKey(name)) { return registeredMetrics.get(name); - } else return registeredMetrics.getOrDefault(service + "_" + name, null); + } else { + return registeredMetrics.getOrDefault(service + "_" + name, null); + } } public Metrics(String namespace, String path) @@ -64,7 +66,7 @@ public Metrics(String namespace, String path) Metric metric = entry.getValue(); Metric.Type type = metric.type; String[] dimensions = metric.dimensions.toArray(new String[0]); - String formattedName = pattern.matcher(StringUtils.toLowerCase(name)).replaceAll("_"); + String formattedName = PATTERN.matcher(StringUtils.toLowerCase(name)).replaceAll("_"); SimpleCollector collector = null; if (Metric.Type.count.equals(type)) { collector = new Counter.Builder() diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java index 86fba72d656c..7612f8b66366 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -114,7 +114,7 @@ private void emitMetric(ServiceMetricEvent metricEvent) String labelName = labelNames[i]; //labelName is controlled by the user. Instead of potential NPE on invalid labelName we use "unknown" as the dimension value Object userDim = userDims.get(labelName); - labelValues[i] = userDim != null ? Metrics.pattern.matcher(userDim.toString()).replaceAll("_") : "unknown"; + labelValues[i] = userDim != null ? Metrics.PATTERN.matcher(userDim.toString()).replaceAll("_") : "unknown"; } if (metric.getCollector() instanceof Counter) { @@ -131,13 +131,15 @@ private void emitMetric(ServiceMetricEvent metricEvent) } } - private void pushMetric(){ + private void pushMetric() + { Map map = metrics.getRegisteredMetrics(); try { for (DimensionsAndCollector collector : map.values()) { pushGateway.push(collector.getCollector(), config.getNamespace(), ImmutableMap.of(config.getNamespace(), identifier)); } - } catch (IOException e) { + } + catch (IOException e) { log.error(e, "Unable to push prometheus metrics to pushGateway"); } } @@ -145,7 +147,7 @@ private void pushMetric(){ @Override public void flush() { - if(pushGateway != null) { + if (pushGateway != null) { pushMetric(); } } diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java index 3a7000be3da6..8a6e7a715a7f 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java @@ -24,7 +24,6 @@ import com.google.common.base.Preconditions; import javax.annotation.Nullable; -import javax.xml.ws.BindingType; import java.util.regex.Pattern; /** @@ -33,7 +32,7 @@ public class PrometheusEmitterConfig { - final static Pattern pattern = Pattern.compile("[a-zA-Z_:][a-zA-Z0-9_:]*"); + static final Pattern PATTERN = Pattern.compile("[a-zA-Z_:][a-zA-Z0-9_:]*"); @JsonProperty private final Strategy strategy; @@ -66,7 +65,7 @@ public PrometheusEmitterConfig( this.strategy = strategy != null ? strategy : Strategy.exporter; this.namespace = namespace != null ? namespace : "druid"; - Preconditions.checkArgument(pattern.matcher(this.namespace).matches(), "Invalid namespace " + this.namespace); + Preconditions.checkArgument(PATTERN.matcher(this.namespace).matches(), "Invalid namespace " + this.namespace); this.dimensionMapPath = dimensionMapPath; this.port = port; this.pushGatewayAddress = pushGatewayAddress; From 3a7a2b620309f2353830366d87d8da5a37cdce6f Mon Sep 17 00:00:00 2001 From: Tianxin Zhao Date: Tue, 5 Jan 2021 00:32:56 -0800 Subject: [PATCH 34/42] Dimension label regex include numeric character back, fix previous commit --- .../apache/druid/emitter/prometheus/PrometheusEmitter.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java index 7612f8b66366..bf98bbba45f3 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -37,6 +37,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.regex.Pattern; /** * @@ -47,6 +48,7 @@ public class PrometheusEmitter implements Emitter private final Metrics metrics; private final PrometheusEmitterConfig config; private final PrometheusEmitterConfig.Strategy strategy; + private static final Pattern PATTERN = Pattern.compile("[^a-zA-Z0-9_][^a-zA-Z0-9_]*"); private HTTPServer server; private PushGateway pushGateway; @@ -114,7 +116,7 @@ private void emitMetric(ServiceMetricEvent metricEvent) String labelName = labelNames[i]; //labelName is controlled by the user. Instead of potential NPE on invalid labelName we use "unknown" as the dimension value Object userDim = userDims.get(labelName); - labelValues[i] = userDim != null ? Metrics.PATTERN.matcher(userDim.toString()).replaceAll("_") : "unknown"; + labelValues[i] = userDim != null ? PATTERN.matcher(userDim.toString()).replaceAll("_") : "unknown"; } if (metric.getCollector() instanceof Counter) { From b61f1b3129a06afd527adbc8cc60228496e04473 Mon Sep 17 00:00:00 2001 From: Tianxin Zhao Date: Fri, 15 Jan 2021 18:33:25 -0800 Subject: [PATCH 35/42] bump prometheus-emitter pom dev version --- extensions-contrib/prometheus-emitter/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-contrib/prometheus-emitter/pom.xml b/extensions-contrib/prometheus-emitter/pom.xml index 9345fcc135ce..f7d7536991fa 100644 --- a/extensions-contrib/prometheus-emitter/pom.xml +++ b/extensions-contrib/prometheus-emitter/pom.xml @@ -23,7 +23,7 @@ druid org.apache.druid - 0.21.0-SNAPSHOT + 0.22.0-SNAPSHOT ../../pom.xml 4.0.0 From b1b8d61a8b62702d31defaa6ed7a7036436c38b6 Mon Sep 17 00:00:00 2001 From: Tianxin Zhao Date: Fri, 15 Jan 2021 18:53:30 -0800 Subject: [PATCH 36/42] Remove scheduled task inside poen that push metrics --- .../apache/druid/emitter/prometheus/PrometheusEmitter.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java index bf98bbba45f3..ed64bbc06478 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -54,10 +54,6 @@ public class PrometheusEmitter implements Emitter private PushGateway pushGateway; private String identifier; - private static ScheduledExecutorService PUSHGATE_WAY_EXECUTOR_SERVICE = - Executors.newSingleThreadScheduledExecutor( - new BasicThreadFactory.Builder().namingPattern("Metric-Pusher-%d").daemon(true).build()); - static PrometheusEmitter of(PrometheusEmitterConfig config) { return new PrometheusEmitter(config); @@ -87,7 +83,6 @@ public void start() } } else if (strategy.equals(PrometheusEmitterConfig.Strategy.pushgateway)) { pushGateway = new PushGateway(config.getPushGatewayAddress()); - PUSHGATE_WAY_EXECUTOR_SERVICE.scheduleAtFixedRate(this::pushMetric, 1L, 5L, TimeUnit.MINUTES); } } From e7e9e2fc4df68cc89c8828ecf05bd2acb830b8ae Mon Sep 17 00:00:00 2001 From: Tianxin Zhao Date: Sun, 7 Feb 2021 14:35:24 -0800 Subject: [PATCH 37/42] Fix checkstyle --- docs/development/extensions-contrib/prometheus.md | 2 +- .../apache/druid/emitter/prometheus/PrometheusEmitter.java | 4 ---- .../druid/emitter/prometheus/PrometheusEmitterModule.java | 2 +- website/.spelling | 1 + 4 files changed, 3 insertions(+), 6 deletions(-) diff --git a/docs/development/extensions-contrib/prometheus.md b/docs/development/extensions-contrib/prometheus.md index 5f695f23c5cd..82f716c73bb9 100644 --- a/docs/development/extensions-contrib/prometheus.md +++ b/docs/development/extensions-contrib/prometheus.md @@ -40,7 +40,7 @@ All the configuration parameters for the Prometheus emitter are under `druid.emi |`druid.emitter.prometheus.port`|The port on which to expose the prometheus HTTPServer. Required if using exporter strategy.|no|none| |`druid.emitter.prometheus.namespace`|Optional metric namespace. Must match the regex `[a-zA-Z_:][a-zA-Z0-9_:]*`|no|"druid"| |`druid.emitter.prometheus.dimensionMapPath`|JSON file defining the Prometheus metric type, desired dimensions, help text, and conversionFactor for every Druid metric.|no|Default mapping provided. See below.| -|`druid.emitter.prometheus.pushGatewayAddress`|Pushgateway address. Required if using pushgateway strategy|no|none| +|`druid.emitter.prometheus.pushGatewayAddress`|Pushgateway address. Required if using Pushgateway strategy|no|none| ### Metric names diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java index ed64bbc06478..4154c8dd21de 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -26,7 +26,6 @@ import io.prometheus.client.Histogram; import io.prometheus.client.exporter.HTTPServer; import io.prometheus.client.exporter.PushGateway; -import org.apache.commons.lang3.concurrent.BasicThreadFactory; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.core.Emitter; import org.apache.druid.java.util.emitter.core.Event; @@ -34,9 +33,6 @@ import java.io.IOException; import java.util.Map; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; /** diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java index bb26c4d37d58..eb74aa7022b7 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java @@ -43,7 +43,7 @@ public class PrometheusEmitterModule implements DruidModule @Override public List getJacksonModules() { - return Collections.EMPTY_LIST; + return Collections.emptyList(); } @Override diff --git a/website/.spelling b/website/.spelling index 31245fb8f4ac..3b60b1d952bc 100644 --- a/website/.spelling +++ b/website/.spelling @@ -586,6 +586,7 @@ convertRange HTTPServer conversionFactor prometheus +Pushgateway - ../docs/development/extensions-contrib/tdigestsketch-quantiles.md postAggregator quantileFromTDigestSketch From b405b558b2176a6262b72ba4bd15ac0f3e1dc895 Mon Sep 17 00:00:00 2001 From: Tianxin Zhao Date: Wed, 10 Feb 2021 12:46:30 -0800 Subject: [PATCH 38/42] Unit test coverage --- .../druid/emitter/prometheus/MetricsTest.java | 6 ++++- .../prometheus/PrometheusEmitterTest.java | 25 ++++++++++++++++++- 2 files changed, 29 insertions(+), 2 deletions(-) diff --git a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java index dfd71ba1c95a..968b29e951d4 100644 --- a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java +++ b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java @@ -30,12 +30,16 @@ public void testMetricsConfiguration() { Metrics metrics = new Metrics("test", null); DimensionsAndCollector dimensionsAndCollector = metrics.getByName("query/time", "historical"); - DimensionsAndCollector d = metrics.getByName("segment/loadQueue/count", "historical"); Assert.assertNotNull(dimensionsAndCollector); String[] dimensions = dimensionsAndCollector.getDimensions(); Assert.assertEquals("dataSource", dimensions[0]); Assert.assertEquals("type", dimensions[1]); Assert.assertEquals(1000.0, dimensionsAndCollector.getConversionFactor(), 0.0); Assert.assertTrue(dimensionsAndCollector.getCollector() instanceof Histogram); + + DimensionsAndCollector d = metrics.getByName("segment/loadQueue/count", "historical"); + Assert.assertNotNull(d); + String[] dims = d.getDimensions(); + Assert.assertEquals("server", dims[0]); } } diff --git a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java index fc2fdd5f7600..34c32ded63ab 100644 --- a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java +++ b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableMap; import io.prometheus.client.CollectorRegistry; +import org.apache.druid.java.util.emitter.core.Emitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.junit.Assert; import org.junit.Test; @@ -32,7 +33,8 @@ public class PrometheusEmitterTest public void testEmitter() { PrometheusEmitterConfig config = new PrometheusEmitterConfig(PrometheusEmitterConfig.Strategy.exporter, null, null, 0, null); - PrometheusEmitter emitter = new PrometheusEmitter(config); + PrometheusEmitterModule prometheusEmitterModule = new PrometheusEmitterModule(); + Emitter emitter = prometheusEmitterModule.getEmitter(config); ServiceMetricEvent build = ServiceMetricEvent.builder() .setDimension("server", "druid-data01.vpc.region") .build("segment/loadQueue/count", 10) @@ -45,4 +47,25 @@ public void testEmitter() ); Assert.assertEquals(10, count.intValue()); } + + @Test + public void testEmitterMetric() + { + PrometheusEmitterConfig config = new PrometheusEmitterConfig(PrometheusEmitterConfig.Strategy.pushgateway, "namespace", null, 0, "localhost"); + PrometheusEmitterModule prometheusEmitterModule = new PrometheusEmitterModule(); + Emitter emitter = prometheusEmitterModule.getEmitter(config); + ServiceMetricEvent build = ServiceMetricEvent.builder() + .setDimension("dataSource", "test") + .setDimension("taskType", "index_parallel") + .build("task/run/time", 500) + .build(ImmutableMap.of("service", "overlord")); + emitter.emit(build); + double assertEpsilon = 0.0001; + Assert.assertEquals(0.0, CollectorRegistry.defaultRegistry.getSampleValue( + "namespace_task_run_time_bucket", new String[]{"dataSource", "taskType", "le"}, new String[]{"test", "index_parallel", "0.1"} + ), assertEpsilon); + Assert.assertEquals(1.0, CollectorRegistry.defaultRegistry.getSampleValue( + "namespace_task_run_time_bucket", new String[]{"dataSource", "taskType", "le"}, new String[]{"test", "index_parallel", "0.5"} + ), assertEpsilon); + } } From 8f3463fc4d4675c69a46ac1026df1dad78aecc79 Mon Sep 17 00:00:00 2001 From: Tianxin Zhao Date: Mon, 15 Feb 2021 16:07:36 -0800 Subject: [PATCH 39/42] Unit test coverage --- .../emitter/prometheus/PrometheusEmitter.java | 19 +++++++- .../prometheus/PrometheusEmitterTest.java | 43 ++++++++++++++++++- 2 files changed, 60 insertions(+), 2 deletions(-) diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java index 4154c8dd21de..19d047fd29cb 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -129,7 +129,9 @@ private void pushMetric() Map map = metrics.getRegisteredMetrics(); try { for (DimensionsAndCollector collector : map.values()) { - pushGateway.push(collector.getCollector(), config.getNamespace(), ImmutableMap.of(config.getNamespace(), identifier)); + if (config.getNamespace() != null) { + pushGateway.push(collector.getCollector(), config.getNamespace(), ImmutableMap.of(config.getNamespace(), identifier)); + } } } catch (IOException e) { @@ -156,4 +158,19 @@ public void close() flush(); } } + + public HTTPServer getServer() + { + return server; + } + + public PushGateway getPushGateway() + { + return pushGateway; + } + + public void setPushGateway(PushGateway pushGateway) + { + this.pushGateway = pushGateway; + } } diff --git a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java index 34c32ded63ab..c9ca139b06d0 100644 --- a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java +++ b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java @@ -20,12 +20,20 @@ package org.apache.druid.emitter.prometheus; import com.google.common.collect.ImmutableMap; +import io.prometheus.client.Collector; import io.prometheus.client.CollectorRegistry; +import io.prometheus.client.exporter.PushGateway; import org.apache.druid.java.util.emitter.core.Emitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.junit.Assert; import org.junit.Test; +import java.io.IOException; + +import static org.easymock.EasyMock.anyObject; +import static org.easymock.EasyMock.anyString; +import static org.easymock.EasyMock.mock; + public class PrometheusEmitterTest { @@ -51,7 +59,7 @@ public void testEmitter() @Test public void testEmitterMetric() { - PrometheusEmitterConfig config = new PrometheusEmitterConfig(PrometheusEmitterConfig.Strategy.pushgateway, "namespace", null, 0, "localhost"); + PrometheusEmitterConfig config = new PrometheusEmitterConfig(PrometheusEmitterConfig.Strategy.pushgateway, "namespace", null, 0, "pushgateway"); PrometheusEmitterModule prometheusEmitterModule = new PrometheusEmitterModule(); Emitter emitter = prometheusEmitterModule.getEmitter(config); ServiceMetricEvent build = ServiceMetricEvent.builder() @@ -68,4 +76,37 @@ public void testEmitterMetric() "namespace_task_run_time_bucket", new String[]{"dataSource", "taskType", "le"}, new String[]{"test", "index_parallel", "0.5"} ), assertEpsilon); } + + @Test + public void testEmitterStart() + { + PrometheusEmitterConfig exportEmitterConfig = new PrometheusEmitterConfig(PrometheusEmitterConfig.Strategy.exporter, "namespace1", null, 0, null); + PrometheusEmitter exportEmitter = new PrometheusEmitter(exportEmitterConfig); + exportEmitter.start(); + Assert.assertNotNull(exportEmitter.getServer()); + + PrometheusEmitterConfig pushEmitterConfig = new PrometheusEmitterConfig(PrometheusEmitterConfig.Strategy.pushgateway, "namespace2", null, 0, "pushgateway"); + PrometheusEmitter pushEmitter = new PrometheusEmitter(pushEmitterConfig); + pushEmitter.start(); + Assert.assertNotNull(pushEmitter.getPushGateway()); + } + + @Test + public void testEmitterPush() throws IOException + { + PrometheusEmitterConfig emitterConfig = new PrometheusEmitterConfig(PrometheusEmitterConfig.Strategy.pushgateway, "namespace3", null, 0, "pushgateway"); + + PushGateway mockPushGateway = mock(PushGateway.class); + mockPushGateway.push(anyObject(Collector.class), anyString(), anyObject(ImmutableMap.class)); + + PrometheusEmitter emitter = new PrometheusEmitter(emitterConfig); + emitter.start(); + emitter.setPushGateway(mockPushGateway); + ServiceMetricEvent build = ServiceMetricEvent.builder() + .setDimension("task", "index_parallel") + .build("task/run/time", 500) + .build(ImmutableMap.of("service", "peon")); + emitter.emit(build); + emitter.flush(); + } } From 5cafe2f298f4c0964a1027df9d29bbfe9e80ab4c Mon Sep 17 00:00:00 2001 From: Tianxin Zhao Date: Mon, 15 Feb 2021 17:28:07 -0800 Subject: [PATCH 40/42] Spelling --- website/.spelling | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/.spelling b/website/.spelling index e26f6dd998a5..183bbf80bcd2 100644 --- a/website/.spelling +++ b/website/.spelling @@ -1228,7 +1228,7 @@ SysMonitor TaskCountStatsMonitor TaskSlotCountStatsMonitor bufferCapacity -bufferpoolName +bufferPoolName cms cpuName cpuTime From 42048a57524fdcd63851b8c70d0bfde6d27c226f Mon Sep 17 00:00:00 2001 From: Tianxin Zhao Date: Mon, 1 Mar 2021 17:23:12 -0800 Subject: [PATCH 41/42] Doc fix --- docs/development/extensions-contrib/prometheus.md | 6 +++--- docs/operations/metrics.md | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/development/extensions-contrib/prometheus.md b/docs/development/extensions-contrib/prometheus.md index 82f716c73bb9..2c7ea089ab1c 100644 --- a/docs/development/extensions-contrib/prometheus.md +++ b/docs/development/extensions-contrib/prometheus.md @@ -23,12 +23,12 @@ title: "Prometheus Emitter" --> -To use this Apache Druid (incubating) extension, make sure to [include](../../development/extensions.md#loading-extensions) `prometheus-emitter` extension. +To use this Apache Druid extension, make sure to [include](../../development/extensions.md#loading-extensions) `prometheus-emitter` extension. ## Introduction -This extension exposes Druid metrics for collection by a Prometheus server -(https://prometheus.io/) +This extension exposes [Druid metrics](https://druid.apache.org/docs/latest/operations/metrics.html) for collection by a Prometheus server (https://prometheus.io/). +Emitter is enabled by setting `druid.emitter=prometheus` [configs](https://druid.apache.org/docs/latest/configuration/index.html#emitting-metrics) or include `prometheus` in the composing emitter list. ## Configuration diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md index db7d22f73bf2..9a1d2c7e4983 100644 --- a/docs/operations/metrics.md +++ b/docs/operations/metrics.md @@ -284,9 +284,9 @@ These metrics are only available if the JVMMonitor module is included. |`jvm/pool/init`|Initial pool.|poolKind, poolName.|Varies.| |`jvm/pool/max`|Max pool.|poolKind, poolName.|Varies.| |`jvm/pool/used`|Pool used.|poolKind, poolName.|< max pool| -|`jvm/bufferpool/count`|Bufferpool count.|bufferPoolName.|Varies.| -|`jvm/bufferpool/used`|Bufferpool used.|bufferPoolName.|close to capacity| -|`jvm/bufferpool/capacity`|Bufferpool capacity.|bufferPoolName.|Varies.| +|`jvm/bufferpool/count`|Bufferpool count.|bufferpoolName.|Varies.| +|`jvm/bufferpool/used`|Bufferpool used.|bufferpoolName.|close to capacity| +|`jvm/bufferpool/capacity`|Bufferpool capacity.|bufferpoolName.|Varies.| |`jvm/mem/init`|Initial memory.|memKind.|Varies.| |`jvm/mem/max`|Max memory.|memKind.|Varies.| |`jvm/mem/used`|Used memory.|memKind.|< max memory| From 807f8dded488869f2147fe7337c3b64f64ec4e78 Mon Sep 17 00:00:00 2001 From: Tianxin Zhao Date: Tue, 2 Mar 2021 12:00:09 -0800 Subject: [PATCH 42/42] spelling --- website/.spelling | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/.spelling b/website/.spelling index 183bbf80bcd2..e26f6dd998a5 100644 --- a/website/.spelling +++ b/website/.spelling @@ -1228,7 +1228,7 @@ SysMonitor TaskCountStatsMonitor TaskSlotCountStatsMonitor bufferCapacity -bufferPoolName +bufferpoolName cms cpuName cpuTime