You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by GitBox <gi...@apache.org> on 2019/12/31 12:08:27 UTC

[GitHub] [hadoop] XuQianJin-Stars opened a new pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay

XuQianJin-Stars opened a new pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay
URL: https://github.com/apache/hadoop/pull/1787
 
 
   https://issues.apache.org/jira/browse/HADOOP-16783

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay
URL: https://github.com/apache/hadoop/pull/1787#discussion_r362580228
 
 

 ##########
 File path: hadoop-common-project/hadoop-common/pom.xml
 ##########
 @@ -300,6 +300,26 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <!-- The client -->
 
 Review comment:
   these will all need to be optional

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] XuQianJin-Stars commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay
URL: https://github.com/apache/hadoop/pull/1787#discussion_r364240699
 
 

 ##########
 File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/prometheus/PushGatewaySink.java
 ##########
 @@ -0,0 +1,191 @@
+/**
+ * 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.metrics2.sink.prometheus;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Gauge;
+import io.prometheus.client.exporter.PushGateway;
+
+import org.apache.commons.configuration2.SubsetConfiguration;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricsException;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.MetricsSink;
+import org.apache.hadoop.metrics2.MetricsTag;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.metrics2.MetricType.COUNTER;
+import static org.apache.hadoop.metrics2.MetricType.GAUGE;
+
+/**
+ * A metrics sink that writes to a Prometheus PushGateWay.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class PushGatewaySink implements MetricsSink, Closeable {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(PushGatewaySink.class);
+
+  private static final String JOB_NAME = "job";
+  private static final String HOST_KEY = "host";
+  private static final String PORT_KEY = "port";
+  private static final String GROUP_KEY = "groupingKey";
+
+  private static final Pattern SPLIT_PATTERN =
+      Pattern.compile("(?<!(^|[A-Z_]))(?=[A-Z])|(?<!^)(?=[A-Z][a-z])");
+
+  private static final String NUM_OPEN_CONNECTION_SPERUSER = "numopenconnectionsperuser";
+
+  private static final String NULL = "null";
+
+  private Map<String, String> groupingKey;
+  private PushGateway pg = null;
+  private String jobName;
+
+  @Override
+  public void init(SubsetConfiguration conf) {
+    // Get PushGateWay host configurations.
+    jobName = conf.getString(JOB_NAME, "hadoop-job");
+    final String serverHost = conf.getString(HOST_KEY);
+    final int serverPort = Integer.parseInt(conf.getString(PORT_KEY));
+
+    if (serverHost == null || serverHost.isEmpty() || serverPort < 1) {
+      throw new MetricsException(
+          "Invalid host/port configuration. Host: " + serverHost + " Port: " + serverPort);
+    }
+
+    groupingKey = parseGroupingKey(conf.getString(GROUP_KEY, ""));
+    pg = new PushGateway(serverHost + ':' + serverPort);
+  }
+
+  @Override
+  public void putMetrics(MetricsRecord metricsRecord) {
+    try {
+      CollectorRegistry registry = new CollectorRegistry();
+      for (AbstractMetric metrics : metricsRecord.metrics()) {
+        if (metrics.type() == COUNTER
+            || metrics.type() == GAUGE) {
+
+          String key = getMetricsName(
+              metricsRecord.name(), metrics.name()).replace(" ", "");
+
+          int tagSize = metricsRecord.tags().size();
+          String[] labelNames = new String[tagSize];
+          String[] labelValues = new String[tagSize];
+          int index = 0;
+          for (MetricsTag tag : metricsRecord.tags()) {
+            String tagName = tag.name().toLowerCase();
+
+            //ignore specific tag which includes sub-hierarchy
+            if (NUM_OPEN_CONNECTION_SPERUSER.equals(tagName)) {
+              continue;
+            }
+            labelNames[index] = tagName;
+            labelValues[index] =
+                tag.value() == null ? NULL : tag.value();
+            index++;
+          }
+
+          switch (metrics.type()) {
+          case GAUGE:
+            Gauge.build(key, key)
+                .labelNames(labelNames)
+                .register(registry)
+                .labels(labelValues)
+                .set(metrics.value().doubleValue());
+            break;
+          case COUNTER:
+            Counter.build(key, key)
+                .labelNames(labelNames)
+                .register(registry)
+                .labels(labelValues)
+                .inc(metrics.value().doubleValue());
+            break;
+          default:
+            break;
+          }
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("register succeed, metrics name is: {} Type is :{} tag is: {} tagValue is {} Value is : {}"
+                , key, metrics.type().toString(), Arrays.toString(labelNames)
+                , Arrays.toString(labelValues), metrics.value().toString());
+          }
 
 Review comment:
   > toString() can be removed here.
   > 
   > * metrics.type().toString() -> metrics.type()
   > * metrics.value().toString() -> metrics.value()
   
   well, I have changed it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] XuQianJin-Stars commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay
URL: https://github.com/apache/hadoop/pull/1787#discussion_r362689844
 
 

 ##########
 File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/prometheus/PushGatewaySink.java
 ##########
 @@ -0,0 +1,191 @@
+/**
+ * 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.metrics2.sink.prometheus;
+
+import org.apache.commons.configuration2.SubsetConfiguration;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricsException;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.MetricsSink;
+import org.apache.hadoop.metrics2.MetricsTag;
+
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Gauge;
+import io.prometheus.client.exporter.PushGateway;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import static org.apache.hadoop.metrics2.MetricType.COUNTER;
+import static org.apache.hadoop.metrics2.MetricType.GAUGE;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class PushGatewaySink implements MetricsSink, Closeable {
+
+  private static final Log LOG = LogFactory.getLog(PushGatewaySink.class);
+
+  private static final String APP_ID = "app_id";
+  private static final String JOB_NAME = "job";
+  private static final String HOST_KEY = "host";
+  private static final String PORT_KEY = "port";
+  private static final String GROUP_KEY = "groupingKey";
+
+  private static final Pattern SPLIT_PATTERN =
+      Pattern.compile("(?<!(^|[A-Z_]))(?=[A-Z])|(?<!^)(?=[A-Z][a-z])");
+
+  private static final String NUM_OPEN_CONNECTION_SPERUSER = "numopenconnectionsperuser";
+
+  private static final String NULL = "null";
+
+  private Map<String, String> groupingKey;
+  private PushGateway pg = null;
+  private String jobName;
+
+  @Override
+  public void init(SubsetConfiguration conf) {
+    // Get PushGateWay host configurations.
+    jobName = conf.getString(JOB_NAME, "hadoop-job");
+    final String serverHost = conf.getString(HOST_KEY);
+    final int serverPort = Integer.parseInt(conf.getString(PORT_KEY));
+
+    if (serverHost == null || serverHost.isEmpty() || serverPort < 1) {
+      throw new MetricsException(
+          "Invalid host/port configuration. Host: " + serverHost + " Port: " + serverPort);
+    }
+
+    groupingKey = parseGroupingKey(conf.getString(GROUP_KEY, ""));
+    pg = new PushGateway(serverHost + ':' + serverPort);
+  }
+
+  @Override
+  public void putMetrics(MetricsRecord metricsRecord) {
+    try {
+      CollectorRegistry registry = new CollectorRegistry();
+      for (AbstractMetric metrics : metricsRecord.metrics()) {
+        if (metrics.type() == COUNTER
+            || metrics.type() == GAUGE) {
+
+          String key = getMetricsName(
+              metricsRecord.name(), metrics.name()).replace(" ", "");
+
+          int tagSize = metricsRecord.tags().size();
+          String[] labelNames = new String[tagSize];
+          String[] labelValues = new String[tagSize];
+          int index = 0;
+          for (MetricsTag tag : metricsRecord.tags()) {
+            String tagName = tag.name().toLowerCase();
+
+            //ignore specific tag which includes sub-hierarchy
+            if (NUM_OPEN_CONNECTION_SPERUSER.equals(tagName)) {
+              continue;
+            }
+            labelNames[index] = tagName;
+            labelValues[index] =
+                tag.value() == null ? NULL : tag.value();
+            index++;
+          }
+
+          switch (metrics.type()) {
+          case GAUGE:
+            Gauge.build(key, key)
+                .labelNames(labelNames)
+                .register(registry)
+                .labels(labelValues)
+                .set(metrics.value().doubleValue());
+            break;
+          case COUNTER:
+            Counter.build(key, key)
+                .labelNames(labelNames)
+                .register(registry)
+                .labels(labelValues)
+                .inc(metrics.value().doubleValue());
+            break;
+          default:
+            break;
+          }
+          LOG.debug(
+              "register succeed, metrics name is: " + key + " Type is :" + metrics.type().toString()
+                  + " tag is: " + Arrays.toString(labelNames)
+                  + " tagValue is " + Arrays.toString(labelValues)
+                  + " Value is : " + metrics.value().toString());
+        }
+      }
+      pg.push(registry, jobName, groupingKey);
+      LOG.info("pushing succeed");
+    } catch (Exception e) {
+      LOG.error("pushing job's metrics to gateway is failed ", e);
+    }
+  }
+
+  /**
+   * Convert CamelCase based names to lower-case names where the separator is the underscore, to
+   * follow prometheus naming conventions.
+   */
+  public String getMetricsName(String recordName,
+      String metricName) {
+    String baseName = StringUtils.capitalize(recordName)
+        + StringUtils.capitalize(metricName);
+    baseName = baseName.replace('-', '_');
+    String[] parts = SPLIT_PATTERN.split(baseName);
+    return String.join("_", parts).toLowerCase();
+  }
+
+  Map<String, String> parseGroupingKey(final String groupingKeyConfig) {
+    if (!groupingKeyConfig.isEmpty()) {
+      Map<String, String> groupingKey = new HashMap<>();
+      String[] kvs = groupingKeyConfig.split(";");
+      for (String kv : kvs) {
+        int idx = kv.indexOf("=");
+        if (idx < 0) {
+          LOG.warn("Invalid prometheusPushGateway groupingKey:" + kv + ", will be ignored");
+          continue;
+        }
+
+        String labelKey = kv.substring(0, idx);
+        String labelValue = kv.substring(idx + 1);
+        if (StringUtils.isEmpty(labelKey) || StringUtils.isEmpty(labelValue)) {
+          LOG.warn("Invalid groupingKey labelKey:" + labelKey + ", labelValue:" + labelValue
+              + " must not be empty");
+          continue;
+        }
+        groupingKey.put(labelKey, labelValue);
+      }
+      return groupingKey;
+    }
+    return Collections.emptyMap();
+  }
+
+  @Override
+  public void flush() {
+
+  }
+
+  @Override
+  public void close() throws IOException {
 
 Review comment:
   > push gateway
   
   Push gateway is not a port service and cannot be closed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay
URL: https://github.com/apache/hadoop/pull/1787#discussion_r362580997
 
 

 ##########
 File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/prometheus/PushGatewaySink.java
 ##########
 @@ -0,0 +1,191 @@
+/**
+ * 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.metrics2.sink.prometheus;
+
+import org.apache.commons.configuration2.SubsetConfiguration;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 
 Review comment:
   use SLF4J

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] aajisaka commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay

Posted by GitBox <gi...@apache.org>.
aajisaka commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay
URL: https://github.com/apache/hadoop/pull/1787#discussion_r364095663
 
 

 ##########
 File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/prometheus/PushGatewaySink.java
 ##########
 @@ -0,0 +1,191 @@
+/**
+ * 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.metrics2.sink.prometheus;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Gauge;
+import io.prometheus.client.exporter.PushGateway;
+
+import org.apache.commons.configuration2.SubsetConfiguration;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricsException;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.MetricsSink;
+import org.apache.hadoop.metrics2.MetricsTag;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.metrics2.MetricType.COUNTER;
+import static org.apache.hadoop.metrics2.MetricType.GAUGE;
+
+/**
+ * A metrics sink that writes to a Prometheus PushGateWay.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class PushGatewaySink implements MetricsSink, Closeable {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(PushGatewaySink.class);
+
+  private static final String JOB_NAME = "job";
+  private static final String HOST_KEY = "host";
+  private static final String PORT_KEY = "port";
+  private static final String GROUP_KEY = "groupingKey";
+
+  private static final Pattern SPLIT_PATTERN =
+      Pattern.compile("(?<!(^|[A-Z_]))(?=[A-Z])|(?<!^)(?=[A-Z][a-z])");
+
+  private static final String NUM_OPEN_CONNECTION_SPERUSER = "numopenconnectionsperuser";
+
+  private static final String NULL = "null";
+
+  private Map<String, String> groupingKey;
+  private PushGateway pg = null;
+  private String jobName;
+
+  @Override
+  public void init(SubsetConfiguration conf) {
+    // Get PushGateWay host configurations.
+    jobName = conf.getString(JOB_NAME, "hadoop-job");
+    final String serverHost = conf.getString(HOST_KEY);
+    final int serverPort = Integer.parseInt(conf.getString(PORT_KEY));
+
+    if (serverHost == null || serverHost.isEmpty() || serverPort < 1) {
+      throw new MetricsException(
+          "Invalid host/port configuration. Host: " + serverHost + " Port: " + serverPort);
+    }
+
+    groupingKey = parseGroupingKey(conf.getString(GROUP_KEY, ""));
+    pg = new PushGateway(serverHost + ':' + serverPort);
+  }
+
+  @Override
+  public void putMetrics(MetricsRecord metricsRecord) {
+    try {
+      CollectorRegistry registry = new CollectorRegistry();
+      for (AbstractMetric metrics : metricsRecord.metrics()) {
+        if (metrics.type() == COUNTER
+            || metrics.type() == GAUGE) {
+
+          String key = getMetricsName(
+              metricsRecord.name(), metrics.name()).replace(" ", "");
+
+          int tagSize = metricsRecord.tags().size();
+          String[] labelNames = new String[tagSize];
+          String[] labelValues = new String[tagSize];
+          int index = 0;
+          for (MetricsTag tag : metricsRecord.tags()) {
+            String tagName = tag.name().toLowerCase();
+
+            //ignore specific tag which includes sub-hierarchy
+            if (NUM_OPEN_CONNECTION_SPERUSER.equals(tagName)) {
+              continue;
+            }
+            labelNames[index] = tagName;
+            labelValues[index] =
+                tag.value() == null ? NULL : tag.value();
+            index++;
+          }
+
+          switch (metrics.type()) {
+          case GAUGE:
+            Gauge.build(key, key)
+                .labelNames(labelNames)
+                .register(registry)
+                .labels(labelValues)
+                .set(metrics.value().doubleValue());
+            break;
+          case COUNTER:
+            Counter.build(key, key)
+                .labelNames(labelNames)
+                .register(registry)
+                .labels(labelValues)
+                .inc(metrics.value().doubleValue());
+            break;
+          default:
+            break;
+          }
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("register succeed, metrics name is: {} Type is :{} tag is: {} tagValue is {} Value is : {}"
+                , key, metrics.type().toString(), Arrays.toString(labelNames)
+                , Arrays.toString(labelValues), metrics.value().toString());
+          }
 
 Review comment:
   toString() can be removed here.
   * metrics.type().toString() -> metrics.type()
   * metrics.value().toString() -> metrics.value()

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay
URL: https://github.com/apache/hadoop/pull/1787#discussion_r362580521
 
 

 ##########
 File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/prometheus/PushGatewaySink.java
 ##########
 @@ -0,0 +1,191 @@
+/**
+ * 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.metrics2.sink.prometheus;
+
+import org.apache.commons.configuration2.SubsetConfiguration;
 
 Review comment:
   prefer import order of
   
   java.*
   
   not asf
   
   org.apache.*
   
   statics

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus removed a comment on issue #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay

Posted by GitBox <gi...@apache.org>.
hadoop-yetus removed a comment on issue #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay
URL: https://github.com/apache/hadoop/pull/1787#issuecomment-569938254
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |  25m  0s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   1m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  18m  9s |  trunk passed  |
   | +1 :green_heart: |  compile  |  16m 36s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   2m 37s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m 57s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  18m 24s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m  7s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   2m  3s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +0 :ok: |  findbugs  |   0m 33s |  branch/hadoop-project no findbugs output file (findbugsXml.xml)  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 29s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m  1s |  the patch passed  |
   | +1 :green_heart: |  compile  |  15m 58s |  the patch passed  |
   | +1 :green_heart: |  javac  |  15m 58s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 38s |  root: The patch generated 8 new + 50 unchanged - 0 fixed = 58 total (was 50)  |
   | +1 :green_heart: |  mvnsite  |   1m 58s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  3s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedclient  |  12m 49s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m  8s |  the patch passed  |
   | +0 :ok: |  findbugs  |   0m 33s |  hadoop-project has no data from findbugs  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 31s |  hadoop-project in the patch passed.  |
   | +1 :green_heart: |  unit  |   9m  6s |  hadoop-common in the patch passed.  |
   | -1 :x: |  asflicense  |   0m 54s |  The patch generated 1 ASF License warnings.  |
   |  |   | 139m 39s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1787 |
   | JIRA Issue | HADOOP-16783 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient xml findbugs checkstyle |
   | uname | Linux 5d2b5c273cbb 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / 6242391 |
   | Default Java | 1.8.0_232 |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/1/artifact/out/diff-checkstyle-root.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/1/testReport/ |
   | asflicense | https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/1/artifact/out/patch-asflicense-problems.txt |
   | Max. process+thread count | 1446 (vs. ulimit of 5500) |
   | modules | C: hadoop-project hadoop-common-project/hadoop-common U: . |
   | Console output | https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/1/console |
   | versions | git=2.7.4 maven=3.3.9 findbugs=3.1.0-RC1 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on issue #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on issue #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay
URL: https://github.com/apache/hadoop/pull/1787#issuecomment-570460450
 
 
   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 35s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   1m  6s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  18m 30s |  trunk passed  |
   | +1 :green_heart: |  compile  |  16m 32s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   2m 35s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m 57s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  18m 35s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m  7s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   2m  5s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +0 :ok: |  findbugs  |   0m 33s |  branch/hadoop-project no findbugs output file (findbugsXml.xml)  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m  1s |  the patch passed  |
   | +1 :green_heart: |  compile  |  15m 50s |  the patch passed  |
   | +1 :green_heart: |  javac  |  15m 50s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 34s |  root: The patch generated 15 new + 50 unchanged - 0 fixed = 65 total (was 50)  |
   | +1 :green_heart: |  mvnsite  |   1m 59s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  3s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedclient  |  12m 33s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m  7s |  the patch passed  |
   | +0 :ok: |  findbugs  |   0m 33s |  hadoop-project has no data from findbugs  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 33s |  hadoop-project in the patch passed.  |
   | +1 :green_heart: |  unit  |   9m  6s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 53s |  The patch does not generate ASF License warnings.  |
   |  |   | 114m 53s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/3/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1787 |
   | JIRA Issue | HADOOP-16783 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient xml findbugs checkstyle |
   | uname | Linux eccbf530e0ab 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / b19d87c |
   | Default Java | 1.8.0_232 |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/3/artifact/out/diff-checkstyle-root.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/3/testReport/ |
   | Max. process+thread count | 1426 (vs. ulimit of 5500) |
   | modules | C: hadoop-project hadoop-common-project/hadoop-common U: . |
   | Console output | https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/3/console |
   | versions | git=2.7.4 maven=3.3.9 findbugs=3.1.0-RC1 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay
URL: https://github.com/apache/hadoop/pull/1787#discussion_r362580087
 
 

 ##########
 File path: hadoop-common-project/hadoop-common/pom.xml
 ##########
 @@ -300,6 +300,26 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <!-- The client -->
 
 Review comment:
   these will all need to be optional

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay
URL: https://github.com/apache/hadoop/pull/1787#discussion_r362581225
 
 

 ##########
 File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/prometheus/PushGatewaySink.java
 ##########
 @@ -0,0 +1,191 @@
+/**
+ * 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.metrics2.sink.prometheus;
+
+import org.apache.commons.configuration2.SubsetConfiguration;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricsException;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.MetricsSink;
+import org.apache.hadoop.metrics2.MetricsTag;
+
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Gauge;
+import io.prometheus.client.exporter.PushGateway;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import static org.apache.hadoop.metrics2.MetricType.COUNTER;
+import static org.apache.hadoop.metrics2.MetricType.GAUGE;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class PushGatewaySink implements MetricsSink, Closeable {
+
+  private static final Log LOG = LogFactory.getLog(PushGatewaySink.class);
+
+  private static final String APP_ID = "app_id";
+  private static final String JOB_NAME = "job";
+  private static final String HOST_KEY = "host";
+  private static final String PORT_KEY = "port";
+  private static final String GROUP_KEY = "groupingKey";
+
+  private static final Pattern SPLIT_PATTERN =
+      Pattern.compile("(?<!(^|[A-Z_]))(?=[A-Z])|(?<!^)(?=[A-Z][a-z])");
+
+  private static final String NUM_OPEN_CONNECTION_SPERUSER = "numopenconnectionsperuser";
+
+  private static final String NULL = "null";
+
+  private Map<String, String> groupingKey;
+  private PushGateway pg = null;
+  private String jobName;
+
+  @Override
+  public void init(SubsetConfiguration conf) {
+    // Get PushGateWay host configurations.
+    jobName = conf.getString(JOB_NAME, "hadoop-job");
+    final String serverHost = conf.getString(HOST_KEY);
+    final int serverPort = Integer.parseInt(conf.getString(PORT_KEY));
+
+    if (serverHost == null || serverHost.isEmpty() || serverPort < 1) {
+      throw new MetricsException(
+          "Invalid host/port configuration. Host: " + serverHost + " Port: " + serverPort);
+    }
+
+    groupingKey = parseGroupingKey(conf.getString(GROUP_KEY, ""));
+    pg = new PushGateway(serverHost + ':' + serverPort);
+  }
+
+  @Override
+  public void putMetrics(MetricsRecord metricsRecord) {
+    try {
+      CollectorRegistry registry = new CollectorRegistry();
+      for (AbstractMetric metrics : metricsRecord.metrics()) {
+        if (metrics.type() == COUNTER
+            || metrics.type() == GAUGE) {
+
+          String key = getMetricsName(
+              metricsRecord.name(), metrics.name()).replace(" ", "");
+
+          int tagSize = metricsRecord.tags().size();
+          String[] labelNames = new String[tagSize];
+          String[] labelValues = new String[tagSize];
+          int index = 0;
+          for (MetricsTag tag : metricsRecord.tags()) {
+            String tagName = tag.name().toLowerCase();
+
+            //ignore specific tag which includes sub-hierarchy
+            if (NUM_OPEN_CONNECTION_SPERUSER.equals(tagName)) {
+              continue;
+            }
+            labelNames[index] = tagName;
+            labelValues[index] =
+                tag.value() == null ? NULL : tag.value();
+            index++;
+          }
+
+          switch (metrics.type()) {
+          case GAUGE:
+            Gauge.build(key, key)
+                .labelNames(labelNames)
+                .register(registry)
+                .labels(labelValues)
+                .set(metrics.value().doubleValue());
+            break;
+          case COUNTER:
+            Counter.build(key, key)
+                .labelNames(labelNames)
+                .register(registry)
+                .labels(labelValues)
+                .inc(metrics.value().doubleValue());
+            break;
+          default:
+            break;
+          }
+          LOG.debug(
+              "register succeed, metrics name is: " + key + " Type is :" + metrics.type().toString()
+                  + " tag is: " + Arrays.toString(labelNames)
+                  + " tagValue is " + Arrays.toString(labelValues)
+                  + " Value is : " + metrics.value().toString());
+        }
+      }
+      pg.push(registry, jobName, groupingKey);
+      LOG.info("pushing succeed");
+    } catch (Exception e) {
+      LOG.error("pushing job's metrics to gateway is failed ", e);
+    }
+  }
+
+  /**
+   * Convert CamelCase based names to lower-case names where the separator is the underscore, to
+   * follow prometheus naming conventions.
+   */
+  public String getMetricsName(String recordName,
+      String metricName) {
+    String baseName = StringUtils.capitalize(recordName)
+        + StringUtils.capitalize(metricName);
+    baseName = baseName.replace('-', '_');
+    String[] parts = SPLIT_PATTERN.split(baseName);
+    return String.join("_", parts).toLowerCase();
+  }
+
+  Map<String, String> parseGroupingKey(final String groupingKeyConfig) {
+    if (!groupingKeyConfig.isEmpty()) {
+      Map<String, String> groupingKey = new HashMap<>();
+      String[] kvs = groupingKeyConfig.split(";");
+      for (String kv : kvs) {
+        int idx = kv.indexOf("=");
+        if (idx < 0) {
+          LOG.warn("Invalid prometheusPushGateway groupingKey:" + kv + ", will be ignored");
 
 Review comment:
   slf4j lets you use {} for expansion

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay
URL: https://github.com/apache/hadoop/pull/1787#discussion_r362580087
 
 

 ##########
 File path: hadoop-common-project/hadoop-common/pom.xml
 ##########
 @@ -300,6 +300,26 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <!-- The client -->
 
 Review comment:
   these will all need to be optional

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] XuQianJin-Stars commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay
URL: https://github.com/apache/hadoop/pull/1787#discussion_r362692298
 
 

 ##########
 File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/prometheus/PushGatewaySink.java
 ##########
 @@ -0,0 +1,191 @@
+/**
+ * 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.metrics2.sink.prometheus;
+
+import org.apache.commons.configuration2.SubsetConfiguration;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricsException;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.MetricsSink;
+import org.apache.hadoop.metrics2.MetricsTag;
+
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Gauge;
+import io.prometheus.client.exporter.PushGateway;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import static org.apache.hadoop.metrics2.MetricType.COUNTER;
+import static org.apache.hadoop.metrics2.MetricType.GAUGE;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
 
 Review comment:
   > add a javadoc. And does it need to be public?
   
   I have observed that FileSink is public, so I think it should be kept public.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on issue #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on issue #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay
URL: https://github.com/apache/hadoop/pull/1787#issuecomment-572044282
 
 
   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 34s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  18m 11s |  trunk passed  |
   | +1 :green_heart: |  compile  |  16m 43s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   2m 40s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m  1s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  18m 29s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m  6s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   2m  5s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +0 :ok: |  findbugs  |   0m 33s |  branch/hadoop-project no findbugs output file (findbugsXml.xml)  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 25s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m  0s |  the patch passed  |
   | +1 :green_heart: |  compile  |  15m 54s |  the patch passed  |
   | +1 :green_heart: |  javac  |  15m 54s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 38s |  root: The patch generated 17 new + 50 unchanged - 0 fixed = 67 total (was 50)  |
   | +1 :green_heart: |  mvnsite  |   1m 58s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  3s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedclient  |  13m  1s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m  4s |  the patch passed  |
   | +0 :ok: |  findbugs  |   0m 34s |  hadoop-project has no data from findbugs  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 33s |  hadoop-project in the patch passed.  |
   | +1 :green_heart: |  unit  |   9m  9s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 54s |  The patch does not generate ASF License warnings.  |
   |  |   | 114m 32s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/5/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1787 |
   | JIRA Issue | HADOOP-16783 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient xml findbugs checkstyle |
   | uname | Linux 02206d997f02 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / 7030722 |
   | Default Java | 1.8.0_232 |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/5/artifact/out/diff-checkstyle-root.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/5/testReport/ |
   | Max. process+thread count | 1472 (vs. ulimit of 5500) |
   | modules | C: hadoop-project hadoop-common-project/hadoop-common U: . |
   | Console output | https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/5/console |
   | versions | git=2.7.4 maven=3.3.9 findbugs=3.1.0-RC1 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on issue #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on issue #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay
URL: https://github.com/apache/hadoop/pull/1787#issuecomment-571481494
 
 
   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 36s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   1m 11s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  18m 34s |  trunk passed  |
   | +1 :green_heart: |  compile  |  16m 38s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   2m 36s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m 58s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  18m 29s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m  6s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   2m  4s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +0 :ok: |  findbugs  |   0m 33s |  branch/hadoop-project no findbugs output file (findbugsXml.xml)  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m  0s |  the patch passed  |
   | +1 :green_heart: |  compile  |  15m 56s |  the patch passed  |
   | +1 :green_heart: |  javac  |  15m 56s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 37s |  root: The patch generated 17 new + 50 unchanged - 0 fixed = 67 total (was 50)  |
   | +1 :green_heart: |  mvnsite  |   1m 58s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  3s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedclient  |  12m 36s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m  7s |  the patch passed  |
   | +0 :ok: |  findbugs  |   0m 32s |  hadoop-project has no data from findbugs  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 32s |  hadoop-project in the patch passed.  |
   | +1 :green_heart: |  unit  |   9m 13s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 53s |  The patch does not generate ASF License warnings.  |
   |  |   | 115m 17s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/4/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1787 |
   | JIRA Issue | HADOOP-16783 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient xml findbugs checkstyle |
   | uname | Linux 9b685cfffc0d 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / 59aac00 |
   | Default Java | 1.8.0_232 |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/4/artifact/out/diff-checkstyle-root.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/4/testReport/ |
   | Max. process+thread count | 1429 (vs. ulimit of 5500) |
   | modules | C: hadoop-project hadoop-common-project/hadoop-common U: . |
   | Console output | https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/4/console |
   | versions | git=2.7.4 maven=3.3.9 findbugs=3.1.0-RC1 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay
URL: https://github.com/apache/hadoop/pull/1787#discussion_r362581100
 
 

 ##########
 File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/prometheus/PushGatewaySink.java
 ##########
 @@ -0,0 +1,191 @@
+/**
+ * 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.metrics2.sink.prometheus;
+
+import org.apache.commons.configuration2.SubsetConfiguration;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricsException;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.MetricsSink;
+import org.apache.hadoop.metrics2.MetricsTag;
+
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Gauge;
+import io.prometheus.client.exporter.PushGateway;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import static org.apache.hadoop.metrics2.MetricType.COUNTER;
+import static org.apache.hadoop.metrics2.MetricType.GAUGE;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class PushGatewaySink implements MetricsSink, Closeable {
+
+  private static final Log LOG = LogFactory.getLog(PushGatewaySink.class);
+
+  private static final String APP_ID = "app_id";
+  private static final String JOB_NAME = "job";
+  private static final String HOST_KEY = "host";
+  private static final String PORT_KEY = "port";
+  private static final String GROUP_KEY = "groupingKey";
+
+  private static final Pattern SPLIT_PATTERN =
+      Pattern.compile("(?<!(^|[A-Z_]))(?=[A-Z])|(?<!^)(?=[A-Z][a-z])");
+
+  private static final String NUM_OPEN_CONNECTION_SPERUSER = "numopenconnectionsperuser";
+
+  private static final String NULL = "null";
+
+  private Map<String, String> groupingKey;
+  private PushGateway pg = null;
+  private String jobName;
+
+  @Override
+  public void init(SubsetConfiguration conf) {
+    // Get PushGateWay host configurations.
+    jobName = conf.getString(JOB_NAME, "hadoop-job");
+    final String serverHost = conf.getString(HOST_KEY);
+    final int serverPort = Integer.parseInt(conf.getString(PORT_KEY));
+
+    if (serverHost == null || serverHost.isEmpty() || serverPort < 1) {
+      throw new MetricsException(
+          "Invalid host/port configuration. Host: " + serverHost + " Port: " + serverPort);
+    }
+
+    groupingKey = parseGroupingKey(conf.getString(GROUP_KEY, ""));
+    pg = new PushGateway(serverHost + ':' + serverPort);
+  }
+
+  @Override
+  public void putMetrics(MetricsRecord metricsRecord) {
+    try {
+      CollectorRegistry registry = new CollectorRegistry();
+      for (AbstractMetric metrics : metricsRecord.metrics()) {
+        if (metrics.type() == COUNTER
+            || metrics.type() == GAUGE) {
+
+          String key = getMetricsName(
+              metricsRecord.name(), metrics.name()).replace(" ", "");
+
+          int tagSize = metricsRecord.tags().size();
+          String[] labelNames = new String[tagSize];
+          String[] labelValues = new String[tagSize];
+          int index = 0;
+          for (MetricsTag tag : metricsRecord.tags()) {
+            String tagName = tag.name().toLowerCase();
+
+            //ignore specific tag which includes sub-hierarchy
+            if (NUM_OPEN_CONNECTION_SPERUSER.equals(tagName)) {
+              continue;
+            }
+            labelNames[index] = tagName;
+            labelValues[index] =
+                tag.value() == null ? NULL : tag.value();
+            index++;
+          }
+
+          switch (metrics.type()) {
+          case GAUGE:
+            Gauge.build(key, key)
+                .labelNames(labelNames)
+                .register(registry)
+                .labels(labelValues)
+                .set(metrics.value().doubleValue());
+            break;
+          case COUNTER:
+            Counter.build(key, key)
+                .labelNames(labelNames)
+                .register(registry)
+                .labels(labelValues)
+                .inc(metrics.value().doubleValue());
+            break;
+          default:
+            break;
+          }
+          LOG.debug(
+              "register succeed, metrics name is: " + key + " Type is :" + metrics.type().toString()
+                  + " tag is: " + Arrays.toString(labelNames)
+                  + " tagValue is " + Arrays.toString(labelValues)
+                  + " Value is : " + metrics.value().toString());
+        }
+      }
+      pg.push(registry, jobName, groupingKey);
+      LOG.info("pushing succeed");
 
 Review comment:
   this is potentially very noisy in the logs

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay
URL: https://github.com/apache/hadoop/pull/1787#discussion_r362580900
 
 

 ##########
 File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/prometheus/PushGatewaySink.java
 ##########
 @@ -0,0 +1,191 @@
+/**
+ * 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.metrics2.sink.prometheus;
+
+import org.apache.commons.configuration2.SubsetConfiguration;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricsException;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.MetricsSink;
+import org.apache.hadoop.metrics2.MetricsTag;
+
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Gauge;
+import io.prometheus.client.exporter.PushGateway;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import static org.apache.hadoop.metrics2.MetricType.COUNTER;
+import static org.apache.hadoop.metrics2.MetricType.GAUGE;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class PushGatewaySink implements MetricsSink, Closeable {
+
+  private static final Log LOG = LogFactory.getLog(PushGatewaySink.class);
+
+  private static final String APP_ID = "app_id";
+  private static final String JOB_NAME = "job";
+  private static final String HOST_KEY = "host";
+  private static final String PORT_KEY = "port";
+  private static final String GROUP_KEY = "groupingKey";
+
+  private static final Pattern SPLIT_PATTERN =
+      Pattern.compile("(?<!(^|[A-Z_]))(?=[A-Z])|(?<!^)(?=[A-Z][a-z])");
+
+  private static final String NUM_OPEN_CONNECTION_SPERUSER = "numopenconnectionsperuser";
+
+  private static final String NULL = "null";
+
+  private Map<String, String> groupingKey;
+  private PushGateway pg = null;
+  private String jobName;
+
+  @Override
+  public void init(SubsetConfiguration conf) {
+    // Get PushGateWay host configurations.
+    jobName = conf.getString(JOB_NAME, "hadoop-job");
+    final String serverHost = conf.getString(HOST_KEY);
+    final int serverPort = Integer.parseInt(conf.getString(PORT_KEY));
+
+    if (serverHost == null || serverHost.isEmpty() || serverPort < 1) {
+      throw new MetricsException(
+          "Invalid host/port configuration. Host: " + serverHost + " Port: " + serverPort);
+    }
+
+    groupingKey = parseGroupingKey(conf.getString(GROUP_KEY, ""));
+    pg = new PushGateway(serverHost + ':' + serverPort);
+  }
+
+  @Override
+  public void putMetrics(MetricsRecord metricsRecord) {
+    try {
+      CollectorRegistry registry = new CollectorRegistry();
+      for (AbstractMetric metrics : metricsRecord.metrics()) {
+        if (metrics.type() == COUNTER
+            || metrics.type() == GAUGE) {
+
+          String key = getMetricsName(
+              metricsRecord.name(), metrics.name()).replace(" ", "");
+
+          int tagSize = metricsRecord.tags().size();
+          String[] labelNames = new String[tagSize];
+          String[] labelValues = new String[tagSize];
+          int index = 0;
+          for (MetricsTag tag : metricsRecord.tags()) {
+            String tagName = tag.name().toLowerCase();
+
+            //ignore specific tag which includes sub-hierarchy
+            if (NUM_OPEN_CONNECTION_SPERUSER.equals(tagName)) {
+              continue;
+            }
+            labelNames[index] = tagName;
+            labelValues[index] =
+                tag.value() == null ? NULL : tag.value();
+            index++;
+          }
+
+          switch (metrics.type()) {
+          case GAUGE:
+            Gauge.build(key, key)
+                .labelNames(labelNames)
+                .register(registry)
+                .labels(labelValues)
+                .set(metrics.value().doubleValue());
+            break;
+          case COUNTER:
+            Counter.build(key, key)
+                .labelNames(labelNames)
+                .register(registry)
+                .labels(labelValues)
+                .inc(metrics.value().doubleValue());
+            break;
+          default:
+            break;
+          }
+          LOG.debug(
 
 Review comment:
   very expensive. Wrap with LOG.isDebugEnabled

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay
URL: https://github.com/apache/hadoop/pull/1787#discussion_r362581354
 
 

 ##########
 File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/prometheus/PushGatewaySink.java
 ##########
 @@ -0,0 +1,191 @@
+/**
+ * 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.metrics2.sink.prometheus;
+
+import org.apache.commons.configuration2.SubsetConfiguration;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricsException;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.MetricsSink;
+import org.apache.hadoop.metrics2.MetricsTag;
+
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Gauge;
+import io.prometheus.client.exporter.PushGateway;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import static org.apache.hadoop.metrics2.MetricType.COUNTER;
+import static org.apache.hadoop.metrics2.MetricType.GAUGE;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class PushGatewaySink implements MetricsSink, Closeable {
+
+  private static final Log LOG = LogFactory.getLog(PushGatewaySink.class);
+
+  private static final String APP_ID = "app_id";
+  private static final String JOB_NAME = "job";
+  private static final String HOST_KEY = "host";
+  private static final String PORT_KEY = "port";
+  private static final String GROUP_KEY = "groupingKey";
+
+  private static final Pattern SPLIT_PATTERN =
+      Pattern.compile("(?<!(^|[A-Z_]))(?=[A-Z])|(?<!^)(?=[A-Z][a-z])");
+
+  private static final String NUM_OPEN_CONNECTION_SPERUSER = "numopenconnectionsperuser";
+
+  private static final String NULL = "null";
+
+  private Map<String, String> groupingKey;
+  private PushGateway pg = null;
+  private String jobName;
+
+  @Override
+  public void init(SubsetConfiguration conf) {
+    // Get PushGateWay host configurations.
+    jobName = conf.getString(JOB_NAME, "hadoop-job");
+    final String serverHost = conf.getString(HOST_KEY);
+    final int serverPort = Integer.parseInt(conf.getString(PORT_KEY));
+
+    if (serverHost == null || serverHost.isEmpty() || serverPort < 1) {
+      throw new MetricsException(
+          "Invalid host/port configuration. Host: " + serverHost + " Port: " + serverPort);
+    }
+
+    groupingKey = parseGroupingKey(conf.getString(GROUP_KEY, ""));
+    pg = new PushGateway(serverHost + ':' + serverPort);
+  }
+
+  @Override
+  public void putMetrics(MetricsRecord metricsRecord) {
+    try {
+      CollectorRegistry registry = new CollectorRegistry();
+      for (AbstractMetric metrics : metricsRecord.metrics()) {
+        if (metrics.type() == COUNTER
+            || metrics.type() == GAUGE) {
+
+          String key = getMetricsName(
+              metricsRecord.name(), metrics.name()).replace(" ", "");
+
+          int tagSize = metricsRecord.tags().size();
+          String[] labelNames = new String[tagSize];
+          String[] labelValues = new String[tagSize];
+          int index = 0;
+          for (MetricsTag tag : metricsRecord.tags()) {
+            String tagName = tag.name().toLowerCase();
+
+            //ignore specific tag which includes sub-hierarchy
+            if (NUM_OPEN_CONNECTION_SPERUSER.equals(tagName)) {
+              continue;
+            }
+            labelNames[index] = tagName;
+            labelValues[index] =
+                tag.value() == null ? NULL : tag.value();
+            index++;
+          }
+
+          switch (metrics.type()) {
+          case GAUGE:
+            Gauge.build(key, key)
+                .labelNames(labelNames)
+                .register(registry)
+                .labels(labelValues)
+                .set(metrics.value().doubleValue());
+            break;
+          case COUNTER:
+            Counter.build(key, key)
+                .labelNames(labelNames)
+                .register(registry)
+                .labels(labelValues)
+                .inc(metrics.value().doubleValue());
+            break;
+          default:
+            break;
+          }
+          LOG.debug(
+              "register succeed, metrics name is: " + key + " Type is :" + metrics.type().toString()
+                  + " tag is: " + Arrays.toString(labelNames)
+                  + " tagValue is " + Arrays.toString(labelValues)
+                  + " Value is : " + metrics.value().toString());
+        }
+      }
+      pg.push(registry, jobName, groupingKey);
+      LOG.info("pushing succeed");
+    } catch (Exception e) {
+      LOG.error("pushing job's metrics to gateway is failed ", e);
+    }
+  }
+
+  /**
+   * Convert CamelCase based names to lower-case names where the separator is the underscore, to
+   * follow prometheus naming conventions.
+   */
+  public String getMetricsName(String recordName,
+      String metricName) {
+    String baseName = StringUtils.capitalize(recordName)
+        + StringUtils.capitalize(metricName);
+    baseName = baseName.replace('-', '_');
+    String[] parts = SPLIT_PATTERN.split(baseName);
+    return String.join("_", parts).toLowerCase();
+  }
+
+  Map<String, String> parseGroupingKey(final String groupingKeyConfig) {
+    if (!groupingKeyConfig.isEmpty()) {
+      Map<String, String> groupingKey = new HashMap<>();
+      String[] kvs = groupingKeyConfig.split(";");
+      for (String kv : kvs) {
+        int idx = kv.indexOf("=");
+        if (idx < 0) {
+          LOG.warn("Invalid prometheusPushGateway groupingKey:" + kv + ", will be ignored");
+          continue;
+        }
+
+        String labelKey = kv.substring(0, idx);
+        String labelValue = kv.substring(idx + 1);
+        if (StringUtils.isEmpty(labelKey) || StringUtils.isEmpty(labelValue)) {
+          LOG.warn("Invalid groupingKey labelKey:" + labelKey + ", labelValue:" + labelValue
 
 Review comment:
   again, SLF4J + {}

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay
URL: https://github.com/apache/hadoop/pull/1787#discussion_r362580702
 
 

 ##########
 File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/prometheus/PushGatewaySink.java
 ##########
 @@ -0,0 +1,191 @@
+/**
+ * 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.metrics2.sink.prometheus;
+
+import org.apache.commons.configuration2.SubsetConfiguration;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricsException;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.MetricsSink;
+import org.apache.hadoop.metrics2.MetricsTag;
+
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Gauge;
+import io.prometheus.client.exporter.PushGateway;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import static org.apache.hadoop.metrics2.MetricType.COUNTER;
+import static org.apache.hadoop.metrics2.MetricType.GAUGE;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
 
 Review comment:
   add a javadoc. And does it need to be public? 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay
URL: https://github.com/apache/hadoop/pull/1787#discussion_r380803849
 
 

 ##########
 File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/prometheus/PushGatewaySink.java
 ##########
 @@ -0,0 +1,191 @@
+/**
+ * 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.metrics2.sink.prometheus;
+
+import org.apache.commons.configuration2.SubsetConfiguration;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricsException;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.MetricsSink;
+import org.apache.hadoop.metrics2.MetricsTag;
+
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Gauge;
+import io.prometheus.client.exporter.PushGateway;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import static org.apache.hadoop.metrics2.MetricType.COUNTER;
+import static org.apache.hadoop.metrics2.MetricType.GAUGE;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
 
 Review comment:
   ok

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on issue #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on issue #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay
URL: https://github.com/apache/hadoop/pull/1787#issuecomment-570122662
 
 
   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 34s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   1m  9s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  17m 59s |  trunk passed  |
   | +1 :green_heart: |  compile  |  16m 36s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   2m 38s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m 59s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  18m 24s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m  8s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   2m  4s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +0 :ok: |  findbugs  |   0m 34s |  branch/hadoop-project no findbugs output file (findbugsXml.xml)  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m  1s |  the patch passed  |
   | +1 :green_heart: |  compile  |  15m 49s |  the patch passed  |
   | +1 :green_heart: |  javac  |  15m 49s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 35s |  root: The patch generated 15 new + 50 unchanged - 0 fixed = 65 total (was 50)  |
   | +1 :green_heart: |  mvnsite  |   2m  0s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  3s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedclient  |  12m 48s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m  9s |  the patch passed  |
   | +0 :ok: |  findbugs  |   0m 32s |  hadoop-project has no data from findbugs  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 32s |  hadoop-project in the patch passed.  |
   | +1 :green_heart: |  unit  |   9m  3s |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 53s |  The patch does not generate ASF License warnings.  |
   |  |   | 114m 32s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/2/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1787 |
   | JIRA Issue | HADOOP-16783 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient xml findbugs checkstyle |
   | uname | Linux bccd099dae58 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / 074050c |
   | Default Java | 1.8.0_232 |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/2/artifact/out/diff-checkstyle-root.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/2/testReport/ |
   | Max. process+thread count | 1479 (vs. ulimit of 5500) |
   | modules | C: hadoop-project hadoop-common-project/hadoop-common U: . |
   | Console output | https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/2/console |
   | versions | git=2.7.4 maven=3.3.9 findbugs=3.1.0-RC1 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay
URL: https://github.com/apache/hadoop/pull/1787#discussion_r362581550
 
 

 ##########
 File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/prometheus/PushGatewaySink.java
 ##########
 @@ -0,0 +1,191 @@
+/**
+ * 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.metrics2.sink.prometheus;
+
+import org.apache.commons.configuration2.SubsetConfiguration;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricsException;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.MetricsSink;
+import org.apache.hadoop.metrics2.MetricsTag;
+
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Gauge;
+import io.prometheus.client.exporter.PushGateway;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import static org.apache.hadoop.metrics2.MetricType.COUNTER;
+import static org.apache.hadoop.metrics2.MetricType.GAUGE;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class PushGatewaySink implements MetricsSink, Closeable {
+
+  private static final Log LOG = LogFactory.getLog(PushGatewaySink.class);
+
+  private static final String APP_ID = "app_id";
+  private static final String JOB_NAME = "job";
+  private static final String HOST_KEY = "host";
+  private static final String PORT_KEY = "port";
+  private static final String GROUP_KEY = "groupingKey";
+
+  private static final Pattern SPLIT_PATTERN =
+      Pattern.compile("(?<!(^|[A-Z_]))(?=[A-Z])|(?<!^)(?=[A-Z][a-z])");
+
+  private static final String NUM_OPEN_CONNECTION_SPERUSER = "numopenconnectionsperuser";
+
+  private static final String NULL = "null";
+
+  private Map<String, String> groupingKey;
+  private PushGateway pg = null;
+  private String jobName;
+
+  @Override
+  public void init(SubsetConfiguration conf) {
+    // Get PushGateWay host configurations.
+    jobName = conf.getString(JOB_NAME, "hadoop-job");
+    final String serverHost = conf.getString(HOST_KEY);
+    final int serverPort = Integer.parseInt(conf.getString(PORT_KEY));
+
+    if (serverHost == null || serverHost.isEmpty() || serverPort < 1) {
+      throw new MetricsException(
+          "Invalid host/port configuration. Host: " + serverHost + " Port: " + serverPort);
+    }
+
+    groupingKey = parseGroupingKey(conf.getString(GROUP_KEY, ""));
+    pg = new PushGateway(serverHost + ':' + serverPort);
+  }
+
+  @Override
+  public void putMetrics(MetricsRecord metricsRecord) {
+    try {
+      CollectorRegistry registry = new CollectorRegistry();
+      for (AbstractMetric metrics : metricsRecord.metrics()) {
+        if (metrics.type() == COUNTER
+            || metrics.type() == GAUGE) {
+
+          String key = getMetricsName(
+              metricsRecord.name(), metrics.name()).replace(" ", "");
+
+          int tagSize = metricsRecord.tags().size();
+          String[] labelNames = new String[tagSize];
+          String[] labelValues = new String[tagSize];
+          int index = 0;
+          for (MetricsTag tag : metricsRecord.tags()) {
+            String tagName = tag.name().toLowerCase();
+
+            //ignore specific tag which includes sub-hierarchy
+            if (NUM_OPEN_CONNECTION_SPERUSER.equals(tagName)) {
+              continue;
+            }
+            labelNames[index] = tagName;
+            labelValues[index] =
+                tag.value() == null ? NULL : tag.value();
+            index++;
+          }
+
+          switch (metrics.type()) {
+          case GAUGE:
+            Gauge.build(key, key)
+                .labelNames(labelNames)
+                .register(registry)
+                .labels(labelValues)
+                .set(metrics.value().doubleValue());
+            break;
+          case COUNTER:
+            Counter.build(key, key)
+                .labelNames(labelNames)
+                .register(registry)
+                .labels(labelValues)
+                .inc(metrics.value().doubleValue());
+            break;
+          default:
+            break;
+          }
+          LOG.debug(
+              "register succeed, metrics name is: " + key + " Type is :" + metrics.type().toString()
+                  + " tag is: " + Arrays.toString(labelNames)
+                  + " tagValue is " + Arrays.toString(labelValues)
+                  + " Value is : " + metrics.value().toString());
+        }
+      }
+      pg.push(registry, jobName, groupingKey);
+      LOG.info("pushing succeed");
+    } catch (Exception e) {
+      LOG.error("pushing job's metrics to gateway is failed ", e);
+    }
+  }
+
+  /**
+   * Convert CamelCase based names to lower-case names where the separator is the underscore, to
+   * follow prometheus naming conventions.
+   */
+  public String getMetricsName(String recordName,
+      String metricName) {
+    String baseName = StringUtils.capitalize(recordName)
+        + StringUtils.capitalize(metricName);
+    baseName = baseName.replace('-', '_');
+    String[] parts = SPLIT_PATTERN.split(baseName);
+    return String.join("_", parts).toLowerCase();
+  }
+
+  Map<String, String> parseGroupingKey(final String groupingKeyConfig) {
+    if (!groupingKeyConfig.isEmpty()) {
+      Map<String, String> groupingKey = new HashMap<>();
+      String[] kvs = groupingKeyConfig.split(";");
+      for (String kv : kvs) {
+        int idx = kv.indexOf("=");
+        if (idx < 0) {
+          LOG.warn("Invalid prometheusPushGateway groupingKey:" + kv + ", will be ignored");
+          continue;
+        }
+
+        String labelKey = kv.substring(0, idx);
+        String labelValue = kv.substring(idx + 1);
+        if (StringUtils.isEmpty(labelKey) || StringUtils.isEmpty(labelValue)) {
+          LOG.warn("Invalid groupingKey labelKey:" + labelKey + ", labelValue:" + labelValue
+              + " must not be empty");
+          continue;
+        }
+        groupingKey.put(labelKey, labelValue);
+      }
+      return groupingKey;
+    }
+    return Collections.emptyMap();
+  }
+
+  @Override
+  public void flush() {
+
+  }
+
+  @Override
+  public void close() throws IOException {
 
 Review comment:
   shouldn't you shut down push gateway here?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on issue #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on issue #1787: [HADOOP-16783] Exports Hadoop metrics to Prometheus PushGateWay
URL: https://github.com/apache/hadoop/pull/1787#issuecomment-569938254
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |  25m  0s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |   1m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  18m  9s |  trunk passed  |
   | +1 :green_heart: |  compile  |  16m 36s |  trunk passed  |
   | +1 :green_heart: |  checkstyle  |   2m 37s |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m 57s |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  18m 24s |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m  7s |  trunk passed  |
   | +0 :ok: |  spotbugs  |   2m  3s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +0 :ok: |  findbugs  |   0m 33s |  branch/hadoop-project no findbugs output file (findbugsXml.xml)  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 29s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m  1s |  the patch passed  |
   | +1 :green_heart: |  compile  |  15m 58s |  the patch passed  |
   | +1 :green_heart: |  javac  |  15m 58s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 38s |  root: The patch generated 8 new + 50 unchanged - 0 fixed = 58 total (was 50)  |
   | +1 :green_heart: |  mvnsite  |   1m 58s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  3s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedclient  |  12m 49s |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m  8s |  the patch passed  |
   | +0 :ok: |  findbugs  |   0m 33s |  hadoop-project has no data from findbugs  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 31s |  hadoop-project in the patch passed.  |
   | +1 :green_heart: |  unit  |   9m  6s |  hadoop-common in the patch passed.  |
   | -1 :x: |  asflicense  |   0m 54s |  The patch generated 1 ASF License warnings.  |
   |  |   | 139m 39s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1787 |
   | JIRA Issue | HADOOP-16783 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient xml findbugs checkstyle |
   | uname | Linux 5d2b5c273cbb 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / 6242391 |
   | Default Java | 1.8.0_232 |
   | checkstyle | https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/1/artifact/out/diff-checkstyle-root.txt |
   |  Test Results | https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/1/testReport/ |
   | asflicense | https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/1/artifact/out/patch-asflicense-problems.txt |
   | Max. process+thread count | 1446 (vs. ulimit of 5500) |
   | modules | C: hadoop-project hadoop-common-project/hadoop-common U: . |
   | Console output | https://builds.apache.org/job/hadoop-multibranch/job/PR-1787/1/console |
   | versions | git=2.7.4 maven=3.3.9 findbugs=3.1.0-RC1 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org