You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2020/06/05 05:55:42 UTC

[GitHub] [hbase] busbey commented on a change in pull request #1814: HBASE-20904 Prometheus /metrics http endpoint for monitoring

busbey commented on a change in pull request #1814:
URL: https://github.com/apache/hbase/pull/1814#discussion_r435690908



##########
File path: hbase-http/src/main/java/org/apache/hadoop/hbase/http/prom/PrometheusMetricsSink.java
##########
@@ -0,0 +1,103 @@
+/**
+ * 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.hbase.http.prom;
+
+import org.apache.commons.configuration2.SubsetConfiguration;
+import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricType;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.MetricsSink;
+import org.apache.hadoop.metrics2.MetricsTag;
+
+import java.io.IOException;
+import java.io.Writer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Metrics sink for prometheus exporter.
+ * <p>
+ * Stores the metric data in-memory and return with it on request.

Review comment:
       Include a description in here about what metrics we do or do not emit.

##########
File path: hbase-http/src/main/java/org/apache/hadoop/hbase/http/prom/PrometheusUtils.java
##########
@@ -0,0 +1,43 @@
+/**
+ * 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.hbase.http.prom;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import java.util.regex.Pattern;
+
+@InterfaceAudience.Private
+public class PrometheusUtils {
+
+  private static final Pattern SPLIT_PATTERN =
+    Pattern.compile("(?<!(^|[A-Z_]))(?=[A-Z])|(?<!^)(?=[A-Z][a-z])");
+
+  public static String toPrometheusName(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()
+            .replace('.', '_')
+            .replace('-', '_')

Review comment:
       we did a replacement above for this same pattern, why do we need to do it again?

##########
File path: hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestPrometheus2Servlet.java
##########
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.http;

Review comment:
       this test should be in the same package as the class it is testing.

##########
File path: hbase-http/src/main/java/org/apache/hadoop/hbase/http/prom/PrometheusHadoop2Servlet.java
##########
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <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.hbase.http.prom;
+
+import org.apache.hadoop.hbase.http.HttpServer;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.yetus.audience.InterfaceAudience;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+
+@InterfaceAudience.Private
+public class PrometheusHadoop2Servlet extends HttpServlet {
+
+  public PrometheusMetricsSink getPrometheusSink() {

Review comment:
       this method can be private? or just incorporated into doGet?

##########
File path: hbase-http/src/main/java/org/apache/hadoop/hbase/http/prom/PrometheusUtils.java
##########
@@ -0,0 +1,43 @@
+/**
+ * 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.hbase.http.prom;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import java.util.regex.Pattern;
+
+@InterfaceAudience.Private
+public class PrometheusUtils {
+
+  private static final Pattern SPLIT_PATTERN =
+    Pattern.compile("(?<!(^|[A-Z_]))(?=[A-Z])|(?<!^)(?=[A-Z][a-z])");
+
+  public static String toPrometheusName(String recordName, String metricName) {

Review comment:
       please add a javadoc that describes what this transformation is doing

##########
File path: hbase-common/src/main/resources/hbase-default.xml
##########
@@ -1727,6 +1727,16 @@ possible configurations would overwhelm and obscure the important.
       ThreadPool.
     </description>
   </property>
+  <property>
+    <name>hbase.http.enable.prometheus.servlets</name>

Review comment:
       To Stack's earlier point about how we have multiple metrics systems getting published, they have an overhead, and we want to avoid duplicative effort: normally if I was going to publish my metrics to prometheus would I still necessarily care about publishing to the `/jmx` HTTP endpoint?
   
   could we make this a list of which metrics servlets to include? something like `hbase.http.metrics.servlets` with possible values of `jmx`, `metrics`, `prometheus`, `prometheus-hadoop2`. then we could default to `jmx,metrics` to have the current default behavior and then folks could decide if they want to keep that when they turn on prometheus integration. it also makes it clear that the "old" version is specifically using the hadoop2 metrics and gives us a config to look for when it comes time to warn about deprecation.

##########
File path: hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestPrometheusServlet.java
##########
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.http;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintWriter;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.http.prom.PrometheusServlet;
+import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.hadoop.hbase.metrics.Histogram;
+import org.apache.hadoop.hbase.metrics.Meter;
+import org.apache.hadoop.hbase.metrics.MetricRegistries;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+import org.apache.hadoop.hbase.metrics.Timer;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ SmallTests.class, MiscTests.class })
+public class TestPrometheusServlet {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_TEST_RULE =
+    HBaseClassTestRule.forClass(TestPrometheusServlet.class);
+
+  PrometheusServlet ps;
+  MetricRegistry mr;
+
+  @Before
+  public void setup() {
+    ps = new PrometheusServlet();
+    MetricRegistryInfo mri = new MetricRegistryInfo(
+      "metricGroup",
+      "dummy metric for tests",
+      "",
+      "mctxt",
+      false);
+    mr = MetricRegistries.global().create(mri);
+  }
+
+  @Test
+  public void testPrometheusServlet() {
+
+    //Counters
+    Counter c1 = mr.counter("c1");
+    c1.increment();
+    test(ps, "metric_group_c1 1\n");
+    mr.remove("c1");
+
+    //Meters
+    Meter m1 = mr.meter("m1");
+    m1.mark(0);
+    test(ps, "metric_group_m1_count 0\n" + "metric_group_m1_mean_rate 0.0\n"
+      + "metric_group_m1_1min_rate 0.0\n" + "metric_group_m1_5min_rate 0.0\n"
+      + "metric_group_m1_15min_rate 0.0\n");
+    mr.remove("m1");
+
+    //Timers (don't update the timer)
+    Timer t1 = mr.timer("t1");
+    test(ps,
+      "metric_group_t1_count 0\n"
+      + "metric_group_t1_mean_rate 0.0\n"
+      + "metric_group_t1_1min_rate 0.0\n"
+      + "metric_group_t1_5min_rate 0.0\n"
+      + "metric_group_t1_15min_rate 0.0\n"
+      + "metric_group_t1_num_ops 0\n"
+      + "metric_group_t1_min 0\n"
+      + "metric_group_t1_max 0\n"
+      + "metric_group_t1_mean 0\n"
+      + "metric_group_t1_25th_percentile 0\n"
+      + "metric_group_t1_median 0\n"
+      + "metric_group_t1_75th_percentile 0\n"
+      + "metric_group_t1_90th_percentile 0\n"
+      + "metric_group_t1_95th_percentile 0\n"
+      + "metric_group_t1_98th_percentile 0\n"
+      + "metric_group_t1_99th_percentile 0\n"
+      + "metric_group_t1_99.9th_percentile 0\n");
+    mr.remove("t1");
+
+    //Histograms
+    Histogram h1 = mr.histogram("h1");
+    h1.update(0);
+    test(ps,
+      "metric_group_h1_num_ops 1\n"
+      + "metric_group_h1_min 0\n"
+      + "metric_group_h1_max 0\n"
+      + "metric_group_h1_mean 0\n"
+      + "metric_group_h1_25th_percentile 0\n"
+      + "metric_group_h1_median 0\n"
+      + "metric_group_h1_75th_percentile 0\n"
+      + "metric_group_h1_90th_percentile 0\n"
+      + "metric_group_h1_95th_percentile 0\n"
+      + "metric_group_h1_98th_percentile 0\n"
+      + "metric_group_h1_99th_percentile 0\n"
+      + "metric_group_h1_99.9th_percentile 0\n");
+    mr.remove("h1");
+
+  }
+
+  @Test
+  public void testStrings() {
+    testWithCounter(ps, "my.counter", "metric_group_my_counter 0\n");
+
+    testWithCounter(ps, "my-counter", "metric_group_my_counter 0\n");
+
+    testWithCounter(ps, "myCounter",  "metric_group_my_counter 0\n");
+
+    testWithCounter(ps, "my_Counter", "metric_group_my_counter 0\n");
+
+    testWithCounter(ps, "my__Counter","metric_group_my_counter 0\n");
+  }
+
+  private void testWithCounter(PrometheusServlet ps, String metricName, String expected) {
+    mr.counter(metricName);
+    test(ps, expected);
+    mr.remove(metricName);
+  }
+
+  private void test(PrometheusServlet ps, String expected) {
+    ByteArrayOutputStream bos = new ByteArrayOutputStream();
+    PrintWriter pw = new PrintWriter(bos);
+    ps.writeMetrics(MetricRegistries.global().getMetricRegistries(), pw);
+    String out = bos.toString();
+    assert out.equals(expected) : String.format("Expected [%s] but result is [%s]", expected, out);

Review comment:
       use a junit `Assert.assertEquals` call here and not a a java assert.

##########
File path: hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
##########
@@ -769,6 +777,16 @@ protected void addDefaultServlets(
       LOG.info("ASYNC_PROFILER_HOME environment variable and async.profiler.home system property " +
         "not specified. Disabling /prof endpoint.");
     }
+
+    if(conf.getBoolean(ENABLE_PROMETHEUS_SERVLETS_CONF_KEY, ENABLE_PROMETHEUS_SERVLETS_DEFAULT)) {
+      PrometheusMetricsSink prometheusMetricsSink = new PrometheusMetricsSink();
+      setAttribute(PROMETHEUS_SINK, prometheusMetricsSink);
+      DefaultMetricsSystem.instance()
+        .register("prometheus", "Hadoop metrics prometheus exporter", prometheusMetricsSink);
+
+      addPrivilegedServlet("prometheus", "/prom", PrometheusServlet.class);

Review comment:
       is there a reason we can't use "/prometheus" for this end point? or some specific advantage to using a shorter name?

##########
File path: hbase-http/src/main/java/org/apache/hadoop/hbase/http/prom/PrometheusMetricsSink.java
##########
@@ -0,0 +1,103 @@
+/**
+ * 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.hbase.http.prom;
+
+import org.apache.commons.configuration2.SubsetConfiguration;
+import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricType;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.MetricsSink;
+import org.apache.hadoop.metrics2.MetricsTag;
+
+import java.io.IOException;
+import java.io.Writer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Metrics sink for prometheus exporter.
+ * <p>
+ * Stores the metric data in-memory and return with it on request.
+ */
+@InterfaceAudience.Private
+public class PrometheusMetricsSink implements MetricsSink {
+
+  /**
+   * Cached output lines for each metrics.
+   */
+  private Map<String, String> metricLines = new HashMap<>();

Review comment:
       when do we ever remove entries from in here?
   
   at a minimum implement Closeable and clear this out.

##########
File path: hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
##########
@@ -769,6 +777,16 @@ protected void addDefaultServlets(
       LOG.info("ASYNC_PROFILER_HOME environment variable and async.profiler.home system property " +
         "not specified. Disabling /prof endpoint.");
     }
+
+    if(conf.getBoolean(ENABLE_PROMETHEUS_SERVLETS_CONF_KEY, ENABLE_PROMETHEUS_SERVLETS_DEFAULT)) {
+      PrometheusMetricsSink prometheusMetricsSink = new PrometheusMetricsSink();
+      setAttribute(PROMETHEUS_SINK, prometheusMetricsSink);
+      DefaultMetricsSystem.instance()
+        .register("prometheus", "Hadoop metrics prometheus exporter", prometheusMetricsSink);
+
+      addPrivilegedServlet("prometheus", "/prom", PrometheusServlet.class);
+      addPrivilegedServlet("prometheus2", "/prom-old", PrometheusHadoop2Servlet.class);

Review comment:
       please use a more descriptive name and end point. e.g. "prometheus-hadoop2"

##########
File path: hbase-http/src/main/java/org/apache/hadoop/hbase/http/prom/PrometheusMetricsSink.java
##########
@@ -0,0 +1,103 @@
+/**
+ * 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.hbase.http.prom;
+
+import org.apache.commons.configuration2.SubsetConfiguration;
+import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricType;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.MetricsSink;
+import org.apache.hadoop.metrics2.MetricsTag;
+
+import java.io.IOException;
+import java.io.Writer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Metrics sink for prometheus exporter.
+ * <p>
+ * Stores the metric data in-memory and return with it on request.
+ */
+@InterfaceAudience.Private
+public class PrometheusMetricsSink implements MetricsSink {
+
+  /**
+   * Cached output lines for each metrics.
+   */
+  private Map<String, String> metricLines = new HashMap<>();
+
+  private static final Pattern SPLIT_PATTERN =

Review comment:
       not used, please remove?

##########
File path: hbase-http/src/main/java/org/apache/hadoop/hbase/http/prom/PrometheusMetricsSink.java
##########
@@ -0,0 +1,103 @@
+/**
+ * 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.hbase.http.prom;
+
+import org.apache.commons.configuration2.SubsetConfiguration;
+import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricType;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.MetricsSink;
+import org.apache.hadoop.metrics2.MetricsTag;
+
+import java.io.IOException;
+import java.io.Writer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Metrics sink for prometheus exporter.
+ * <p>
+ * Stores the metric data in-memory and return with it on request.
+ */
+@InterfaceAudience.Private
+public class PrometheusMetricsSink implements MetricsSink {

Review comment:
       I think this class is supposed to be thread safe? could you either confirm that it isn't or safely access metricLines?

##########
File path: hbase-http/src/main/java/org/apache/hadoop/hbase/http/prom/PrometheusHadoop2Servlet.java
##########
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <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.hbase.http.prom;

Review comment:
       please use `org.apache.hadoop.hbase.http.prometheus`

##########
File path: hbase-http/src/main/java/org/apache/hadoop/hbase/http/prom/PrometheusHadoop2Servlet.java
##########
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>

Review comment:
       please correct all of the license headers added in this PR to consistently match the expected format. note that there should be no HTML paragraph markers and the license header should _not_ be a javadoc comment.
   
   Here is an example of a file that has a correctly formatted license header:
   https://github.com/apache/hbase/blob/rel/2.2.5/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java

##########
File path: hbase-http/src/main/java/org/apache/hadoop/hbase/http/prom/PrometheusServlet.java
##########
@@ -0,0 +1,148 @@
+/**
+ * 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.hbase.http.prom;
+
+import static org.apache.hadoop.hbase.http.prom.PrometheusUtils.toPrometheusName;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.hadoop.hbase.metrics.Gauge;
+import org.apache.hadoop.hbase.metrics.Histogram;
+import org.apache.hadoop.hbase.metrics.Meter;
+import org.apache.hadoop.hbase.metrics.Metric;
+import org.apache.hadoop.hbase.metrics.MetricRegistries;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.Snapshot;
+import org.apache.hadoop.hbase.metrics.Timer;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class PrometheusServlet extends HttpServlet {
+
+  //Strings used to create metrics names.
+  String NUM_OPS_METRIC_NAME = "_num_ops";
+  String MIN_METRIC_NAME = "_min";
+  String MAX_METRIC_NAME = "_max";
+  String MEAN_METRIC_NAME = "_mean";
+  String MEDIAN_METRIC_NAME = "_median";
+  String TWENTY_FIFTH_PERCENTILE_METRIC_NAME = "_25th_percentile";
+  String SEVENTY_FIFTH_PERCENTILE_METRIC_NAME = "_75th_percentile";
+  String NINETIETH_PERCENTILE_METRIC_NAME = "_90th_percentile";
+  String NINETY_FIFTH_PERCENTILE_METRIC_NAME = "_95th_percentile";
+  String NINETY_EIGHTH_PERCENTILE_METRIC_NAME = "_98th_percentile";
+  String NINETY_NINETH_PERCENTILE_METRIC_NAME = "_99th_percentile";
+  String NINETY_NINE_POINT_NINETH_PERCENTILE_METRIC_NAME = "_99.9th_percentile";
+
+  Logger LOG = LoggerFactory.getLogger(PrometheusServlet.class);
+
+  @Override protected void doGet(HttpServletRequest req, HttpServletResponse resp)

Review comment:
       please put `@Override` annotation on its own line.

##########
File path: hbase-http/src/main/java/org/apache/hadoop/hbase/http/prom/PrometheusServlet.java
##########
@@ -0,0 +1,148 @@
+/**
+ * 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.hbase.http.prom;
+
+import static org.apache.hadoop.hbase.http.prom.PrometheusUtils.toPrometheusName;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.hadoop.hbase.metrics.Gauge;
+import org.apache.hadoop.hbase.metrics.Histogram;
+import org.apache.hadoop.hbase.metrics.Meter;
+import org.apache.hadoop.hbase.metrics.Metric;
+import org.apache.hadoop.hbase.metrics.MetricRegistries;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.Snapshot;
+import org.apache.hadoop.hbase.metrics.Timer;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class PrometheusServlet extends HttpServlet {
+
+  //Strings used to create metrics names.
+  String NUM_OPS_METRIC_NAME = "_num_ops";
+  String MIN_METRIC_NAME = "_min";
+  String MAX_METRIC_NAME = "_max";
+  String MEAN_METRIC_NAME = "_mean";
+  String MEDIAN_METRIC_NAME = "_median";
+  String TWENTY_FIFTH_PERCENTILE_METRIC_NAME = "_25th_percentile";
+  String SEVENTY_FIFTH_PERCENTILE_METRIC_NAME = "_75th_percentile";
+  String NINETIETH_PERCENTILE_METRIC_NAME = "_90th_percentile";
+  String NINETY_FIFTH_PERCENTILE_METRIC_NAME = "_95th_percentile";
+  String NINETY_EIGHTH_PERCENTILE_METRIC_NAME = "_98th_percentile";
+  String NINETY_NINETH_PERCENTILE_METRIC_NAME = "_99th_percentile";
+  String NINETY_NINE_POINT_NINETH_PERCENTILE_METRIC_NAME = "_99.9th_percentile";
+
+  Logger LOG = LoggerFactory.getLogger(PrometheusServlet.class);
+
+  @Override protected void doGet(HttpServletRequest req, HttpServletResponse resp)
+    throws IOException {
+
+    final PrintWriter op = resp.getWriter();
+
+    writeMetrics(MetricRegistries.global().getMetricRegistries(), op);
+  }
+
+  public void writeMetrics(Collection<MetricRegistry> metricRegistries, PrintWriter pw) {

Review comment:
       note that this method exists for testing.  also make it package private.

##########
File path: hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestPrometheus2Servlet.java
##########
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.http;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.hbase.http.prom.PrometheusUtils.toPrometheusName;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.http.prom.PrometheusMetricsSink;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test prometheus Sink.
+ */
+@Category({ SmallTests.class, MiscTests.class })
+public class TestPrometheus2Servlet {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_TEST_RULE =
+    HBaseClassTestRule.forClass(TestPrometheus2Servlet.class);
+
+  @Test
+  public void testPublish() throws IOException {
+    //GIVEN
+    MetricsSystem metrics = DefaultMetricsSystem.instance();
+
+    metrics.init("test");
+    PrometheusMetricsSink sink = new PrometheusMetricsSink();
+    metrics.register("Prometheus", "Prometheus", sink);
+    TestMetrics testMetrics = metrics.register("TestMetrics", "Testing metrics", new TestMetrics());
+
+    metrics.start();
+    testMetrics.numBucketCreateFails.incr();
+    metrics.publishMetricsNow();
+    ByteArrayOutputStream stream = new ByteArrayOutputStream();
+    OutputStreamWriter writer = new OutputStreamWriter(stream, UTF_8);
+
+    //WHEN
+    sink.writeMetrics(writer);
+    writer.flush();
+
+    //THEN
+    String writtenMetrics = stream.toString(UTF_8.name());
+    System.out.println(writtenMetrics);

Review comment:
       use a slf4j logger and not System.out.println.

##########
File path: hbase-http/src/main/java/org/apache/hadoop/hbase/http/prom/PrometheusMetricsSink.java
##########
@@ -0,0 +1,103 @@
+/**
+ * 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.hbase.http.prom;
+
+import org.apache.commons.configuration2.SubsetConfiguration;
+import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricType;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.MetricsSink;
+import org.apache.hadoop.metrics2.MetricsTag;
+
+import java.io.IOException;
+import java.io.Writer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Metrics sink for prometheus exporter.
+ * <p>
+ * Stores the metric data in-memory and return with it on request.
+ */
+@InterfaceAudience.Private
+public class PrometheusMetricsSink implements MetricsSink {
+
+  /**
+   * Cached output lines for each metrics.
+   */
+  private Map<String, String> metricLines = new HashMap<>();
+
+  private static final Pattern SPLIT_PATTERN =
+    Pattern.compile("(?<!(^|[A-Z_]))(?=[A-Z])|(?<!^)(?=[A-Z][a-z])");
+
+  public PrometheusMetricsSink() {
+  }
+
+  @Override
+  public void putMetrics(MetricsRecord metricsRecord) {
+    for (AbstractMetric metrics : metricsRecord.metrics()) {
+      if (metrics.type() == MetricType.COUNTER
+        || metrics.type() == MetricType.GAUGE) {
+
+        String key = PrometheusUtils.toPrometheusName(
+          metricsRecord.name(), metrics.name());
+
+        StringBuilder builder = new StringBuilder();
+        builder.append("# TYPE " + key + " " +
+          metrics.type().toString().toLowerCase() + "\n");

Review comment:
       use multiple `builder.append` calls instead of string concatenation that is then passed to `builder.append`

##########
File path: hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestPrometheusServlet.java
##########
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.http;

Review comment:
       this test should be in the same package as the class it is testing.

##########
File path: hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestPrometheus2Servlet.java
##########
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.http;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.hbase.http.prom.PrometheusUtils.toPrometheusName;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.http.prom.PrometheusMetricsSink;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test prometheus Sink.
+ */
+@Category({ SmallTests.class, MiscTests.class })
+public class TestPrometheus2Servlet {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_TEST_RULE =
+    HBaseClassTestRule.forClass(TestPrometheus2Servlet.class);
+
+  @Test
+  public void testPublish() throws IOException {
+    //GIVEN
+    MetricsSystem metrics = DefaultMetricsSystem.instance();
+
+    metrics.init("test");
+    PrometheusMetricsSink sink = new PrometheusMetricsSink();
+    metrics.register("Prometheus", "Prometheus", sink);
+    TestMetrics testMetrics = metrics.register("TestMetrics", "Testing metrics", new TestMetrics());
+
+    metrics.start();
+    testMetrics.numBucketCreateFails.incr();
+    metrics.publishMetricsNow();
+    ByteArrayOutputStream stream = new ByteArrayOutputStream();
+    OutputStreamWriter writer = new OutputStreamWriter(stream, UTF_8);
+
+    //WHEN
+    sink.writeMetrics(writer);
+    writer.flush();
+
+    //THEN
+    String writtenMetrics = stream.toString(UTF_8.name());
+    System.out.println(writtenMetrics);
+    Assert.assertTrue("The expected metric line is missing from prometheus metrics output",
+      writtenMetrics.contains("test_metrics_num_bucket_create_fails{context=\"dfs\""));
+
+    metrics.stop();
+    metrics.shutdown();
+  }
+
+  @Test
+  public void testNamingCamelCase() {
+    PrometheusMetricsSink sink = new PrometheusMetricsSink();
+
+    Assert.assertEquals("rpc_time_some_metrics", toPrometheusName("RpcTime", "SomeMetrics"));

Review comment:
       move testing of the `toPrometheusName` into a test class that is specific to the utility class.




----------------------------------------------------------------
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