You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2021/07/02 15:47:55 UTC

[GitHub] [accumulo] DomGarguilo opened a new pull request #2188: WIP: Add micrometer metrics implementation

DomGarguilo opened a new pull request #2188:
URL: https://github.com/apache/accumulo/pull/2188


   This WIP PR serves to track changes made toward implementing micrometer as an alternative metrics system.
   
   Fixes #1134


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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] EdColeman edited a comment on pull request #2188: WIP: Add micrometer metrics implementation

Posted by GitBox <gi...@apache.org>.
EdColeman edited a comment on pull request #2188:
URL: https://github.com/apache/accumulo/pull/2188#issuecomment-883595649


   A follow-on task may be to implement a bridge between micrometer and the hadoop metrics system so that, if a hadoop exporter was present, it would read metrics from micrometer and then publish to hadoop configured sinks.
   
   The basic concept would be to find / extend a micrometer registry (see https://micrometer.io/docs/guide/customMeterRegistry) that would instantiate a hadoop metrics system (see System design - https://blog.cloudera.com/what-is-hadoop-metrics2/ ) The custom micrometer registry would function as the "metrics sources" and the hadoop metrics system would configure the sinks by reading the hadoop.metrics.xxx.properties file.  So when the hadoop metrics system "read" metrics from the sources, the values would be obtained via the micrometer metrics registry.  The goal would be that the Accumulo would only have micrometer Meters (gauges / counters,...) and if a hadoop metrics exporter was configured, it would read from the micrometer registry to get those values.  
   
   I'm not sure that the hadoop metrics would be compatible with a StepMeterRegistry - it might require the more general MeterRegistry.  Using a StepMeterRegistry might be enough to figure out some of the concepts - and then find an existing registry (etsy statsd, datadog,...) that matches the hadoop metrics system dimensionality (hierarchical / dimensional), rate aggregation (client / server) and publication (pull / push)  My impression is that hadoop would be hierarchical, client and pull) but that's making some guesses and needs to be confirmed.


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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] foster33 commented on a change in pull request #2188: WIP: Add micrometer metrics implementation

Posted by GitBox <gi...@apache.org>.
foster33 commented on a change in pull request #2188:
URL: https://github.com/apache/accumulo/pull/2188#discussion_r669545672



##########
File path: core/src/main/java/org/apache/accumulo/core/conf/Property.java
##########
@@ -243,7 +243,13 @@
   GENERAL_MAX_SCANNER_RETRY_PERIOD("general.max.scanner.retry.period", "5s",
       PropertyType.TIMEDURATION,
       "The maximum amount of time that a Scanner should wait before retrying a failed RPC"),
-
+  // experimental - micrometer properties
+  @Experimental
+  GENERAL_METRICS_CONFIGURATION_PROPERTIES_FILE("general.metrics.configuration.properties",
+      "accumulo.metrics.configuration.properties", PropertyType.STRING,
+      "The Accumulo metrics configuration properties filename. The file is used to"
+          + " configure micrometer MeterRegistries that loaded at runtime using classes that"
+          + " extend MetricsServiceLoader.class"),

Review comment:
       After the most recent changes in #2190, properties will need `since` information. I figured I would leave a comment regarding it.
   
   ```suggestion
     @Experimental
     GENERAL_METRICS_CONFIGURATION_PROPERTIES_FILE("general.metrics.configuration.properties",
         "accumulo.metrics.configuration.properties", PropertyType.STRING,
         "The Accumulo metrics configuration properties filename. The file is used to"
             + " configure micrometer MeterRegistries that loaded at runtime using classes that"
             + " extend MetricsServiceLoader.class", "2.1.0"),
   ```




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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] DomGarguilo commented on a change in pull request #2188: WIP: Add micrometer metrics implementation

Posted by GitBox <gi...@apache.org>.
DomGarguilo commented on a change in pull request #2188:
URL: https://github.com/apache/accumulo/pull/2188#discussion_r669624714



##########
File path: core/src/main/java/org/apache/accumulo/core/conf/Property.java
##########
@@ -243,7 +243,13 @@
   GENERAL_MAX_SCANNER_RETRY_PERIOD("general.max.scanner.retry.period", "5s",
       PropertyType.TIMEDURATION,
       "The maximum amount of time that a Scanner should wait before retrying a failed RPC"),
-
+  // experimental - micrometer properties
+  @Experimental
+  GENERAL_METRICS_CONFIGURATION_PROPERTIES_FILE("general.metrics.configuration.properties",
+      "accumulo.metrics.configuration.properties", PropertyType.STRING,
+      "The Accumulo metrics configuration properties filename. The file is used to"
+          + " configure micrometer MeterRegistries that loaded at runtime using classes that"
+          + " extend MetricsServiceLoader.class"),

