From 020e0fff27d775f006a70c6f13d94610b658003f Mon Sep 17 00:00:00 2001 From: Janos Matyas Date: Tue, 17 Oct 2017 20:51:50 +0200 Subject: [PATCH 1/4] Add support for prometheus --- conf/metrics.properties.template | 8 + core/pom.xml | 17 +- .../exporter/PushGatewayWithTimestamp.java | 296 ++++++++++++++++++ .../exporter/TextFormatWithTimestamp.java | 174 ++++++++++ .../spark/metrics/sink/PrometheusSink.scala | 157 ++++++++++ pom.xml | 26 +- 6 files changed, 670 insertions(+), 8 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/metrics/prometheus/client/exporter/PushGatewayWithTimestamp.java create mode 100644 core/src/main/java/org/apache/spark/metrics/prometheus/client/exporter/TextFormatWithTimestamp.java create mode 100644 core/src/main/scala/org/apache/spark/metrics/sink/PrometheusSink.scala diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template index aeb76c9b2f6ea..8da34b110e9af 100644 --- a/conf/metrics.properties.template +++ b/conf/metrics.properties.template @@ -160,6 +160,14 @@ # Unit of the polling period for the Slf4jSink #*.sink.slf4j.unit=minutes +# Enable Prometheus for all instances by class name +#*.sink.prometheus.class=org.apache.spark.metrics.sink.PrometheusSink + +#Prometheus host +#*.sink.prometheus.pushgateway-address= - defaults to 127.0.0.1:9091 +#*.sink.prometheus.unit=< unit> - defaults to seconds (TimeUnit.SECONDS) +#*.sink.prometheus.period= - defaults to 10 + # Enable JvmSource for instance master, worker, driver and executor #master.source.jvm.class=org.apache.spark.metrics.source.JvmSource diff --git a/core/pom.xml b/core/pom.xml index 15488ae7eafcc..be1e29f408f39 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -257,6 +257,18 @@ io.dropwizard.metrics metrics-graphite + + io.prometheus + simpleclient + + + io.prometheus + simpleclient_dropwizard + + + io.prometheus + simpleclient_pushgateway + com.fasterxml.jackson.core jackson-databind @@ -407,7 +419,7 @@ copy-dependencies - + ${project.build.directory} false false @@ -471,5 +483,4 @@ - - + \ No newline at end of file diff --git a/core/src/main/java/org/apache/spark/metrics/prometheus/client/exporter/PushGatewayWithTimestamp.java b/core/src/main/java/org/apache/spark/metrics/prometheus/client/exporter/PushGatewayWithTimestamp.java new file mode 100644 index 0000000000000..d04b69ea4a8b3 --- /dev/null +++ b/core/src/main/java/org/apache/spark/metrics/prometheus/client/exporter/PushGatewayWithTimestamp.java @@ -0,0 +1,296 @@ +/* + * Licensed 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.spark.metrics.prometheus.client.exporter; + +import io.prometheus.client.Collector; +import io.prometheus.client.CollectorRegistry; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.HttpURLConnection; +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.net.URL; +import java.net.URLEncoder; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import java.io.BufferedWriter; +import java.io.IOException; +import java.io.OutputStreamWriter; + +/** + * Export metrics via the Prometheus Pushgateway. + *

+ * The Prometheus Pushgateway exists to allow ephemeral and batch jobs to expose their metrics to Prometheus. + * Since these kinds of jobs may not exist long enough to be scraped, they can instead push their metrics + * to a Pushgateway. This class allows pushing the contents of a {@link CollectorRegistry} to + * a Pushgateway. + *

+ * Example usage: + *

