Browse Source

HDDS-846. Exports ozone metrics to prometheus.
Contributed by Elek, Marton.

Anu Engineer 6 years ago
parent
commit
34a914be03

+ 3 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java

@@ -104,4 +104,7 @@ public final class HddsConfigKeys {
   // In any real cluster where performance matters, the SCM, OM and DN
   // metadata locations must be configured explicitly.
   public static final String OZONE_METADATA_DIRS = "ozone.metadata.dirs";
+
+  public static final String HDDS_PROMETHEUS_ENABLED =
+      "hdds.prometheus.endpoint.enabled";
 }

+ 11 - 0
hadoop-hdds/common/src/main/resources/ozone-default.xml

@@ -270,6 +270,17 @@
       datanode periodically send pipeline report to SCM. Unit could be
       defined with postfix (ns,ms,s,m,h,d)</description>
   </property>
+
+
+  <property>
+    <name>hdds.prometheus.endpoint.enabled</name>
+    <value>false</value>
+    <tag>OZONE, MANAGEMENT</tag>
+    <description>Enable prometheus compatible metric page on the HTTP
+      servers.
+    </description>
+  </property>
+
   <!--Ozone Settings-->
   <property>
     <name>ozone.administrators</name>

+ 5 - 0
hadoop-hdds/docs/config.yaml

@@ -39,3 +39,8 @@ menu:
     title: Tools
     url: dozone.html
     weight: 3
+  - identifier: Recipes
+    name: Recipes
+    title: Recipes
+    url: prometheus.html
+    weight: 4

+ 94 - 0
hadoop-hdds/docs/content/Prometheus.md

