Github user squito commented on a diff in the pull request: https://github.com/apache/spark/pull/21221#discussion_r195291213 --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala --- @@ -98,14 +101,53 @@ class ExecutorSummary private[spark]( val removeReason: Option[String], val executorLogs: Map[String, String], val memoryMetrics: Option[MemoryMetrics], - val blacklistedInStages: Set[Int]) + val blacklistedInStages: Set[Int], + @JsonSerialize(using = classOf[PeakMemoryMetricsSerializer]) + @JsonDeserialize(using = classOf[PeakMemoryMetricsDeserializer]) + val peakMemoryMetrics: Option[Array[Long]]) class MemoryMetrics private[spark]( val usedOnHeapStorageMemory: Long, val usedOffHeapStorageMemory: Long, val totalOnHeapStorageMemory: Long, val totalOffHeapStorageMemory: Long) +/** deserialzer for peakMemoryMetrics: convert to array ordered by metric name */ +class PeakMemoryMetricsDeserializer extends JsonDeserializer[Option[Array[Long]]] { + override def deserialize( + jsonParser: JsonParser, + deserializationContext: DeserializationContext): Option[Array[Long]] = { + val metricsMap = jsonParser.readValueAs(classOf[Option[Map[String, Object]]]) + metricsMap match { + case Some(metrics) => + Some(MetricGetter.values.map { m => + metrics.getOrElse (m.name, 0L) match { + case intVal: Int => intVal.toLong + case longVal: Long => longVal + } + }.toArray) + case None => None + } + } +} + +/** serializer for peakMemoryMetrics: convert array to map with metric name as key */ +class PeakMemoryMetricsSerializer extends JsonSerializer[Option[Array[Long]]] { + override def serialize( + metrics: Option[Array[Long]], + jsonGenerator: JsonGenerator, + serializerProvider: SerializerProvider): Unit = { + metrics match { + case Some(m) => + val metricsMap = (0 until MetricGetter.values.length).map { idx => --- End diff -- ``` MetricGetter.idxAndValues.map { case (idx, getter) => getter.name -> m(idx) } ``` (or maybe we can get rid of `idxAndValues` if it doesn't really help ...)
--- --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org