suizhe007 commented on code in PR #2089:
URL: 
https://github.com/apache/incubator-celeborn/pull/2089#discussion_r1405838649


##########
common/src/main/scala/org/apache/celeborn/common/metrics/sink/JsonServlet.scala:
##########
@@ -0,0 +1,316 @@
+/*
+ * 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.celeborn.common.metrics.sink
+
+import java.util
+import java.util.Properties
+import java.util.concurrent.ConcurrentLinkedQueue
+
+import scala.beans.BeanProperty
+
+import com.codahale.metrics.MetricRegistry
+import io.netty.channel.ChannelHandler.Sharable
+import 
org.apache.hadoop.shaded.com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility
+import 
org.apache.hadoop.shaded.com.fasterxml.jackson.annotation.PropertyAccessor
+import org.apache.hadoop.shaded.com.fasterxml.jackson.databind.ObjectMapper
+
+import org.apache.celeborn.common.metrics.{CelebornHistogram, CelebornTimer, 
ResettableSlidingWindowReservoir}
+import org.apache.celeborn.common.metrics.source.{AbstractSource, 
NamedCounter, NamedGauge, NamedHistogram, NamedTimer, Source}
+
+class MetricData(
+    @BeanProperty var name: String,
+    @BeanProperty var value: Any,
+    @BeanProperty var timestampMs: Long,
+    @BeanProperty var labelNames: util.List[String],
+    @BeanProperty var labelValues: util.List[String]) {}
+
+class JsonServlet(
+    val property: Properties,
+    val registry: MetricRegistry,
+    val sources: Seq[Source],
+    val servletPath: String) extends AbstractServlet(sources) {
+
+  val innerMetrics: ConcurrentLinkedQueue[MetricData] = new 
ConcurrentLinkedQueue[MetricData]
+
+  val mapper = new ObjectMapper()
+  mapper.setVisibility(PropertyAccessor.FIELD, Visibility.ANY)
+
+  override def getMetricsSnapshot: String = {
+    val metricDatas = new util.ArrayList[MetricData]
+    try {
+      sources.map(source => metricDatas.addAll(getMetrics(source)))
+      mapper.writeValueAsString(metricDatas)
+    } catch {
+      case e: Throwable =>
+        logError("failed to get json data for metrics", e)
+        mapper.writeValueAsString(new util.ArrayList[MetricData])
+    }
+  }
+
+  override def createHttpRequestHandler(): ServletHttpRequestHandler = {
+    new JsonHttpRequestHandler(servletPath, this)
+  }
+
+  override def stop(): Unit = {
+    innerMetrics.clear()
+  }
+
+  def getMetrics(source: Source): util.List[MetricData] = {
+    val absSource = source.asInstanceOf[AbstractSource]
+    absSource.counters().foreach(c => recordCounter(absSource, c))
+    absSource.gauges().foreach(g => recordGauge(absSource, g))
+    absSource.histograms().foreach(h => {
+      recordHistogram(absSource, h)
+      h.asInstanceOf[CelebornHistogram].reservoir
+        .asInstanceOf[ResettableSlidingWindowReservoir].reset()
+    })
+    absSource.timers().foreach(t => {
+      recordTimer(absSource, t)
+      t.timer.asInstanceOf[CelebornTimer].reservoir
+        .asInstanceOf[ResettableSlidingWindowReservoir].reset()
+    })
+    val metricDatas = new util.ArrayList[MetricData]
+    innerMetrics.synchronized {
+      while (!innerMetrics.isEmpty) {
+        metricDatas.add(innerMetrics.poll)
+      }
+      innerMetrics.clear()
+    }
+    metricDatas
+  }
+
+  def recordCounter(absSource: AbstractSource, nc: NamedCounter): Unit = {
+    val timestamp = System.currentTimeMillis
+    val labelNames = new util.ArrayList[String]
+    val labelValues = new util.ArrayList[String]
+    nc.labels.map { case (k, v) =>
+      labelNames.add(k)
+      labelValues.add(v)
+    }
+    val metricData =
+      new MetricData(nc.name, nc.counter.getCount, timestamp, labelNames, 
labelValues)
+    updateInnerMetrics(absSource, metricData)
+  }
+
+  def recordGauge(absSource: AbstractSource, ng: NamedGauge[_]): Unit = {
+    val timestamp = System.currentTimeMillis
+    val labelNames = new util.ArrayList[String]
+    val labelValues = new util.ArrayList[String]
+    ng.labels.map { case (k, v) =>
+      labelNames.add(k)
+      labelValues.add(v)
+    }
+    val metricData = new MetricData(ng.name, ng.gauge.getValue, timestamp, 
labelNames, labelValues)
+    updateInnerMetrics(absSource, metricData)
+  }
+
+  def recordHistogram(absSource: AbstractSource, nh: NamedHistogram): Unit = {
+    val timestamp = System.currentTimeMillis
+    val labelNames = new util.ArrayList[String]
+    val labelValues = new util.ArrayList[String]
+    nh.labels.map { case (k, v) =>
+      labelNames.add(k)
+      labelValues.add(v)
+    }
+    val snapshot = nh.histogram.getSnapshot
+    updateInnerMetrics(
+      absSource,
+      new MetricData(
+        s"${nh.name}_Count",
+        nh.histogram.getCount,
+        timestamp,
+        labelNames,
+        labelValues))
+    updateInnerMetrics(
+      absSource,
+      new MetricData(
+        s"${nh.name}_Max",
+        absSource.reportNanosAsMills(snapshot.getMax),
+        timestamp,
+        labelNames,
+        labelValues))
+    updateInnerMetrics(
+      absSource,
+      new MetricData(
+        s"${nh.name}_Mean",
+        absSource.reportNanosAsMills(snapshot.getMean),
+        timestamp,
+        labelNames,
+        labelValues))
+    updateInnerMetrics(
+      absSource,
+      new MetricData(
+        s"${nh.name}_Min",
+        absSource.reportNanosAsMills(snapshot.getMin),
+        timestamp,
+        labelNames,
+        labelValues))
+    updateInnerMetrics(
+      absSource,
+      new MetricData(
+        s"${nh.name}_50thPercentile",
+        absSource.reportNanosAsMills(snapshot.getMedian),
+        timestamp,
+        labelNames,
+        labelValues))
+    updateInnerMetrics(
+      absSource,
+      new MetricData(
+        s"${nh.name}_75thPercentile",
+        absSource.reportNanosAsMills(snapshot.get75thPercentile),
+        timestamp,
+        labelNames,
+        labelValues))
+    updateInnerMetrics(
+      absSource,
+      new MetricData(
+        s"${nh.name}_95thPercentile",
+        absSource.reportNanosAsMills(snapshot.get95thPercentile),
+        timestamp,
+        labelNames,
+        labelValues))
+    updateInnerMetrics(
+      absSource,
+      new MetricData(
+        s"${nh.name}_98thPercentile",
+        absSource.reportNanosAsMills(snapshot.get98thPercentile),
+        timestamp,
+        labelNames,
+        labelValues))
+    updateInnerMetrics(
+      absSource,
+      new MetricData(
+        s"${nh.name}_99thPercentile",
+        absSource.reportNanosAsMills(snapshot.get99thPercentile),
+        timestamp,
+        labelNames,
+        labelValues))
+    updateInnerMetrics(
+      absSource,
+      new MetricData(
+        s"${nh.name}_999thPercentile",
+        absSource.reportNanosAsMills(snapshot.get999thPercentile),
+        timestamp,
+        labelNames,
+        labelValues))
+  }
+
+  def recordTimer(absSource: AbstractSource, nt: NamedTimer): Unit = {
+    val timestamp = System.currentTimeMillis
+    val labelNames = new util.ArrayList[String]
+    val labelValues = new util.ArrayList[String]
+    nt.labels.map { case (k, v) =>
+      labelNames.add(k)
+      labelValues.add(v)
+    }
+    val snapshot = nt.timer.getSnapshot
+    updateInnerMetrics(
+      absSource,
+      new MetricData(s"${nt.name}_Count", nt.timer.getCount, timestamp, 
labelNames, labelValues))
+    updateInnerMetrics(
+      absSource,
+      new MetricData(
+        s"${nt.name}_Max",
+        absSource.reportNanosAsMills(snapshot.getMax),
+        timestamp,
+        labelNames,
+        labelValues))
+    updateInnerMetrics(
+      absSource,
+      new MetricData(
+        s"${nt.name}_Mean",
+        absSource.reportNanosAsMills(snapshot.getMean),
+        timestamp,
+        labelNames,
+        labelValues))
+    updateInnerMetrics(
+      absSource,
+      new MetricData(
+        s"${nt.name}_Min",
+        absSource.reportNanosAsMills(snapshot.getMin),
+        timestamp,
+        labelNames,
+        labelValues))
+    updateInnerMetrics(
+      absSource,
+      new MetricData(
+        s"${nt.name}_50thPercentile",
+        absSource.reportNanosAsMills(snapshot.getMedian),
+        timestamp,
+        labelNames,
+        labelValues))
+    updateInnerMetrics(
+      absSource,
+      new MetricData(
+        s"${nt.name}_75thPercentile",
+        absSource.reportNanosAsMills(snapshot.get75thPercentile),
+        timestamp,
+        labelNames,
+        labelValues))
+    updateInnerMetrics(
+      absSource,
+      new MetricData(
+        s"${nt.name}_95thPercentile",
+        absSource.reportNanosAsMills(snapshot.get95thPercentile),
+        timestamp,
+        labelNames,
+        labelValues))
+    updateInnerMetrics(
+      absSource,
+      new MetricData(
+        s"${nt.name}_98thPercentile",
+        absSource.reportNanosAsMills(snapshot.get98thPercentile),
+        timestamp,
+        labelNames,
+        labelValues))
+    updateInnerMetrics(
+      absSource,
+      new MetricData(
+        s"${nt.name}_99thPercentile",
+        absSource.reportNanosAsMills(snapshot.get99thPercentile),
+        timestamp,
+        labelNames,
+        labelValues))
+    updateInnerMetrics(
+      absSource,
+      new MetricData(
+        s"${nt.name}_999thPercentile",
+        absSource.reportNanosAsMills(snapshot.get999thPercentile),
+        timestamp,
+        labelNames,
+        labelValues))
+  }
+
+  private def updateInnerMetrics(absSource: AbstractSource, metricData: 
MetricData): Unit = {
+    innerMetrics.synchronized {
+      if (innerMetrics.size() >= absSource.metricsCapacity) {

Review Comment:
   
   
   
   
   > if condition meet, can we directly return? as remove and add new 
metricData seems useless
   
   @RexXiong   Thank you for your suggestion. I have resubmitted, please help 
me review it again.



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

Reply via email to