Review comment:
       thanks!




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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] DomGarguilo commented on pull request #2188: WIP: Add micrometer metrics implementation

Posted by GitBox <gi...@apache.org>.
DomGarguilo commented on pull request #2188:
URL: https://github.com/apache/accumulo/pull/2188#issuecomment-937838453


   Superseded by #2305 


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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] DomGarguilo commented on pull request #2188: WIP: Add micrometer metrics implementation

Posted by GitBox <gi...@apache.org>.
DomGarguilo commented on pull request #2188:
URL: https://github.com/apache/accumulo/pull/2188#issuecomment-875927752


   I marked this as ready for review so I could start getting feedback on these changes. At this point I could use some suggestions on next steps or other improvements.


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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] DomGarguilo commented on pull request #2188: WIP: Add micrometer metrics implementation

Posted by GitBox <gi...@apache.org>.
DomGarguilo commented on pull request #2188:
URL: https://github.com/apache/accumulo/pull/2188#issuecomment-937838453


   Superseded by #2305 


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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] dlmarion commented on a change in pull request #2188: WIP: Add micrometer metrics implementation

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2188:
URL: https://github.com/apache/accumulo/pull/2188#discussion_r710013279



##########
File path: pom.xml
##########
@@ -243,6 +244,16 @@
         <artifactId>commons-logging</artifactId>
         <version>1.2</version>
       </dependency>
+      <dependency>
+        <groupId>io.micrometer</groupId>
+        <artifactId>micrometer-core</artifactId>
+        <version>${micrometer.metrics.version}</version>
+      </dependency>
+      <dependency>

Review comment:
       I'm thinking that the default metrics implementation would do nothing. If someone wants to use Prometheus (or logging for that matter), then they would drop the appropriate jar file (and configuration information) on the classpath. These implementations could then be moved to another repo (accumulo-metrics?)




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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] dlmarion commented on a change in pull request #2188: WIP: Add micrometer metrics implementation

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2188:
URL: https://github.com/apache/accumulo/pull/2188#discussion_r710021766



##########
File path: server/base/src/main/java/org/apache/accumulo/server/metrics/Metrics.java
##########
@@ -18,6 +18,9 @@
  */
 package org.apache.accumulo.server.metrics;
 
+import java.util.ServiceLoader;
+
+import org.apache.accumulo.server.metrics.service.MetricsRegistrationService;
 import org.apache.hadoop.metrics2.MetricsCollector;

Review comment:
       I'm of the opinion that the Hadoop Metrics2 stuff should be ripped out. If I'm understanding this correctly, we are publishing metrics via Micrometer and Hadoop Metrics concurrently. If both are configured to send metrics to the same destination, then we will be double reporting the numbers. IMO, Hadoop can send it's metrics via mechanism M and Micrometer can send it's metrics via mechanism M if the user so chooses.




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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] DomGarguilo commented on pull request #2188: WIP: Add micrometer metrics implementation

Posted by GitBox <gi...@apache.org>.
DomGarguilo commented on pull request #2188:
URL: https://github.com/apache/accumulo/pull/2188#issuecomment-890066855


   > A follow-on task may be to implement a bridge between micrometer and the hadoop metrics system
   
   @EdColeman, do you think this should wait until after these changes are made or incorporated in some way into these changes?


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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] EdColeman commented on pull request #2188: WIP: Add micrometer metrics implementation

Posted by GitBox <gi...@apache.org>.
EdColeman commented on pull request #2188:
URL: https://github.com/apache/accumulo/pull/2188#issuecomment-883595649


   A follow task may be to implement a bridge between micrometer and the hadoop metrics system so that, if a hadoop exporter was present, it would read metrics from micrometer and then publish to hadoop configured sinks.
   
   The basic concept would be to find / extend a micrometer registry (see https://micrometer.io/docs/guide/customMeterRegistry) that would instantiate a hadoop metrics system (see System design - https://blog.cloudera.com/what-is-hadoop-metrics2/ ) The custom micrometer registry would function as the "metrics sources" and the hadoop metrics system would configure the sinks by reading the hadoop.metrics.xxx.properties file.  So when the hadoop metrics system "read" metrics from the sources, the values would be obtained via the micrometer metrics registry.  The goal would be that the Accumulo would only have micrometer Meters (gauges / counters,...) and if a hadoop metrics exporter was configured, it would read from the micrometer registry to get those values.  
   
   I'm not sure that the hadoop metrics would be compatible with a StepMeterRegistry - it might require the more general MeterRegistry.  Using a StepMeterRegistry might be enough to figure out some of the concepts - and then find an existing registry (etsy statsd, datadog,...) that matches the hadoop metrics system dimensionality (hierarchical / dimensional), rate aggregation (client / server) and publication (pull / push)  My impression is that hadoop would be hierarchical, client and pull) but that's making some guesses and needs to be confirmed.


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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] EdColeman commented on pull request #2188: WIP: Add micrometer metrics implementation