@@ -0,0 +1,94 @@
+---
+title: Monitoring with Prometheus
+menu:
+   main:
+      parent: Recipes
+---
+<!---
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+[Prometheus](https://prometheus.io/) is an open-source monitoring server developed under under the [Cloud Native Foundation](Cloud Native Foundation).
+
+Ozone supports Prometheus out of the box. The servers start a prometheus 
+compatible metrics endpoint where all the available hadoop metrics are published in prometheus exporter format.
+
+## Prerequisites
+
+ 1. [Install the and start]({{< ref "RunningViaDocker.md" >}}) an Ozone cluster.
+ 2. [Download](https://prometheus.io/download/#prometheus) the prometheus binary.
+
+## Monitoring with prometheus
+
+(1) To enable the Prometheus metrics endpoint you need to add a new configuration to the `ozone-site.xml` file:
+
+```
+  <property>
+    <name>hdds.prometheus.endpoint.enabled</name>
+    <value>true</value>
+  </property>
+```
+
+_Note_: for Docker compose based pseudo cluster put the `OZONE-SITE.XML_hdds.prometheus.endpoint.enabled=true` line to the `docker-config` file.
+
+(2) Restart the Ozone Manager and Storage Container Manager and check the prometheus endpoints:
+
+ * http://scm:9874/prom
+
+ * http://ozoneManager:9876/prom
+
+(3) Create a prometheus.yaml configuration with the previous endpoints:
+
+```yaml
+global:
+  scrape_interval:     15s
+
+scrape_configs:
+  - job_name: ozone
+    metrics_path: /prom
+    static_configs:
+     - targets:
+        - "scm:9876"
+        - "ozoneManager:9874"
+```
+
+(4) Start with prometheus from the directory where you have the prometheus.yaml file:
+
+```
+prometheus
+```
+
+(5) Check the active targets in the prometheus web-ui:
+
+http://localhost:9090/targets
+
+![Prometheus target page example](../../prometheus.png)
+
+
+(6) Check any metrics on the prometheus web ui. For example:
+
+http://localhost:9090/graph?g0.range_input=1h&g0.expr=om_metrics_num_key_allocate&g0.tab=1
+
+![Prometheus target page example](../../prometheus-key-allocate.png)
+
+## Note
+
+The ozone distribution contains a ready-to-use, dockerized environment to try out ozone and prometheus. It can be found under `compose/ozoneperf` directory.
+
+```bash
+cd compose/ozoneperf
+docker-compose up -d
+```

+ 20 - 0
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/BaseHttpServer.java

@@ -18,11 +18,13 @@
 package org.apache.hadoop.hdds.server;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdds.conf.HddsConfServlet;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.http.HttpServer2;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.net.NetUtils;
 import org.eclipse.jetty.webapp.WebAppContext;
 import org.slf4j.Logger;
@@ -43,6 +45,7 @@ public abstract class BaseHttpServer {
 
   private static final Logger LOG =
       LoggerFactory.getLogger(BaseHttpServer.class);
+  protected static final String PROMETHEUS_SINK = "PROMETHEUS_SINK";
 
   private HttpServer2 httpServer;
   private final Configuration conf;
@@ -53,6 +56,9 @@ public abstract class BaseHttpServer {
   private HttpConfig.Policy policy;
 
   private String name;
+  private PrometheusMetricsSink prometheusMetricsSink;
+
+  private boolean prometheusSupport;
 
   public BaseHttpServer(Configuration conf, String name) throws IOException {
     this.name = name;
@@ -82,6 +88,15 @@ public abstract class BaseHttpServer {
       httpServer = builder.build();
       httpServer.addServlet("conf", "/conf", HddsConfServlet.class);
 
+      prometheusSupport =
+          conf.getBoolean(HddsConfigKeys.HDDS_PROMETHEUS_ENABLED, false);
+
+      if (prometheusSupport) {
+        prometheusMetricsSink = new PrometheusMetricsSink();
+        httpServer.getWebAppContext().getServletContext()
+            .setAttribute(PROMETHEUS_SINK, prometheusMetricsSink);
+        httpServer.addServlet("prometheus", "/prom", PrometheusServlet.class);
+      }
     }
 
   }
@@ -150,6 +165,11 @@ public abstract class BaseHttpServer {
   public void start() throws IOException {
     if (httpServer != null && isEnabled()) {
       httpServer.start();
+      if (prometheusSupport) {
+        DefaultMetricsSystem.instance()
+            .register("prometheus", "Hadoop metrics prometheus exporter",
+                prometheusMetricsSink);
+      }
       updateConnectorAddress();
     }
 

+ 133 - 0
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/PrometheusMetricsSink.java

@@ -0,0 +1,133 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.hdds.server;
+
+import java.io.IOException;
+import java.io.Writer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricType;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.MetricsSink;
+import org.apache.hadoop.metrics2.MetricsTag;
+
+import org.apache.commons.configuration2.SubsetConfiguration;
+
+/**
+ * Metrics sink for prometheus exporter.
+ * <p>
+ * Stores the metric data in-memory and return with it on request.
+ */
+public class PrometheusMetricsSink implements MetricsSink {
+
+  /**
+   * Cached output lines for each metrics.
+   */
+  private Map<String, String> metricLines = new HashMap<>();
+
+  private static final Pattern UPPER_CASE_SEQ =
+      Pattern.compile("([A-Z]*)([A-Z])");
+
+  public PrometheusMetricsSink() {
+  }
+
+  @Override
+  public void putMetrics(MetricsRecord metricsRecord) {
+    for (AbstractMetric metrics : metricsRecord.metrics()) {
+      if (metrics.type() == MetricType.COUNTER
+          || metrics.type() == MetricType.GAUGE) {
+
+        String key = prometheusName(
+            metricsRecord.name(), metrics.name());
+
+        StringBuilder builder = new StringBuilder();
+        builder.append("# TYPE " + key + " " +
+            metrics.type().toString().toLowerCase() + "\n");
+        builder.append(key + "{");
+        String sep = "";
+
+        //add tags
+        for (MetricsTag tag : metricsRecord.tags()) {
+          String tagName = tag.name().toLowerCase();
+
+          //ignore specific tag which includes sub-hierarchy
+          if (!tagName.equals("numopenconnectionsperuser")) {
+            builder.append(
+                sep + tagName + "=\"" + tag.value() + "\"");
+            sep = ",";
+          }
+        }
+        builder.append("} ");
+        builder.append(metrics.value());
+        metricLines.put(key, builder.toString());
+
+      }
+    }
+  }
+
+  /**
+   * Convert CamelCase based namess to lower-case names where the separator
+   * is the underscore, to follow prometheus naming conventions.
+   */
+  public String prometheusName(String recordName,
+      String metricName) {
+    String baseName = upperFirst(recordName) + upperFirst(metricName);
+    Matcher m = UPPER_CASE_SEQ.matcher(baseName);
+    StringBuffer sb = new StringBuffer();
+    while (m.find()) {
+      String replacement = "_" + m.group(2).toLowerCase();
+      if (m.group(1).length() > 0) {
+        replacement = "_" + m.group(1).toLowerCase() + replacement;
+      }
+      m.appendReplacement(sb, replacement);
+    }
+    m.appendTail(sb);
+
+    //always prefixed with "_"
+    return sb.toString().substring(1);
+  }
+
+  private String upperFirst(String name) {
+    if (Character.isLowerCase(name.charAt(0))) {
+      return Character.toUpperCase(name.charAt(0)) + name.substring(1);
+    } else {
+      return name;
+    }
+
+  }
+
+  @Override
+  public void flush() {
+
+  }
+
+  @Override
+  public void init(SubsetConfiguration subsetConfiguration) {
+
+  }
+
+  public void writeMetrics(Writer writer) throws IOException {
+    for (String line : metricLines.values()) {
+      writer.write(line + "\n");
+    }
+  }
+}

+ 46 - 0
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/PrometheusServlet.java

@@ -0,0 +1,46 @@
+/**
+ * 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.hadoop.hdds.server;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+
+/**
+ * Servlet to publish hadoop metrics in prometheus format.
+ */
+public class PrometheusServlet extends HttpServlet {
+
+  public PrometheusMetricsSink getPrometheusSink() {
+    return
+        (PrometheusMetricsSink) getServletContext().getAttribute(
+            BaseHttpServer.PROMETHEUS_SINK);
+  }
+
+  @Override
+  protected void doGet(HttpServletRequest req, HttpServletResponse resp)
+      throws ServletException, IOException {
+    DefaultMetricsSystem.instance().publishMetricsNow();
+    getPrometheusSink().writeMetrics(resp.getWriter());
+    resp.getWriter().flush();
+  }
+}

+ 96 - 0
hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/server/TestPrometheusMetricsSink.java

@@ -0,0 +1,96 @@
+/**
+ * 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.hadoop.hdds.server;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.commons.codec.CharEncoding.UTF_8;
+
+/**
+ * Test prometheus Sink.
+ */
+public class TestPrometheusMetricsSink {
+
+  @Test
+  public void testPublish() throws IOException {
+    //GIVEN
+    MetricsSystem metrics = DefaultMetricsSystem.instance();
+
+    metrics.init("test");
+    PrometheusMetricsSink sink = new PrometheusMetricsSink();
+    metrics.register("Prometheus", "Prometheus", sink);
+    TestMetrics testMetrics = metrics
+        .register("TestMetrics", "Testing metrics", new TestMetrics());
+
+    metrics.start();
+    testMetrics.numBucketCreateFails.incr();
+    metrics.publishMetricsNow();
+    ByteArrayOutputStream stream = new ByteArrayOutputStream();
+    OutputStreamWriter writer = new OutputStreamWriter(stream, UTF_8);
+
+    //WHEN
+    sink.writeMetrics(writer);
+    writer.flush();
+
+    //THEN
+    System.out.println(stream.toString(UTF_8));
+    Assert.assertTrue(
+        "The expected metric line is missing from prometheus metrics output",
+        stream.toString(UTF_8).contains(
+            "test_metrics_num_bucket_create_fails{context=\"dfs\"")
+    );
+
+    metrics.stop();
+    metrics.shutdown();
+  }
+
+  @Test
+  public void testNaming() throws IOException {
+    PrometheusMetricsSink sink = new PrometheusMetricsSink();
+
+    Assert.assertEquals("rpc_time_some_metrics",
+        sink.prometheusName("RpcTime", "SomeMetrics"));
+
+    Assert.assertEquals("om_rpc_time_om_info_keys",
+        sink.prometheusName("OMRpcTime", "OMInfoKeys"));
+
+    Assert.assertEquals("rpc_time_small",
+        sink.prometheusName("RpcTime", "small"));
+  }
+
+  /**
+   * Example metric pojo.
+   */
+  @Metrics(about = "Test Metrics", context = "dfs")
+  public static class TestMetrics {
+
+    @Metric
+    private MutableCounterLong numBucketCreateFails;
+  }
+}

+ 10 - 39
hadoop-ozone/dist/src/main/compose/ozoneperf/README.md

@@ -17,57 +17,28 @@
 This directory contains docker-compose definition for an ozone cluster where
 all the metrics are saved to a prometheus instance.
 
- Prometheus follows a pull based approach where the metrics are published
+Prometheus follows a pull based approach where the metrics are published
  on a HTTP endpoint.
 
- Our current approach:
+Prometheus compatible metrics endpoint can be enabled by setting `hdds.prometheus.endpoint.enabled` property to `true`
 
-  1. A Java agent activates a prometheus metrics endpoint in every JVM instance
-   (use `init.sh` to download the agent)
+## How to start
 
-  2. The Java agent publishes all the jmx parameters in prometheus format AND
-  register the endpoint address to the consul.
-
-  3. Prometheus polls all the endpoints which are registered to consul.
-
-
-
-## How to use
-
-First of all download the required Java agent with running `./init.sh`
-
-After that you can start the cluster with docker-compose:
+Start the cluster with `docker-compose`
 
 ```
 docker-compose up -d
 ```
 
-After a while the cluster will be started. You can check the ozone web ui-s:
-
-https://localhost:9874
-https://localhost:9876
-
-You can also scale up the datanodes:
+Note: The freon test will be started after 30 seconds.
 
-```
-docker-compose scale datanode=3
-```
+## How to use
 
-Freon (Ozone test generator tool) is not part of docker-compose by default,
-you can activate it using `compose-all.sh` instead of `docker-compose`:
+You can check the ozone web ui-s:
 
-```
-compose-all.sh up -d
-```
+https://localhost:9874
+https://localhost:9876
 
-Now Freon is running. Let's try to check the metrics from the local Prometheus:
+You can check the ozone metrics from the prometheus we ui.
 
 http://localhost:9090/graph
-
-Example queries:
-
-```
-Hadoop_OzoneManager_NumKeyCommits
-rate(Hadoop_OzoneManager_NumKeyCommits[10m])
-rate(Hadoop_Ozone_BYTES_WRITTEN[10m])
-```

+ 0 - 18
hadoop-ozone/dist/src/main/compose/ozoneperf/compose-all.sh

@@ -1,18 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-
-docker-compose -f docker-compose.yaml -f docker-compose-freon.yaml "$@"

+ 0 - 26
hadoop-ozone/dist/src/main/compose/ozoneperf/docker-compose-freon.yaml

@@ -1,26 +0,0 @@
-# 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.
-
-version: "3"
-services:
-   freon:
-      image: apache/hadoop-runner
-      volumes:
-         - ../../ozone:/opt/hadoop
-         - ./jmxpromo.jar:/opt/jmxpromo.jar
-      env_file:
-         - ./docker-config
-      command: ["/opt/hadoop/bin/ozone","freon"]

+ 12 - 11
hadoop-ozone/dist/src/main/compose/ozoneperf/docker-compose.yaml

@@ -20,41 +20,33 @@ services:
       image: apache/hadoop-runner
       volumes:
         - ../..:/opt/hadoop
-        - ./jmxpromo.jar:/opt/jmxpromo.jar
       ports:
         - 9864
-      command: ["/opt/hadoop/bin/ozone","datanode"]
+      command: ["ozone","datanode"]
       env_file:
         - ./docker-config
    ozoneManager:
       image: apache/hadoop-runner
       volumes:
         - ../..:/opt/hadoop
-        - ./jmxpromo.jar:/opt/jmxpromo.jar
       ports:
          - 9874:9874
       environment:
          ENSURE_OM_INITIALIZED: /data/metadata/ozoneManager/current/VERSION
       env_file:
           - ./docker-config
-      command: ["/opt/hadoop/bin/ozone","om"]
+      command: ["ozone","om"]
    scm:
       image: apache/hadoop-runner
       volumes:
          - ../..:/opt/hadoop
-         - ./jmxpromo.jar:/opt/jmxpromo.jar
       ports:
          - 9876:9876
       env_file:
           - ./docker-config
       environment:
           ENSURE_SCM_INITIALIZED: /data/metadata/scm/current/VERSION
-      command: ["/opt/hadoop/bin/ozone","scm"]
-   consul:
-      image: consul
-      command: ["agent", "-dev", "-ui", "-client", "0.0.0.0"]
-      ports:
-         - 8500:8500
+      command: ["ozone","scm"]
    prometheus:
      image: prom/prometheus
      volumes:
@@ -62,3 +54,12 @@ services:
      command: ["--config.file","/etc/prometheus.yml"]
      ports:
         - 9090:9090
+   freon:
+      image: apache/hadoop-runner
+      volumes:
+         - ../..:/opt/hadoop
+      environment:
+         SLEEP_SECONDS: 30
+      env_file:
+         - ./docker-config
+      command: ["ozone","freon","rk"]

+ 2 - 3
hadoop-ozone/dist/src/main/compose/ozoneperf/docker-config

@@ -17,13 +17,13 @@
 OZONE-SITE.XML_ozone.om.address=ozoneManager
 OZONE-SITE.XML_ozone.om.http-address=ozoneManager:9874
 OZONE-SITE.XML_ozone.scm.names=scm
-OZONE-SITE.XML_ozone.enabled=True
+OZONE-SITE.XML_ozone.enabled=true
 OZONE-SITE.XML_ozone.scm.datanode.id=/data/datanode.id
 OZONE-SITE.XML_ozone.scm.block.client.address=scm
 OZONE-SITE.XML_ozone.metadata.dirs=/data/metadata
-OZONE-SITE.XML_ozone.handler.type=distributed
 OZONE-SITE.XML_ozone.scm.client.address=scm
 OZONE-SITE.XML_hdds.datanode.dir=/data/hdds
+OZONE-SITE.XML_hdds.prometheus.endpoint.enabled=true
 
 HDFS-SITE.XML_rpc.metrics.quantile.enable=true
 HDFS-SITE.XML_rpc.metrics.percentiles.intervals=60,300
@@ -32,7 +32,6 @@ LOG4J.PROPERTIES_log4j.rootLogger=INFO, stdout
 LOG4J.PROPERTIES_log4j.appender.stdout=org.apache.log4j.ConsoleAppender
 LOG4J.PROPERTIES_log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
 LOG4J.PROPERTIES_log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n
-HADOOP_OPTS=-javaagent:/opt/jmxpromo.jar=port=0:consulHost=consul:consulMode=node
 LOG4J.PROPERTIES_log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR
 LOG4J.PROPERTIES_log4j.logger.org.apache.ratis.conf.ConfUtils=WARN
 LOG4J.PROPERTIES_log4j.logger.org.apache.hadoop.security.ShellBasedUnixGroupsMapping=ERROR

+ 0 - 21
hadoop-ozone/dist/src/main/compose/ozoneperf/init.sh

@@ -1,21 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
-EXPORTER_FILE="$DIR/jmxpromo.jar"
-if [ ! -f "$EXPORTER_FILE" ]; then
-   wget https://github.com/flokkr/jmxpromo/releases/download/0.11/jmx_prometheus_javaagent-0.11.jar -O $EXPORTER_FILE
-fi

+ 6 - 5
hadoop-ozone/dist/src/main/compose/ozoneperf/prometheus.yml

@@ -17,8 +17,9 @@ global:
   scrape_interval:     15s # By default, scrape targets every 15 seconds.
 
 scrape_configs:
-  - job_name: jmxexporter
-    consul_sd_configs:
-        - server: consul:8500
-          services:
-           - jmxexporter
+  - job_name: ozone
+    metrics_path: /prom
+    static_configs:
+     - targets:
+        - "scm:9876"
+        - "ozoneManager:9874"