+ * {@code
+ *   void executeBatchJob() throws Exception {
+ *     CollectorRegistry registry = new CollectorRegistry();
+ *     Gauge duration = Gauge.build()
+ *         .name("my_batch_job_duration_seconds").help("Duration of my batch job in seconds.").register(registry);
+ *     Gauge.Timer durationTimer = duration.startTimer();
+ *     try {
+ *       // Your code here.
+ *
+ *       // This is only added to the registry after success,
+ *       // so that a previous success in the Pushgateway isn't overwritten on failure.
+ *       Gauge lastSuccess = Gauge.build()
+ *           .name("my_batch_job_last_success").help("Last time my batch job succeeded, in unixtime.").register(registry);
+ *       lastSuccess.setToCurrentTime();
+ *     } finally {
+ *       durationTimer.setDuration();
+ *       PushGatewayWithTimestamp pg = new PushGatewayWithTimestamp("127.0.0.1:9091");
+ *       pg.pushAdd(registry, "my_batch_job");
+ *     }
+ *   }
+ * }
+ * 
+ *

+ * See https://github.com/prometheus/pushgateway + */ +public class PushGatewayWithTimestamp { + + private static final Logger logger = LoggerFactory.getLogger(PushGatewayWithTimestamp.class); + private final String address; + private static final int SECONDS_PER_MILLISECOND = 1000; + /** + * Construct a Pushgateway, with the given address. + *