Posted by GitBox <gi...@apache.org>.
EdColeman commented on pull request #2188:
URL: https://github.com/apache/accumulo/pull/2188#issuecomment-890074825


   These changes would have more impact if the hadoop metrics could be removed from the code base - and to do that it, it would ease transitions if there was a way to do the translation for users.  If the translation happens where the metrics are collected / reported, then that would be the only place where the hadoop metrics classes are used / needed.


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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] DomGarguilo closed pull request #2188: WIP: Add micrometer metrics implementation

Posted by GitBox <gi...@apache.org>.
DomGarguilo closed pull request #2188:
URL: https://github.com/apache/accumulo/pull/2188


   


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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] DomGarguilo closed pull request #2188: WIP: Add micrometer metrics implementation

Posted by GitBox <gi...@apache.org>.
DomGarguilo closed pull request #2188:
URL: https://github.com/apache/accumulo/pull/2188


   


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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] dlmarion commented on a change in pull request #2188: WIP: Add micrometer metrics implementation

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2188:
URL: https://github.com/apache/accumulo/pull/2188#discussion_r710037575



##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/CompactionExecutorsMetrics.java
##########
@@ -24,115 +24,231 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.IntSupplier;
 import java.util.function.Supplier;
 
 import org.apache.accumulo.core.spi.compaction.CompactionExecutorId;
+import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.tserver.compactions.CompactionManager.ExtCompMetric;
 import org.apache.hadoop.metrics2.lib.MetricsRegistry;
 import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
 
 import com.google.common.collect.Sets;
 
