diff --git a/docs/configuration/settings.md b/docs/configuration/settings.md
index acbc36197fa..bb81d8de534 100644
--- a/docs/configuration/settings.md
+++ b/docs/configuration/settings.md
@@ -399,16 +399,17 @@ You can configure the Kyuubi properties in `$KYUUBI_HOME/conf/kyuubi-defaults.co
### Metrics
-| Key | Default | Meaning | Type | Since |
-|---------------------------------|------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|----------|-------|
-| kyuubi.metrics.console.interval | PT5S | How often should report metrics to console | duration | 1.2.0 |
-| kyuubi.metrics.enabled | true | Set to true to enable kyuubi metrics system | boolean | 1.2.0 |
-| kyuubi.metrics.json.interval | PT5S | How often should report metrics to JSON file | duration | 1.2.0 |
-| kyuubi.metrics.json.location | metrics | Where the JSON metrics file located | string | 1.2.0 |
-| kyuubi.metrics.prometheus.path | /metrics | URI context path of prometheus metrics HTTP server | string | 1.2.0 |
-| kyuubi.metrics.prometheus.port | 10019 | Prometheus metrics HTTP server port | int | 1.2.0 |
-| kyuubi.metrics.reporters | PROMETHEUS | A comma-separated list for all metrics reporters
- CONSOLE - ConsoleReporter which outputs measurements to CONSOLE periodically.
- JMX - JmxReporter which listens for new metrics and exposes them as MBeans.
- JSON - JsonReporter which outputs measurements to json file periodically.
- PROMETHEUS - PrometheusReporter which exposes metrics in Prometheus format.
- SLF4J - Slf4jReporter which outputs measurements to system log periodically.
| set | 1.2.0 |
-| kyuubi.metrics.slf4j.interval | PT5S | How often should report metrics to SLF4J logger | duration | 1.2.0 |
+| Key | Default | Meaning | Type | Since |
+|---------------------------------------------------|------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|----------|--------|
+| kyuubi.metrics.console.interval | PT5S | How often should report metrics to console | duration | 1.2.0 |
+| kyuubi.metrics.enabled | true | Set to true to enable kyuubi metrics system | boolean | 1.2.0 |
+| kyuubi.metrics.json.interval | PT5S | How often should report metrics to JSON file | duration | 1.2.0 |
+| kyuubi.metrics.json.location | metrics | Where the JSON metrics file located | string | 1.2.0 |
+| kyuubi.metrics.prometheus.labels.instance.enabled | false | Whether to add instance label to prometheus metrics | boolean | 1.10.2 |
+| kyuubi.metrics.prometheus.path | /metrics | URI context path of prometheus metrics HTTP server | string | 1.2.0 |
+| kyuubi.metrics.prometheus.port | 10019 | Prometheus metrics HTTP server port | int | 1.2.0 |
+| kyuubi.metrics.reporters | PROMETHEUS | A comma-separated list for all metrics reporters - CONSOLE - ConsoleReporter which outputs measurements to CONSOLE periodically.
- JMX - JmxReporter which listens for new metrics and exposes them as MBeans.
- JSON - JsonReporter which outputs measurements to json file periodically.
- PROMETHEUS - PrometheusReporter which exposes metrics in Prometheus format.
- SLF4J - Slf4jReporter which outputs measurements to system log periodically.
| set | 1.2.0 |
+| kyuubi.metrics.slf4j.interval | PT5S | How often should report metrics to SLF4J logger | duration | 1.2.0 |
### Operation
diff --git a/kyuubi-metrics/src/main/scala/org/apache/kyuubi/metrics/MetricsConf.scala b/kyuubi-metrics/src/main/scala/org/apache/kyuubi/metrics/MetricsConf.scala
index 9bc2e63243a..887f6e72130 100644
--- a/kyuubi-metrics/src/main/scala/org/apache/kyuubi/metrics/MetricsConf.scala
+++ b/kyuubi-metrics/src/main/scala/org/apache/kyuubi/metrics/MetricsConf.scala
@@ -80,6 +80,13 @@ object MetricsConf {
.checkValue(path => path.startsWith("/"), "Context path must start with '/'")
.createWithDefault("/metrics")
+ val METRICS_PROMETHEUS_LABELS_INSTANCE_ENABLED: ConfigEntry[Boolean] =
+ buildConf("kyuubi.metrics.prometheus.labels.instance.enabled")
+ .doc("Whether to add instance label to prometheus metrics")
+ .version("1.10.2")
+ .booleanConf
+ .createWithDefault(false)
+
val METRICS_SLF4J_INTERVAL: ConfigEntry[Long] = buildConf("kyuubi.metrics.slf4j.interval")
.doc("How often should report metrics to SLF4J logger")
.version("1.2.0")
diff --git a/kyuubi-metrics/src/main/scala/org/apache/kyuubi/metrics/PrometheusReporterService.scala b/kyuubi-metrics/src/main/scala/org/apache/kyuubi/metrics/PrometheusReporterService.scala
index e62e2190906..0008c2b9ca8 100644
--- a/kyuubi-metrics/src/main/scala/org/apache/kyuubi/metrics/PrometheusReporterService.scala
+++ b/kyuubi-metrics/src/main/scala/org/apache/kyuubi/metrics/PrometheusReporterService.scala
@@ -17,10 +17,13 @@
package org.apache.kyuubi.metrics
+import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse}
+
import com.codahale.metrics.MetricRegistry
import io.prometheus.client.CollectorRegistry
import io.prometheus.client.dropwizard.DropwizardExports
import io.prometheus.client.exporter.MetricsServlet
+import io.prometheus.client.exporter.common.TextFormat
import org.eclipse.jetty.server.{HttpConfiguration, HttpConnectionFactory, Server, ServerConnector}
import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder}
@@ -28,6 +31,7 @@ import org.apache.kyuubi.KyuubiException
import org.apache.kyuubi.config.KyuubiConf
import org.apache.kyuubi.config.KyuubiConf.FRONTEND_JETTY_SEND_VERSION_ENABLED
import org.apache.kyuubi.service.AbstractService
+import org.apache.kyuubi.util.JavaUtils
class PrometheusReporterService(registry: MetricRegistry)
extends AbstractService("PrometheusReporterService") {
@@ -56,8 +60,16 @@ class PrometheusReporterService(registry: MetricRegistry)
httpServer.setHandler(context)
new DropwizardExports(registry).register(bridgeRegistry)
- val metricsServlet = new MetricsServlet(bridgeRegistry)
- context.addServlet(new ServletHolder(metricsServlet), contextPath)
+ if (conf.get(MetricsConf.METRICS_PROMETHEUS_LABELS_INSTANCE_ENABLED)) {
+ val instanceLabel =
+ Map("instance" -> s"${JavaUtils.findLocalInetAddress.getCanonicalHostName}:$port")
+ context.addServlet(
+ new ServletHolder(createPrometheusServletWithLabels(instanceLabel)),
+ contextPath)
+ } else {
+ val metricsServlet = new MetricsServlet(bridgeRegistry)
+ context.addServlet(new ServletHolder(metricsServlet), contextPath)
+ }
super.initialize(conf)
}
@@ -100,4 +112,48 @@ class PrometheusReporterService(registry: MetricRegistry)
}
}
}
+
+ private def createPrometheusServletWithLabels(labels: Map[String, String]): HttpServlet = {
+ new HttpServlet {
+ override def doGet(request: HttpServletRequest, response: HttpServletResponse): Unit = {
+ try {
+ response.setContentType("text/plain;charset=utf-8")
+ response.setStatus(HttpServletResponse.SC_OK)
+ response.getWriter.print(getMetricsSnapshot(labels))
+ } catch {
+ case e: IllegalArgumentException =>
+ response.sendError(HttpServletResponse.SC_BAD_REQUEST, e.getMessage)
+ case e: Exception =>
+ warn(s"GET ${request.getRequestURI} failed: $e", e)
+ throw e
+ }
+ }
+
+ // ensure TRACE is not supported
+ override protected def doTrace(req: HttpServletRequest, res: HttpServletResponse): Unit = {
+ res.sendError(HttpServletResponse.SC_METHOD_NOT_ALLOWED)
+ }
+ }
+ }
+
+ private def getMetricsSnapshot(labels: Map[String, String]): String = {
+ val metricsSnapshotWriter = new java.io.StringWriter
+ val contentType = TextFormat.chooseContentType(null)
+ TextFormat.writeFormat(contentType, metricsSnapshotWriter, bridgeRegistry.metricFamilySamples())
+ val labelStr = labelString(labels)
+ metricsSnapshotWriter.toString.split("\n").map { line =>
+ if (line.startsWith("#")) {
+ line
+ } else {
+ line.split("\\s+", 2) match {
+ case Array(metrics, rest) => s"""$metrics${labelStr} $rest"""
+ case _ => line
+ }
+ }
+ }.mkString("\n")
+ }
+
+ private def labelString(labels: Map[String, String]): String = {
+ labels.map { case (k, v) => s"""$k="$v"""" }.toArray.sorted.mkString("{", ",", "}")
+ }
}