+ * @param address host:port or ip:port of the Pushgateway. + */ + public PushGatewayWithTimestamp(String address) { + this.address = address; + } + + /** + * Pushes all metrics in a registry, replacing all those with the same job and no grouping key. + *

+ * This uses the PUT HTTP method. + */ + public void push(CollectorRegistry registry, String job) throws IOException { + doRequest(registry, job, null, "PUT", null); + } + + /** + * Pushes all metrics in a Collector, replacing all those with the same job and no grouping key. + *

+ * This is useful for pushing a single Gauge. + *

+ * This uses the PUT HTTP method. + */ + public void push(Collector collector, String job) throws IOException { + CollectorRegistry registry = new CollectorRegistry(); + collector.register(registry); + push(registry, job); + } + + /** + * Pushes all metrics in a registry, replacing all those with the same job and grouping key. + *

+ * This uses the PUT HTTP method. + */ + public void push(CollectorRegistry registry, String job, Map groupingKey) throws IOException { + doRequest(registry, job, groupingKey, "PUT", null); + } + + /** + * Pushes all metrics in a Collector, replacing all those with the same job and grouping key. + *

+ * This is useful for pushing a single Gauge. + *

+ * This uses the PUT HTTP method. + */ + public void push(Collector collector, String job, Map groupingKey) throws IOException { + CollectorRegistry registry = new CollectorRegistry(); + collector.register(registry); + push(registry, job, groupingKey); + } + + /** + * Pushes all metrics in a registry, replacing only previously pushed metrics of the same name and job and no grouping key. + *

+ * This uses the POST HTTP method. + */ + public void pushAdd(CollectorRegistry registry, String job, String timestamp) throws IOException { + doRequest(registry, job, null, "POST", timestamp); + } + + /** + * Pushes all metrics in a Collector, replacing only previously pushed metrics of the same name and job and no grouping key. + *

+ * This is useful for pushing a single Gauge. + *

+ * This uses the POST HTTP method. + */ + public void pushAdd(Collector collector, String job) throws IOException { + CollectorRegistry registry = new CollectorRegistry(); + collector.register(registry); + pushAdd(registry, job, ""); + } + + /** + * Pushes all metrics in a registry, replacing only previously pushed metrics of the same name, job and grouping key. + *

+ * This uses the POST HTTP method. + */ + public void pushAdd(CollectorRegistry registry, String job, Map groupingKey, String timestamp) throws IOException { + doRequest(registry, job, groupingKey, "POST", timestamp); + } + + /** + * Pushes all metrics in a Collector, replacing only previously pushed metrics of the same name, job and grouping key. + *

+ * This is useful for pushing a single Gauge. + *

+ * This uses the POST HTTP method. + */ + public void pushAdd(Collector collector, String job, Map groupingKey) throws IOException { + CollectorRegistry registry = new CollectorRegistry(); + collector.register(registry); + pushAdd(registry, job, groupingKey, null); + } + + + /** + * Deletes metrics from the Pushgateway. + *

+ * Deletes metrics with no grouping key and the provided job. + * This uses the DELETE HTTP method. + */ + public void delete(String job) throws IOException { + doRequest(null, job, null, "DELETE", null); + } + + /** + * Deletes metrics from the Pushgateway. + *

+ * Deletes metrics with the provided job and grouping key. + * This uses the DELETE HTTP method. + */ + public void delete(String job, Map groupingKey) throws IOException { + doRequest(null, job, groupingKey, "DELETE", null); + } + + + /** + * Pushes all metrics in a registry, replacing all those with the same job and instance. + *

+ * This uses the PUT HTTP method. + * @deprecated use {@link #push(CollectorRegistry, String, Map)} + */ + @Deprecated + public void push(CollectorRegistry registry, String job, String instance) throws IOException { + push(registry, job, Collections.singletonMap("instance", instance)); + } + + /** + * Pushes all metrics in a Collector, replacing all those with the same job and instance. + *

+ * This is useful for pushing a single Gauge. + *

+ * This uses the PUT HTTP method. + * @deprecated use {@link #push(Collector, String, Map)} + */ + @Deprecated + public void push(Collector collector, String job, String instance) throws IOException { + push(collector, job, Collections.singletonMap("instance", instance)); + } + + /** + * Pushes all metrics in a Collector, replacing only previously pushed metrics of the same name. + *

+ * This is useful for pushing a single Gauge. + *

+ * This uses the POST HTTP method. + * @deprecated use {@link #pushAdd(Collector, String, Map)} + */ + @Deprecated + public void pushAdd(Collector collector, String job, String instance) throws IOException { + pushAdd(collector, job, Collections.singletonMap("instance", instance)); + } + + /** + * Deletes metrics from the Pushgateway. + *

+ * This uses the DELETE HTTP method. + * @deprecated use {@link #delete(String, Map)} + */ + @Deprecated + public void delete(String job, String instance) throws IOException { + delete(job, Collections.singletonMap("instance", instance)); + } + + void doRequest(CollectorRegistry registry, String job, Map groupingKey, String method, String timestamp) throws IOException { + String url = "http://" + address + "/metrics/job/" + URLEncoder.encode(job, "UTF-8"); + if (groupingKey != null) { + for (Map.Entry entry: groupingKey.entrySet()) { + url += "/" + entry.getKey() + "/" + URLEncoder.encode(entry.getValue(), "UTF-8"); + } + } + + logger.info("Sending metrics data to '{}'", url); + + HttpURLConnection connection = (HttpURLConnection) new URL(url).openConnection(); + connection.setRequestProperty("Content-Type", TextFormatWithTimestamp.CONTENT_TYPE_004); + if (!method.equals("DELETE")) { + connection.setDoOutput(true); + } + connection.setRequestMethod(method); + + connection.setConnectTimeout(10 * SECONDS_PER_MILLISECOND); + connection.setReadTimeout(10 * SECONDS_PER_MILLISECOND); + connection.connect(); + + try { + if (!method.equals("DELETE")) { + BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(connection.getOutputStream(), "UTF-8")); + TextFormatWithTimestamp.write004(writer, registry.metricFamilySamples(), timestamp); + writer.flush(); + writer.close(); + } + + int response = connection.getResponseCode(); + if (response != HttpURLConnection.HTTP_ACCEPTED) { + throw new IOException("Response code from " + url + " was " + response); + } + } catch (Exception ex) { + logger.error("Sending metrics failed due to: ", ex); + } + + finally { + connection.disconnect(); + } + } + + /** + * Returns a grouping key with the instance label set to the machine's IP address. + *

+ * This is a convenience function, and should only be used where you want to + * push per-instance metrics rather than cluster/job level metrics. + */ + public static Map instanceIPGroupingKey() throws UnknownHostException { + Map groupingKey = new HashMap(); + groupingKey.put("instance", InetAddress.getLocalHost().getHostAddress()); + return groupingKey; + } + +} + diff --git a/core/src/main/java/org/apache/spark/metrics/prometheus/client/exporter/TextFormatWithTimestamp.java b/core/src/main/java/org/apache/spark/metrics/prometheus/client/exporter/TextFormatWithTimestamp.java new file mode 100644 index 0000000000000..5f3548bfe8481 --- /dev/null +++ b/core/src/main/java/org/apache/spark/metrics/prometheus/client/exporter/TextFormatWithTimestamp.java @@ -0,0 +1,174 @@ +/* + * Licensed 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.spark.metrics.prometheus.client.exporter; + +import java.io.IOException; +import java.io.Writer; +import java.util.Enumeration; + +import io.prometheus.client.Collector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TextFormatWithTimestamp { + private static final Logger logger = LoggerFactory.getLogger(TextFormatWithTimestamp.class); + + /** + * Content-type for text version 0.0.4. + */ + public final static String CONTENT_TYPE_004 = "text/plain; version=0.0.4; charset=utf-8"; + + private static StringBuilder jsonMessageLogBuilder = new StringBuilder(); + + public static void write004(Writer writer, Enumeration mfs)throws IOException { + write004(writer, mfs, null); + } + + /** + * Write out the text version 0.0.4 of the given MetricFamilySamples. + */ + public static void write004(Writer writer, Enumeration mfs, String timestamp) throws IOException { + /* See http://prometheus.io/docs/instrumenting/exposition_formats/ + * for the output format specification. */ + while(mfs.hasMoreElements()) { + Collector.MetricFamilySamples metricFamilySamples = mfs.nextElement(); + + logger.debug("Metrics data"); + logger.debug(metricFamilySamples.toString()); + logger.debug("Logging metrics as a json format:"); + + + writer.write("# HELP "); + appendToJsonMessageLogBuilder("# HELP "); + writer.write(metricFamilySamples.name); + appendToJsonMessageLogBuilder(metricFamilySamples.name); + writer.write(' '); + appendToJsonMessageLogBuilder(' '); + writeEscapedHelp(writer, metricFamilySamples.help); + writer.write('\n'); + appendToJsonMessageLogBuilder('\n'); + + writer.write("# TYPE "); + appendToJsonMessageLogBuilder("# TYPE "); + writer.write(metricFamilySamples.name); + appendToJsonMessageLogBuilder(metricFamilySamples.name); + writer.write(' '); + appendToJsonMessageLogBuilder(' '); + writer.write(typeString(metricFamilySamples.type)); + appendToJsonMessageLogBuilder(typeString(metricFamilySamples.type)); + writer.write('\n'); + appendToJsonMessageLogBuilder('\n'); + + for (Collector.MetricFamilySamples.Sample sample: metricFamilySamples.samples) { + writer.write(sample.name); + appendToJsonMessageLogBuilder(sample.name); + if (sample.labelNames.size() > 0) { + writer.write('{'); + appendToJsonMessageLogBuilder('{'); + for (int i = 0; i < sample.labelNames.size(); ++i) { + writer.write(sample.labelNames.get(i)); + appendToJsonMessageLogBuilder(sample.labelNames.get(i)); + writer.write("=\""); + appendToJsonMessageLogBuilder("=\""); + writeEscapedLabelValue(writer, sample.labelValues.get(i)); + writer.write("\","); + appendToJsonMessageLogBuilder("\","); + } + writer.write('}'); + appendToJsonMessageLogBuilder('}'); + } + writer.write(' '); + appendToJsonMessageLogBuilder(' '); + writer.write(Collector.doubleToGoString(sample.value)); + appendToJsonMessageLogBuilder(Collector.doubleToGoString(sample.value)); + if(timestamp != null && !timestamp.isEmpty()) { + writer.write(" " + timestamp); + appendToJsonMessageLogBuilder(" " + timestamp); + } + writer.write('\n'); + appendToJsonMessageLogBuilder('\n'); + } + logger.debug("JSON: "+ jsonMessageLogBuilder); + } + } + + private static void writeEscapedHelp(Writer writer, String s) throws IOException { + for (int i = 0; i < s.length(); i++) { + char c = s.charAt(i); + switch (c) { + case '\\': + + writer.append("\\\\"); + appendToJsonMessageLogBuilder("\\\\"); + break; + case '\n': + writer.append("\\n"); + appendToJsonMessageLogBuilder("\\n"); + break; + default: + writer.append(c); + appendToJsonMessageLogBuilder(c); + } + } + } + + private static void writeEscapedLabelValue(Writer writer, String s) throws IOException { + for (int i = 0; i < s.length(); i++) { + char c = s.charAt(i); + switch (c) { + case '\\': + writer.append("\\\\"); + appendToJsonMessageLogBuilder("\\\\"); + break; + case '\"': + writer.append("\\\""); + appendToJsonMessageLogBuilder("\\\""); + break; + case '\n': + writer.append("\\n"); + appendToJsonMessageLogBuilder("\\n"); + break; + default: + writer.append(c); + appendToJsonMessageLogBuilder(c); + } + } + } + + private static String typeString(Collector.Type t) { + switch (t) { + case GAUGE: + return "gauge"; + case COUNTER: + return "counter"; + case SUMMARY: + return "summary"; + case HISTOGRAM: + return "histogram"; + default: + return "untyped"; + } + } + + private static void appendToJsonMessageLogBuilder(String msg) { + if (logger.isDebugEnabled()) + jsonMessageLogBuilder.append(msg); + } + + private static void appendToJsonMessageLogBuilder(char c) { + if (logger.isDebugEnabled()) + jsonMessageLogBuilder.append(c); + } +} + diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/PrometheusSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/PrometheusSink.scala new file mode 100644 index 0000000000000..5a3cf27308bd4 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/metrics/sink/PrometheusSink.scala @@ -0,0 +1,157 @@ +/* + * 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.spark.metrics.sink + +import java.net.URI +import java.util +import java.util.Properties +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConverters._ +import scala.util.Try + +import com.codahale.metrics._ +import io.prometheus.client.CollectorRegistry +import io.prometheus.client.dropwizard.DropwizardExports + +import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.METRICS_NAMESPACE +import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.metrics.prometheus.client.exporter.PushGatewayWithTimestamp + + +private[spark] class PrometheusSink( + val property: Properties, + val registry: MetricRegistry, + securityMgr: SecurityManager) + extends Sink with Logging { + + protected class Reporter(registry: MetricRegistry) + extends ScheduledReporter( + registry, + "prometheus-reporter", + MetricFilter.ALL, + TimeUnit.SECONDS, + TimeUnit.MILLISECONDS) { + val sparkConf: SparkConf = new SparkConf + + private[this] lazy val metricsNamespace: Option[String] = + sparkConf.get(METRICS_NAMESPACE) + .orElse(Option(sparkConf.getenv("METRICS_NAMESPACE"))) + + private[this] lazy val sparkAppId: Option[String] = + sparkConf.getOption("spark.app.id") + .orElse(Option(sparkConf.getenv("SPARK_APPLICATION_ID"))) + + private[this] lazy val executorId: Option[String] = + Option(sparkConf.getenv("SPARK_EXECUTOR_ID")) + .orElse(sparkConf.getOption("spark.executor.id")) + + + + override def report( + gauges: util.SortedMap[String, Gauge[_]], + counters: util.SortedMap[String, Counter], + histograms: util.SortedMap[String, Histogram], + meters: util.SortedMap[String, Meter], + timers: util.SortedMap[String, Timer]): Unit = { + + logInfo(s"metricsNamespace=$metricsNamespace, sparkAppId=$sparkAppId, " + + s"executorId=$executorId") + + val role: String = (sparkAppId, executorId) match { + case (Some(_), None) => "driver" + case (Some(_), Some(_)) => "executor" + case _ => "shuffle" + } + + val job: String = role match { + case "driver" => metricsNamespace.getOrElse(sparkAppId.get) + case "executor" => metricsNamespace.getOrElse(sparkAppId.get) + case _ => metricsNamespace.getOrElse("shuffle") + } + logInfo(s"role=$role, job=$job") + + val groupingKey: Map[String, String] = (role, executorId) match { + case ("driver", _) => Map("role" -> role) + case ("executor", Some(id)) => Map ("role" -> role, "number" -> id) + case _ => Map("role" -> role) + } + + + pushGateway.pushAdd(pushRegistry, job, groupingKey.asJava, + s"${System.currentTimeMillis}") + + } + + } + + val DEFAULT_PUSH_PERIOD: Int = 10 + val DEFAULT_PUSH_PERIOD_UNIT: TimeUnit = TimeUnit.SECONDS + val DEFAULT_PUSHGATEWAY_ADDRESS: String = "127.0.0.1:9091" + + val KEY_PUSH_PERIOD = "period" + val KEY_PUSH_PERIOD_UNIT = "unit" + val KEY_PUSHGATEWAY_ADDRESS = "pushgateway-address" + + + val pollPeriod: Int = + Option(property.getProperty(KEY_PUSH_PERIOD)) + .map(_.toInt) + .getOrElse(DEFAULT_PUSH_PERIOD) + + val pollUnit: TimeUnit = + Option(property.getProperty(KEY_PUSH_PERIOD_UNIT)) + .map { s => TimeUnit.valueOf(s.toUpperCase) } + .getOrElse(DEFAULT_PUSH_PERIOD_UNIT) + + val pushGatewayAddress = + Option(property.getProperty(KEY_PUSHGATEWAY_ADDRESS)) + .getOrElse(DEFAULT_PUSHGATEWAY_ADDRESS) + + // validate pushgateway host:port + Try(new URI(s"http://$pushGatewayAddress")).get + + MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) + + logInfo("Initializing Prometheus Sink...") + logInfo(s"Metrics polling period -> $pollPeriod $pollUnit") + logInfo(s"$KEY_PUSHGATEWAY_ADDRESS -> $pushGatewayAddress") + + val pushRegistry: CollectorRegistry = new CollectorRegistry() + val sparkMetricExports: DropwizardExports = new DropwizardExports(registry) + val pushGateway: PushGatewayWithTimestamp = new PushGatewayWithTimestamp(pushGatewayAddress) + + val reporter = new Reporter(registry) + + override def start(): Unit = { + sparkMetricExports.register(pushRegistry) + + reporter.start(pollPeriod, pollUnit) + } + + override def stop(): Unit = { + reporter.stop() + pushRegistry.unregister(sparkMetricExports) + } + + override def report(): Unit = { + reporter.report() + } +} diff --git a/pom.xml b/pom.xml index e8e9d736da9bd..5935490da1feb 100644 --- a/pom.xml +++ b/pom.xml @@ -141,6 +141,7 @@ 2.4.0 2.0.8 3.1.2 + 0.0.23 1.7.7 hadoop2 0.9.3 @@ -646,6 +647,21 @@ metrics-graphite ${codahale.metrics.version} + + io.prometheus + simpleclient + ${prometheus.version} + + + io.prometheus + simpleclient_dropwizard + ${prometheus.version} + + + io.prometheus + simpleclient_pushgateway + ${prometheus.version} + com.fasterxml.jackson.core jackson-databind @@ -1396,9 +1412,9 @@ ${hive.group} hive-exec - + ${hive.version} ${hive.deps.scope} @@ -2282,7 +2298,7 @@ default-cli - build-classpath + build-classpath