+import io.micrometer.core.instrument.MeterRegistry;
+
 public class CompactionExecutorsMetrics extends TServerMetrics {
 
-  private volatile List<CeMetrics> ceml = List.of();
-  private Map<CompactionExecutorId,CeMetrics> metrics = new HashMap<>();
-  private Map<CompactionExecutorId,ExMetrics> exMetrics = new HashMap<>();
   private volatile Supplier<Collection<ExtCompMetric>> externalMetricsSupplier;
 
+  private volatile List<CeMetrics> ceMetricsList = List.of();
+  private final Map<CompactionExecutorId,CeMetrics> ceMetricsMap = new HashMap<>();
+  private final Map<CompactionExecutorId,ExMetrics> exCeMetricsMap = new HashMap<>();
+  private final MeterRegistry micrometerRegistry;
+
+  CompactionExecutorsMetricsHadoop hadoopMetrics;
+
   private static class CeMetrics {
-    MutableGaugeInt queuedGauge;
-    MutableGaugeInt runningGauge;
+    AtomicInteger queued;
+    AtomicInteger running;
 
     IntSupplier runningSupplier;
     IntSupplier queuedSupplier;
   }
 
   private static class ExMetrics {
-    MutableGaugeInt queuedGauge;
-    MutableGaugeInt runningGauge;
-
+    AtomicInteger queued;
+    AtomicInteger running;
   }
 
-  public CompactionExecutorsMetrics() {
+  public CompactionExecutorsMetrics(MeterRegistry registryMicrometer) {
     super("compactionExecutors");
+    this.micrometerRegistry = registryMicrometer;
+
+    ScheduledExecutorService scheduler =
+        ThreadPools.createScheduledExecutorService(1, "compactionExecutorsMetricsPoller", false);
+    Runtime.getRuntime().addShutdownHook(new Thread(scheduler::shutdownNow));

Review comment:
       Just like the `ThreadPools` class is for creating pools, there is a `Threads` class for a common way to create a Thread.




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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] EdColeman commented on a change in pull request #2188: WIP: Add micrometer metrics implementation

Posted by GitBox <gi...@apache.org>.
EdColeman commented on a change in pull request #2188:
URL: https://github.com/apache/accumulo/pull/2188#discussion_r710181041



##########
File path: server/base/src/main/java/org/apache/accumulo/server/metrics/service/MicrometerMetricsFactory.java
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.accumulo.server.metrics.service;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.time.Duration;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.ServiceLoader;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.server.ServerContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.micrometer.core.instrument.Meter;
+import io.micrometer.core.instrument.MeterRegistry;
+import io.micrometer.core.instrument.composite.CompositeMeterRegistry;
+import io.micrometer.core.instrument.config.MeterFilter;
+import io.micrometer.core.instrument.distribution.DistributionStatisticConfig;
+
+public class MicrometerMetricsFactory {
+
+  private static final Pattern MATCH_ON_NAME =
+      Pattern.compile("metrics.service.(?<index>\\d+).name");
+
+  private static final Property METRICS_CONFIG_PROPERTIES =
+      Property.GENERAL_METRICS_CONFIGURATION_PROPERTIES_FILE;
+
+  private static final Logger log = LoggerFactory.getLogger(MicrometerMetricsFactory.class);
+  public static final String CALLING_SERVICE_NAME = "callingServiceName";
+
+  private final CompositeMeterRegistry registry;
+
+  private MicrometerMetricsFactory(final ServerContext context, final String appName) {
+
+    registry = new CompositeMeterRegistry();
+    // Configure replication metrics to display percentiles and change its expiry to 10 mins
+    registry.config().meterFilter(new MeterFilter() {
+      @Override
+      public DistributionStatisticConfig configure(Meter.Id id,
+          DistributionStatisticConfig config) {
+        if (id.getName().equals("replicationQueue")) {
+          return DistributionStatisticConfig.builder().percentiles(0.5, 0.75, 0.9, 0.95, 0.99)
+              .expiry(Duration.ofMinutes(10)).build().merge(config);
+        }
+        return config;
+      }
+    });
+
+    var propKey = METRICS_CONFIG_PROPERTIES.getKey();

Review comment:
       The intent was to show that we could provide a standard way to allow metrics configuration. This keeps us inline with what is currently provided with the current metrics.  
   
   Also, it seems desirable to allow some configuration flexibility - it may be easier for some users to deploy with a standard set of metrics available in the metrics registration and then configure them through a configuration file.
   
   That was the intent - not sure if this meets those goals or not...




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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] dlmarion commented on a change in pull request #2188: WIP: Add micrometer metrics implementation

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2188:
URL: https://github.com/apache/accumulo/pull/2188#discussion_r710011984



##########
File path: server/base/src/main/java/org/apache/accumulo/server/metrics/service/MicrometerMetricsFactory.java
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.accumulo.server.metrics.service;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.time.Duration;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.ServiceLoader;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.server.ServerContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.micrometer.core.instrument.Meter;
+import io.micrometer.core.instrument.MeterRegistry;
+import io.micrometer.core.instrument.composite.CompositeMeterRegistry;
+import io.micrometer.core.instrument.config.MeterFilter;
+import io.micrometer.core.instrument.distribution.DistributionStatisticConfig;
+
+public class MicrometerMetricsFactory {
+
+  private static final Pattern MATCH_ON_NAME =
+      Pattern.compile("metrics.service.(?<index>\\d+).name");
+
+  private static final Property METRICS_CONFIG_PROPERTIES =
+      Property.GENERAL_METRICS_CONFIGURATION_PROPERTIES_FILE;
+
+  private static final Logger log = LoggerFactory.getLogger(MicrometerMetricsFactory.class);
+  public static final String CALLING_SERVICE_NAME = "callingServiceName";
+
+  private final CompositeMeterRegistry registry;
+
+  private MicrometerMetricsFactory(final ServerContext context, final String appName) {
+
+    registry = new CompositeMeterRegistry();
+    // Configure replication metrics to display percentiles and change its expiry to 10 mins
+    registry.config().meterFilter(new MeterFilter() {
+      @Override
+      public DistributionStatisticConfig configure(Meter.Id id,
+          DistributionStatisticConfig config) {
+        if (id.getName().equals("replicationQueue")) {
+          return DistributionStatisticConfig.builder().percentiles(0.5, 0.75, 0.9, 0.95, 0.99)
+              .expiry(Duration.ofMinutes(10)).build().merge(config);
+        }
+        return config;
+      }
+    });
+
+    var propKey = METRICS_CONFIG_PROPERTIES.getKey();

Review comment:
       Is there a reason that we have to have a configuration file in Accumulo for these pluggable Registries? Can't implementations of the MetricsRegistrationService configure the Registry in its own way?




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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] dlmarion commented on a change in pull request #2188: WIP: Add micrometer metrics implementation

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2188:
URL: https://github.com/apache/accumulo/pull/2188#discussion_r710021766



##########
File path: server/base/src/main/java/org/apache/accumulo/server/metrics/Metrics.java
##########
@@ -18,6 +18,9 @@
  */
 package org.apache.accumulo.server.metrics;
 
+import java.util.ServiceLoader;
+
+import org.apache.accumulo.server.metrics.service.MetricsRegistrationService;
 import org.apache.hadoop.metrics2.MetricsCollector;

Review comment:
       I'm of the opinion that the Hadoop Metrics2 stuff should be ripped out. If I'm understanding this correctly, we are publishing metrics via Micrometer and Hadoop Metrics concurrently. If both are configured to send metrics to the same destination, then we will be double reporting the numbers. IMO, Hadoop can send it's metrics via mechanism M and Micrometer can send it's metrics via mechanism M if the user so chooses.
   
   Update: reading through the comments again I see that @EdColeman and I may have a difference of opinion 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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org