xBis7 commented on code in PR #5479:
URL: https://github.com/apache/ozone/pull/5479#discussion_r1406549126


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/ProtocolMessageMetrics.java:
##########
@@ -129,4 +168,8 @@ public String description() {
       return description;
     }
   }
+
+  public boolean isQuantileEnable() {

Review Comment:
   This doesn't seem used anywhere.



##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/ProtocolMessageMetrics.java:
##########
@@ -22,50 +22,80 @@
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.metrics2.MetricsCollector;
 import org.apache.hadoop.metrics2.MetricsInfo;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.MetricsSource;
 import org.apache.hadoop.metrics2.MetricsTag;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.lib.Interns;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MutableQuantiles;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.ratis.util.UncheckedAutoCloseable;
 
 /**
  * Metrics to count all the subtypes of a specific message.
  */
-public class ProtocolMessageMetrics<KEY> implements MetricsSource {
+public final class ProtocolMessageMetrics<KEY> implements MetricsSource {
 
   private final String name;
 
   private final String description;
 
+  private final boolean quantileEnable;
+
   private final Map<KEY, AtomicLong> counters =
       new ConcurrentHashMap<>();
 
   private final Map<KEY, AtomicLong> elapsedTimes =
       new ConcurrentHashMap<>();
 
+  private final Map<KEY, MutableQuantiles[]> quantiles =
+      new ConcurrentHashMap<>();
+
   private final AtomicInteger concurrency = new AtomicInteger(0);
 
   public static <KEY> ProtocolMessageMetrics<KEY> create(String name,
-      String description, KEY[] types) {
-    return new ProtocolMessageMetrics<KEY>(name, description, types);
+      String description, KEY[] types, ConfigurationSource conf) {
+    return new ProtocolMessageMetrics<KEY>(name, description, types, conf);
   }
 
-  public ProtocolMessageMetrics(String name, String description,
-      KEY[] values) {
+  private ProtocolMessageMetrics(String name, String description,
+      KEY[] values, ConfigurationSource conf) {
     this.name = name;
     this.description = description;
+    int[] intervals = conf.getInts(
+        OzoneConfigKeys.OZONE_PROTOCOL_MESSAGE_METRICS_PERCENTILES_INTERVALS);
+    quantileEnable = (intervals.length > 0);
     for (KEY value : values) {
       counters.put(value, new AtomicLong(0));
       elapsedTimes.put(value, new AtomicLong(0));
+      if (quantileEnable) {
+        MetricsRegistry registry =
+            new MetricsRegistry(value.toString() + "MessageMetrics");
+        MutableQuantiles[] mutableQuantiles =
+            new MutableQuantiles[intervals.length];
+        quantiles.put(value, mutableQuantiles);
+        for (int i = 0; i < intervals.length; i++) {
+          mutableQuantiles[i] = registry.newQuantiles(
+              intervals[i] + "s",
+              value.toString() + "rpc time in milli second",
+              "ops", "latencyMs", intervals[i]);
+        }
+      }
     }
   }
 
   public void increment(KEY key, long duration) {
     counters.get(key).incrementAndGet();
     elapsedTimes.get(key).addAndGet(duration);
+    if (quantileEnable) {
+      for (MutableQuantiles q : quantiles.get(key)) {

Review Comment:
   Since using a Map for tracking the quantiles and accessing them, shouldn't 
you also update the value on the map? You get the `MutableQuantiles` from the 
Map, update their value but don't put the new value back in the Map. 
   
   I haven't tested it locally but it seems that it will always read the same 
original value because it doesn't get updated on the map.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to