You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/12/23 09:13:12 UTC

[GitHub] [spark] panbingkun opened a new pull request, #39192: [WIP][SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

panbingkun opened a new pull request, #39192:
URL: https://github.com/apache/spark/pull/39192

   ### What changes were proposed in this pull request?
   
   
   ### Why are the changes needed?
   
   
   ### Does this PR introduce _any_ user-facing change?
   No.
   
   ### How was this patch tested?
   Add new UT.


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gengliangwang commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1059196954


##########
core/src/test/scala/org/apache/spark/status/protobuf/KVStoreProtobufSerializerSuite.scala:
##########
@@ -773,4 +740,568 @@ class KVStoreProtobufSerializerSuite extends SparkFunSuite {
       assert(result.info.processLogs(k) == input.info.processLogs(k))
     }
   }
+
+  test("Stage Data") {
+    val accumulatorUpdates = Seq(
+      new AccumulableInfo(1L, "duration", Some("update"), "value1"),
+      new AccumulableInfo(2L, "duration2", None, "value2")
+    )
+    val inputMetrics = new InputMetrics(
+      bytesRead = 1L,
+      recordsRead = 2L)
+    val outputMetrics = new OutputMetrics(
+      bytesWritten = 1L,
+      recordsWritten = 2L
+    )
+    val shuffleReadMetrics = new ShuffleReadMetrics(
+      remoteBlocksFetched = 1L,
+      localBlocksFetched = 2L,
+      fetchWaitTime = 3L,
+      remoteBytesRead = 4L,
+      remoteBytesReadToDisk = 5L,
+      localBytesRead = 6L,
+      recordsRead = 7L
+    )
+    val shuffleWriteMetrics = new ShuffleWriteMetrics(
+      bytesWritten = 1L,
+      writeTime = 2L,
+      recordsWritten = 3L
+    )
+    val taskMetrics = new TaskMetrics(
+      executorDeserializeTime = 1L,
+      executorDeserializeCpuTime = 2L,
+      executorRunTime = 3L,
+      executorCpuTime = 4L,
+      resultSize = 5L,
+      jvmGcTime = 6L,
+      resultSerializationTime = 7L,
+      memoryBytesSpilled = 8L,
+      diskBytesSpilled = 9L,
+      peakExecutionMemory = 10L,
+      inputMetrics = inputMetrics,
+      outputMetrics = outputMetrics,
+      shuffleReadMetrics = shuffleReadMetrics,
+      shuffleWriteMetrics = shuffleWriteMetrics
+    )
+    val taskData1 = new TaskData(
+      taskId = 1L,
+      index = 2,
+      attempt = 3,
+      partitionId = 4,
+      launchTime = new Date(123456L),
+      resultFetchStart = Some(new Date(223456L)),
+      duration = Some(10000L),
+      executorId = "executor_id_1",
+      host = "host_name_1",
+      status = "SUCCESS",
+      taskLocality = "LOCAL",
+      speculative = true,
+      accumulatorUpdates = accumulatorUpdates,
+      errorMessage = Some("error_1"),
+      taskMetrics = Some(taskMetrics),
+      executorLogs = Map("executor_id_1" -> "executor_log_1"),
+      schedulerDelay = 5L,
+      gettingResultTime = 6L
+    )
+    val taskData2 = new TaskData(
+      taskId = 11L,
+      index = 12,
+      attempt = 13,
+      partitionId = 14,
+      launchTime = new Date(1123456L),
+      resultFetchStart = Some(new Date(1223456L)),
+      duration = Some(110000L),
+      executorId = "executor_id_2",
+      host = "host_name_2",
+      status = "SUCCESS",
+      taskLocality = "LOCAL",
+      speculative = false,
+      accumulatorUpdates = accumulatorUpdates,
+      errorMessage = Some("error_2"),
+      taskMetrics = Some(taskMetrics),
+      executorLogs = Map("executor_id_2" -> "executor_log_2"),
+      schedulerDelay = 15L,
+      gettingResultTime = 16L
+    )
+    val tasks = Some(
+      Map(1L -> taskData1, 2L -> taskData2)
+    )
+    val peakMemoryMetrics =
+      Some(new ExecutorMetrics(Array(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 1024L)))
+    val executorStageSummary1 = new ExecutorStageSummary(
+      taskTime = 1L,
+      failedTasks = 2,
+      succeededTasks = 3,
+      killedTasks = 4,
+      inputBytes = 5L,
+      inputRecords = 6L,
+      outputBytes = 7L,
+      outputRecords = 8L,
+      shuffleRead = 9L,
+      shuffleReadRecords = 10L,
+      shuffleWrite = 11L,
+      shuffleWriteRecords = 12L,
+      memoryBytesSpilled = 13L,
+      diskBytesSpilled = 14L,
+      isBlacklistedForStage = true,
+      peakMemoryMetrics = peakMemoryMetrics,
+      isExcludedForStage = false)
+    val executorStageSummary2 = new ExecutorStageSummary(
+      taskTime = 11L,
+      failedTasks = 12,
+      succeededTasks = 13,
+      killedTasks = 14,
+      inputBytes = 15L,
+      inputRecords = 16L,
+      outputBytes = 17L,
+      outputRecords = 18L,
+      shuffleRead = 19L,
+      shuffleReadRecords = 110L,
+      shuffleWrite = 111L,
+      shuffleWriteRecords = 112L,
+      memoryBytesSpilled = 113L,
+      diskBytesSpilled = 114L,
+      isBlacklistedForStage = false,
+      peakMemoryMetrics = peakMemoryMetrics,
+      isExcludedForStage = true)
+    val executorSummary = Some(
+      Map("executor_id_1" -> executorStageSummary1, "executor_id_2" -> executorStageSummary2)
+    )
+    val speculationStageSummary = new SpeculationStageSummary(
+      numTasks = 3,
+      numActiveTasks = 4,
+      numCompletedTasks = 5,
+      numFailedTasks = 6,
+      numKilledTasks = 7
+    )
+    val inputMetricDistributions = new InputMetricDistributions(
+      bytesRead = IndexedSeq(1.001D, 2.001D),
+      recordsRead = IndexedSeq(3.001D, 4.001D)
+    )
+    val outputMetricDistributions = new OutputMetricDistributions(
+      bytesWritten = IndexedSeq(1.001D, 2.001D),
+      recordsWritten = IndexedSeq(3.001D, 4.001D)
+    )
+    val shuffleReadMetricDistributions = new ShuffleReadMetricDistributions(
+      readBytes = IndexedSeq(1.001D, 2.001D),
+      readRecords = IndexedSeq(3.001D, 4.001D),
+      remoteBlocksFetched = IndexedSeq(5.001D, 6.001D),
+      localBlocksFetched = IndexedSeq(7.001D, 8.001D),
+      fetchWaitTime = IndexedSeq(9.001D, 10.001D),
+      remoteBytesRead = IndexedSeq(11.001D, 12.001D),
+      remoteBytesReadToDisk = IndexedSeq(13.001D, 14.001D),
+      totalBlocksFetched = IndexedSeq(15.001D, 16.001D)
+    )
+    val shuffleWriteMetricDistributions = new ShuffleWriteMetricDistributions(
+      writeBytes = IndexedSeq(1.001D, 2.001D),
+      writeRecords = IndexedSeq(3.001D, 4.001D),
+      writeTime = IndexedSeq(5.001D, 6.001D)
+    )
+    val taskMetricDistributions = new TaskMetricDistributions(
+      quantiles = IndexedSeq(1.001D, 2.001D),
+      duration = IndexedSeq(3.001D, 4.001D),
+      executorDeserializeTime = IndexedSeq(5.001D, 6.001D),
+      executorDeserializeCpuTime = IndexedSeq(7.001D, 8.001D),
+      executorRunTime = IndexedSeq(9.001D, 10.001D),
+      executorCpuTime = IndexedSeq(11.001D, 12.001D),
+      resultSize = IndexedSeq(13.001D, 14.001D),
+      jvmGcTime = IndexedSeq(15.001D, 16.001D),
+      resultSerializationTime = IndexedSeq(17.001D, 18.001D),
+      gettingResultTime = IndexedSeq(19.001D, 20.001D),
+      schedulerDelay = IndexedSeq(21.001D, 22.001D),
+      peakExecutionMemory = IndexedSeq(23.001D, 24.001D),
+      memoryBytesSpilled = IndexedSeq(25.001D, 26.001D),
+      diskBytesSpilled = IndexedSeq(27.001D, 28.001D),
+      inputMetrics = inputMetricDistributions,
+      outputMetrics = outputMetricDistributions,
+      shuffleReadMetrics = shuffleReadMetricDistributions,
+      shuffleWriteMetrics = shuffleWriteMetricDistributions
+    )
+    val executorPeakMetricsDistributions = new ExecutorPeakMetricsDistributions(
+      quantiles = IndexedSeq(1.001D, 2.001D),
+      executorMetrics = IndexedSeq(
+        new ExecutorMetrics(Array(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 1024L)))
+    )
+    val executorMetricsDistributions = new ExecutorMetricsDistributions(
+      quantiles = IndexedSeq(1.001D, 2.001D),
+      taskTime = IndexedSeq(3.001D, 4.001D),
+      failedTasks = IndexedSeq(5.001D, 6.001D),
+      succeededTasks = IndexedSeq(7.001D, 8.001D),
+      killedTasks = IndexedSeq(9.001D, 10.001D),
+      inputBytes = IndexedSeq(11.001D, 12.001D),
+      inputRecords = IndexedSeq(13.001D, 14.001D),
+      outputBytes = IndexedSeq(15.001D, 16.001D),
+      outputRecords = IndexedSeq(17.001D, 18.001D),
+      shuffleRead = IndexedSeq(19.001D, 20.001D),
+      shuffleReadRecords = IndexedSeq(21.001D, 22.001D),
+      shuffleWrite = IndexedSeq(23.001D, 24.001D),
+      shuffleWriteRecords = IndexedSeq(25.001D, 24.001D),
+      memoryBytesSpilled = IndexedSeq(27.001D, 28.001D),
+      diskBytesSpilled = IndexedSeq(29.001D, 30.001D),
+      peakMemoryMetrics = executorPeakMetricsDistributions
+    )
+    val info = new StageData(
+      status = StageStatus.COMPLETE,
+      stageId = 1,
+      attemptId = 2,
+      numTasks = 3,
+      numActiveTasks = 4,
+      numCompleteTasks = 5,
+      numFailedTasks = 6,
+      numKilledTasks = 7,
+      numCompletedIndices = 8,
+      submissionTime = Some(new Date(123456L)),
+      firstTaskLaunchedTime = Some(new Date(234567L)),
+      completionTime = Some(new Date(654321L)),
+      failureReason = Some("failure reason"),
+      executorDeserializeTime = 9L,
+      executorDeserializeCpuTime = 10L,
+      executorRunTime = 11L,
+      executorCpuTime = 12L,
+      resultSize = 13L,
+      jvmGcTime = 14L,
+      resultSerializationTime = 15L,
+      memoryBytesSpilled = 16L,
+      diskBytesSpilled = 17L,
+      peakExecutionMemory = 18L,
+      inputBytes = 19L,
+      inputRecords = 20L,
+      outputBytes = 21L,
+      outputRecords = 22L,
+      shuffleRemoteBlocksFetched = 23L,
+      shuffleLocalBlocksFetched = 24L,
+      shuffleFetchWaitTime = 25L,
+      shuffleRemoteBytesRead = 26L,
+      shuffleRemoteBytesReadToDisk = 27L,
+      shuffleLocalBytesRead = 28L,
+      shuffleReadBytes = 29L,
+      shuffleReadRecords = 30L,
+      shuffleWriteBytes = 31L,
+      shuffleWriteTime = 32L,
+      shuffleWriteRecords = 33L,
+      name = "name",
+      description = Some("test description"),
+      details = "test details",
+      schedulingPool = "test scheduling pool",
+      rddIds = Seq(1, 2, 3, 4, 5, 6),
+      accumulatorUpdates = accumulatorUpdates,
+      tasks = tasks,
+      executorSummary = executorSummary,
+      speculationSummary = Some(speculationStageSummary),
+      killedTasksSummary = Map("task_1" -> 1),
+      resourceProfileId = 34,
+      peakExecutorMetrics = peakMemoryMetrics,
+      taskMetricsDistributions = Some(taskMetricDistributions),
+      executorMetricsDistributions = Some(executorMetricsDistributions)
+    )
+    val input = new StageDataWrapper(
+      info = info,
+      jobIds = Set(1, 2, 3, 4),
+      locality = Map(
+        "PROCESS_LOCAL" -> 1L,
+        "NODE_LOCAL" -> 2L
+      )
+    )
+
+    val bytes = serializer.serialize(input)
+    val result = serializer.deserialize(bytes, classOf[StageDataWrapper])
+
+    assert(result.jobIds == input.jobIds)
+    assert(result.locality == input.locality)
+
+    assert(result.info.status == input.info.status)
+    assert(result.info.stageId == input.info.stageId)
+    assert(result.info.attemptId == input.info.attemptId)
+    assert(result.info.numTasks == input.info.numTasks)
+    assert(result.info.numActiveTasks == input.info.numActiveTasks)
+    assert(result.info.numCompleteTasks == input.info.numCompleteTasks)
+    assert(result.info.numFailedTasks == input.info.numFailedTasks)
+    assert(result.info.numKilledTasks == input.info.numKilledTasks)
+    assert(result.info.numCompletedIndices == input.info.numCompletedIndices)
+
+    assert(result.info.submissionTime == input.info.submissionTime)
+    assert(result.info.firstTaskLaunchedTime == input.info.firstTaskLaunchedTime)
+    assert(result.info.completionTime == input.info.completionTime)
+    assert(result.info.failureReason == input.info.failureReason)
+
+    assert(result.info.executorDeserializeTime == input.info.executorDeserializeTime)
+    assert(result.info.executorDeserializeCpuTime == input.info.executorDeserializeCpuTime)
+    assert(result.info.executorRunTime == input.info.executorRunTime)
+    assert(result.info.executorCpuTime == input.info.executorCpuTime)
+    assert(result.info.resultSize == input.info.resultSize)
+    assert(result.info.jvmGcTime == input.info.jvmGcTime)
+    assert(result.info.resultSerializationTime == input.info.resultSerializationTime)
+    assert(result.info.memoryBytesSpilled == input.info.memoryBytesSpilled)
+    assert(result.info.diskBytesSpilled == input.info.diskBytesSpilled)
+    assert(result.info.peakExecutionMemory == input.info.peakExecutionMemory)
+    assert(result.info.inputBytes == input.info.inputBytes)
+    assert(result.info.inputRecords == input.info.inputRecords)
+    assert(result.info.outputBytes == input.info.outputBytes)
+    assert(result.info.outputRecords == input.info.outputRecords)
+    assert(result.info.shuffleRemoteBlocksFetched == input.info.shuffleRemoteBlocksFetched)
+    assert(result.info.shuffleLocalBlocksFetched == input.info.shuffleLocalBlocksFetched)
+    assert(result.info.shuffleFetchWaitTime == input.info.shuffleFetchWaitTime)
+    assert(result.info.shuffleRemoteBytesRead == input.info.shuffleRemoteBytesRead)
+    assert(result.info.shuffleRemoteBytesReadToDisk == input.info.shuffleRemoteBytesReadToDisk)
+    assert(result.info.shuffleLocalBytesRead == input.info.shuffleLocalBytesRead)
+    assert(result.info.shuffleReadBytes == input.info.shuffleReadBytes)
+    assert(result.info.shuffleReadRecords == input.info.shuffleReadRecords)
+    assert(result.info.shuffleWriteBytes == input.info.shuffleWriteBytes)
+    assert(result.info.shuffleWriteTime == input.info.shuffleWriteTime)
+    assert(result.info.shuffleWriteRecords == input.info.shuffleWriteRecords)
+
+    assert(result.info.name == input.info.name)
+    assert(result.info.description == input.info.description)
+    assert(result.info.details == input.info.details)
+    assert(result.info.schedulingPool == input.info.schedulingPool)
+
+    assert(result.info.rddIds == input.info.rddIds)
+    assert(result.info.accumulatorUpdates, input.info.accumulatorUpdates)
+
+    assert(result.info.tasks.isDefined == input.info.tasks.isDefined)
+    if (result.info.tasks.isDefined && input.info.tasks.isDefined) {
+      assertIdTask(result.info.tasks.get, input.info.tasks.get)
+    }
+
+    assert(result.info.executorSummary.isDefined == input.info.executorSummary.isDefined)
+    if (result.info.executorSummary.isDefined && input.info.executorSummary.isDefined) {
+      assert(result.info.executorSummary.get, input.info.executorSummary.get)
+    }
+
+    assert(result.info.speculationSummary.isDefined == input.info.speculationSummary.isDefined)
+    if (result.info.speculationSummary.isDefined && input.info.speculationSummary.isDefined) {
+      assert(result.info.speculationSummary.get, input.info.speculationSummary.get)
+    }
+    assert(result.info.killedTasksSummary == input.info.killedTasksSummary)
+    assert(result.info.resourceProfileId == input.info.resourceProfileId)
+    assert(result.info.peakExecutorMetrics.isDefined == input.info.peakExecutorMetrics.isDefined)
+    if (result.info.peakExecutorMetrics.isDefined && input.info.peakExecutorMetrics.isDefined) {
+      assert(result.info.peakExecutorMetrics.get, input.info.peakExecutorMetrics.get)
+    }
+    assert(result.info.taskMetricsDistributions.isDefined ==
+      input.info.taskMetricsDistributions.isDefined)
+    if (result.info.taskMetricsDistributions.isDefined &&
+      input.info.taskMetricsDistributions.isDefined) {
+      assert(result.info.taskMetricsDistributions.get, input.info.taskMetricsDistributions.get)
+    }
+    assert(result.info.executorMetricsDistributions.isDefined ==
+      input.info.executorMetricsDistributions.isDefined)
+    if (result.info.executorMetricsDistributions.isDefined &&
+      input.info.executorMetricsDistributions.isDefined) {
+      assert(result.info.executorMetricsDistributions.get,
+        input.info.executorMetricsDistributions.get)
+    }
+  }
+
+  private def assert(result: TaskMetrics, input: TaskMetrics): Unit = {

Review Comment:
   nit: rename all the `assert` methods as `checkAnwser(result, expected)`?



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] LuciferYang commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1058797949


##########
core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto:
##########
@@ -390,3 +390,214 @@ message SQLExecutionUIData {
   repeated int64 stages = 11;
   map<int64, string> metric_values = 12;
 }
+
+message StageDataWrapper {
+  StageData info = 1;
+  repeated int64 job_ids = 2;
+  map<string, int64> locality = 3;
+}
+
+message TaskData {
+  int64 task_id = 1;
+  int32 index = 2;
+  int32 attempt = 3;
+  int32 partition_id = 4;
+  int64 launch_time = 5;
+  optional int64 result_fetch_start = 6;
+  optional int64 duration = 7;
+  string executor_id = 8;
+  string host = 9;
+  string status = 10;
+  string task_locality = 11;
+  bool speculative = 12;
+  repeated AccumulableInfo accumulator_updates = 13;
+  optional string error_message = 14;
+  optional TaskMetrics task_metrics = 15;
+  map<string, string> executor_logs = 16;
+  int64 scheduler_delay = 17;
+  int64 getting_result_time = 18;
+}
+
+message StageData {
+  enum StageStatus {

Review Comment:
   As described in  https://github.com/apache/spark/pull/39270/files, `UNSPECIFIED` in `StageStatus`  should change to `STAGE_STATUS_UNSPECIFIED`
   
   



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on pull request #39192: [WIP][SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
panbingkun commented on PR #39192:
URL: https://github.com/apache/spark/pull/39192#issuecomment-1363766697

   Waiting for me to add new UT.


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gengliangwang commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1059196346


##########
core/src/main/scala/org/apache/spark/status/protobuf/ExecutorStageSummarySerializer.scala:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.spark.status.protobuf
+
+import org.apache.spark.status.api.v1.ExecutorStageSummary
+import org.apache.spark.status.protobuf.Utils.getOptional
+
+object ExecutorStageSummarySerializer {
+
+  private[protobuf] def serialize(input: ExecutorStageSummary): StoreTypes.ExecutorStageSummary = {

Review Comment:
   ditto



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gengliangwang commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1059196737


##########
core/src/main/scala/org/apache/spark/status/protobuf/StageStatusSerializer.scala:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.spark.status.protobuf
+
+import org.apache.commons.lang3.StringUtils
+
+import org.apache.spark.status.api.v1.StageStatus
+
+object StageStatusSerializer {

Review Comment:
   ditto



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gengliangwang commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1059196174


##########
core/src/main/scala/org/apache/spark/status/protobuf/AccumulableInfoSerializer.scala:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.{List => JList}
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.status.api.v1.AccumulableInfo
+import org.apache.spark.status.protobuf.Utils.getOptional
+
+object AccumulableInfoSerializer {

Review Comment:
   Let's put the `private[protobuf]` before the object `AccumulableInfoSerializer`



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gengliangwang commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1057953273


##########
core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto:
##########
@@ -390,3 +390,214 @@ message SQLExecutionUIData {
   repeated int64 stages = 11;
   map<int64, string> metric_values = 12;
 }
+
+message StageDataWrapper {
+  StageData info = 1;
+  repeated int64 job_ids = 2;
+  map<string, int64> locality = 3;
+}
+
+message TaskData {
+  int64 task_id = 1;
+  int32 index = 2;
+  int32 attempt = 3;
+  int32 partition_id = 4;
+  int64 launch_time = 5;
+  optional int64 result_fetch_start = 6;
+  optional int64 duration = 7;
+  string executor_id = 8;
+  string host = 9;
+  string status = 10;
+  string task_locality = 11;
+  bool speculative = 12;
+  repeated AccumulableInfo accumulator_updates = 13;
+  optional string error_message = 14;
+  optional TaskMetrics task_metrics = 15;
+  map<string, string> executor_logs = 16;
+  int64 scheduler_delay = 17;
+  int64 getting_result_time = 18;
+}
+
+message StageData {
+  enum StageStatus {
+    UNSPECIFIED = 0;
+    ACTIVE = 1;
+    COMPLETE = 2;
+    FAILED = 3;
+    PENDING = 4;
+    SKIPPED = 5;
+  }
+
+  StageStatus status = 1;
+  int64 stage_id = 2;
+  int32 attempt_id = 3;
+  int32 num_tasks = 4;
+  int32 num_active_tasks = 5;
+  int32 num_complete_tasks = 6;
+  int32 num_failed_tasks = 7;
+  int32 num_killed_tasks = 8;
+  int32 num_completed_indices = 9;
+
+  optional int64 submission_time = 10;
+  optional int64 first_task_launched_time = 11;
+  optional int64 completion_time = 12;
+  optional string failure_reason = 13;
+
+  int64 executor_deserialize_time = 14;
+  int64 executor_deserialize_cpu_time = 15;
+  int64 executor_run_time = 16;
+  int64 executor_cpu_time = 17;
+  int64 result_size = 18;
+  int64 jvm_gc_time = 19;
+  int64 result_serialization_time = 20;
+  int64 memory_bytes_spilled = 21;
+  int64 disk_bytes_spilled = 22;
+  int64 peak_execution_memory = 23;
+  int64 input_bytes = 24;
+  int64 input_records = 25;
+  int64 output_bytes = 26;
+  int64 output_records = 27;
+  int64 shuffle_remote_blocks_fetched = 28;
+  int64 shuffle_local_blocks_fetched = 29;
+  int64 shuffle_fetch_wait_time = 30;
+  int64 shuffle_remote_bytes_read = 31;
+  int64 shuffle_remote_bytes_read_to_disk = 32;
+  int64 shuffle_local_bytes_read = 33;
+  int64 shuffle_read_bytes = 34;
+  int64 shuffle_read_records = 35;
+  int64 shuffle_write_bytes = 36;
+  int64 shuffle_write_time = 37;
+  int64 shuffle_write_records = 38;
+
+  string name = 39;
+  optional string description = 40;
+  string details = 41;
+  string scheduling_pool = 42;
+
+  repeated int64 rdd_ids = 43;
+  repeated AccumulableInfo accumulator_updates = 44;
+  map<int64, TaskData> tasks = 45;

Review Comment:
   Simply a map is OK here. An empty map should make no difference with `None` 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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
panbingkun commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1058307597


##########
core/src/main/scala/org/apache/spark/status/protobuf/StageDataWrapperSerializer.scala:
##########
@@ -0,0 +1,627 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.Date
+
+import collection.JavaConverters._
+import org.apache.commons.collections4.MapUtils
+
+import org.apache.spark.status.StageDataWrapper
+import org.apache.spark.status.api.v1.{ExecutorMetricsDistributions, ExecutorPeakMetricsDistributions, InputMetricDistributions, InputMetrics, OutputMetricDistributions, OutputMetrics, ShuffleReadMetricDistributions, ShuffleReadMetrics, ShuffleWriteMetricDistributions, ShuffleWriteMetrics, SpeculationStageSummary, StageData, StageStatus, TaskData, TaskMetricDistributions, TaskMetrics}
+import org.apache.spark.status.protobuf.Utils.getOptional
+import org.apache.spark.util.Utils.weakIntern
+
+class StageDataWrapperSerializer extends ProtobufSerDe {
+
+  override val supportClass: Class[_] = classOf[StageDataWrapper]
+
+  override def serialize(input: Any): Array[Byte] =
+    serialize(input.asInstanceOf[StageDataWrapper])
+
+  private def serialize(s: StageDataWrapper): Array[Byte] = {
+    val builder = StoreTypes.StageDataWrapper.newBuilder()
+    builder.setInfo(serializeStageData(s.info))
+    s.jobIds.foreach(id => builder.addJobIds(id.toLong))
+    s.locality.foreach { entry =>
+      builder.putLocality(entry._1, entry._2)
+    }
+    builder.build().toByteArray
+  }
+
+  private def serializeStageData(stageData: StageData): StoreTypes.StageData = {
+    val stageDataBuilder = StoreTypes.StageData.newBuilder()
+    stageDataBuilder
+      .setStatus(serializeStageStatus(stageData.status))
+      .setStageId(stageData.stageId.toLong)
+      .setAttemptId(stageData.attemptId)
+      .setNumTasks(stageData.numTasks)
+      .setNumActiveTasks(stageData.numActiveTasks)
+      .setNumCompleteTasks(stageData.numCompleteTasks)
+      .setNumFailedTasks(stageData.numFailedTasks)
+      .setNumKilledTasks(stageData.numKilledTasks)
+      .setNumCompletedIndices(stageData.numCompletedIndices)
+      .setExecutorDeserializeTime(stageData.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(stageData.executorDeserializeCpuTime)
+      .setExecutorRunTime(stageData.executorRunTime)
+      .setExecutorCpuTime(stageData.executorCpuTime)
+      .setResultSize(stageData.resultSize)
+      .setJvmGcTime(stageData.jvmGcTime)
+      .setResultSerializationTime(stageData.resultSerializationTime)
+      .setMemoryBytesSpilled(stageData.memoryBytesSpilled)
+      .setDiskBytesSpilled(stageData.diskBytesSpilled)
+      .setPeakExecutionMemory(stageData.peakExecutionMemory)
+      .setInputBytes(stageData.inputBytes)
+      .setInputRecords(stageData.inputRecords)
+      .setOutputBytes(stageData.outputBytes)
+      .setOutputRecords(stageData.outputRecords)
+      .setShuffleRemoteBlocksFetched(stageData.shuffleRemoteBlocksFetched)
+      .setShuffleLocalBlocksFetched(stageData.shuffleLocalBlocksFetched)
+      .setShuffleFetchWaitTime(stageData.shuffleFetchWaitTime)
+      .setShuffleRemoteBytesRead(stageData.shuffleRemoteBytesRead)
+      .setShuffleRemoteBytesReadToDisk(stageData.shuffleRemoteBytesReadToDisk)
+      .setShuffleLocalBytesRead(stageData.shuffleLocalBytesRead)
+      .setShuffleReadBytes(stageData.shuffleReadBytes)
+      .setShuffleReadRecords(stageData.shuffleReadRecords)
+      .setShuffleWriteBytes(stageData.shuffleWriteBytes)
+      .setShuffleWriteTime(stageData.shuffleWriteTime)
+      .setShuffleWriteRecords(stageData.shuffleWriteRecords)
+      .setName(stageData.name)
+      .setDetails(stageData.details)
+      .setSchedulingPool(stageData.schedulingPool)
+      .setResourceProfileId(stageData.resourceProfileId)
+    stageData.submissionTime.foreach { d =>
+      stageDataBuilder.setSubmissionTime(d.getTime)
+    }
+    stageData.firstTaskLaunchedTime.foreach { d =>
+      stageDataBuilder.setFirstTaskLaunchedTime(d.getTime)
+    }
+    stageData.completionTime.foreach { d =>
+      stageDataBuilder.setCompletionTime(d.getTime)
+    }
+    stageData.failureReason.foreach { fr =>
+      stageDataBuilder.setFailureReason(fr)
+    }
+    stageData.description.foreach { d =>
+      stageDataBuilder.setDescription(d)
+    }
+    stageData.rddIds.foreach(id => stageDataBuilder.addRddIds(id.toLong))
+    stageData.accumulatorUpdates.foreach { update =>
+      stageDataBuilder.addAccumulatorUpdates(
+        AccumulableInfoSerializer.serializeAccumulableInfo(update))
+    }
+    stageData.tasks.foreach { t =>
+      t.foreach { entry =>
+        stageDataBuilder.putTasks(entry._1, serializeTaskData(entry._2))
+      }
+    }
+    stageData.executorSummary.foreach { es =>
+      es.foreach { entry =>
+        stageDataBuilder.putExecutorSummary(entry._1,
+          ExecutorStageSummarySerializer.serializeExecutorStageSummary(entry._2))
+      }
+    }
+    stageData.speculationSummary.foreach { ss =>
+      stageDataBuilder.setSpeculationSummary(serializeSpeculationStageSummary(ss))
+    }
+    stageData.killedTasksSummary.foreach { entry =>
+      stageDataBuilder.putKilledTasksSummary(entry._1, entry._2)
+    }
+    stageData.peakExecutorMetrics.foreach { pem =>
+      stageDataBuilder.setPeakExecutorMetrics(ExecutorMetricsSerializer.serialize(pem))
+    }
+    stageData.taskMetricsDistributions.foreach { tmd =>
+      stageDataBuilder.setTaskMetricsDistributions(serializeTaskMetricDistributions(tmd))
+    }
+    stageData.executorMetricsDistributions.foreach { emd =>
+      stageDataBuilder.setExecutorMetricsDistributions(serializeExecutorMetricsDistributions(emd))
+    }
+    stageDataBuilder.build()
+  }
+
+  private def serializeStageStatus(s: StageStatus): StoreTypes.StageData.StageStatus = {
+    StoreTypes.StageData.StageStatus.valueOf(s.toString)
+  }
+
+  private def serializeTaskData(t: TaskData): StoreTypes.TaskData = {
+    val taskDataBuilder = StoreTypes.TaskData.newBuilder()
+    taskDataBuilder
+      .setTaskId(t.taskId)
+      .setIndex(t.index)
+      .setAttempt(t.attempt)
+      .setPartitionId(t.partitionId)
+      .setLaunchTime(t.launchTime.getTime)
+      .setExecutorId(t.executorId)
+      .setHost(t.host)
+      .setStatus(t.status)
+      .setTaskLocality(t.taskLocality)
+      .setSpeculative(t.speculative)
+      .setSchedulerDelay(t.schedulerDelay)
+      .setGettingResultTime(t.gettingResultTime)
+    t.resultFetchStart.foreach { rfs =>
+      taskDataBuilder.setResultFetchStart(rfs.getTime)
+    }
+    t.duration.foreach { d =>
+      taskDataBuilder.setDuration(d)
+    }
+    t.accumulatorUpdates.foreach { update =>
+      taskDataBuilder.addAccumulatorUpdates(
+        AccumulableInfoSerializer.serializeAccumulableInfo(update))
+    }
+    t.errorMessage.foreach { em =>
+      taskDataBuilder.setErrorMessage(em)
+    }
+    t.taskMetrics.foreach { tm =>
+      taskDataBuilder.setTaskMetrics(serializeTaskMetrics(tm))
+    }
+    t.executorLogs.foreach { entry =>
+      taskDataBuilder.putExecutorLogs(entry._1, entry._2)
+    }
+    taskDataBuilder.build()
+  }
+
+  private def serializeTaskMetrics(tm: TaskMetrics): StoreTypes.TaskMetrics = {
+    val taskMetricsBuilder = StoreTypes.TaskMetrics.newBuilder()
+    taskMetricsBuilder
+      .setExecutorDeserializeTime(tm.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(tm.executorDeserializeCpuTime)
+      .setExecutorRunTime(tm.executorRunTime)
+      .setExecutorCpuTime(tm.executorCpuTime)
+      .setResultSize(tm.resultSize)
+      .setJvmGcTime(tm.jvmGcTime)
+      .setResultSerializationTime(tm.resultSerializationTime)
+      .setMemoryBytesSpilled(tm.memoryBytesSpilled)
+      .setDiskBytesSpilled(tm.diskBytesSpilled)
+      .setPeakExecutionMemory(tm.peakExecutionMemory)
+      .setInputMetrics(serializeInputMetrics(tm.inputMetrics))
+      .setOutputMetrics(serializeOutputMetrics(tm.outputMetrics))
+      .setShuffleReadMetrics(serializeShuffleReadMetrics(tm.shuffleReadMetrics))
+      .setShuffleWriteMetrics(serializeShuffleWriteMetrics(tm.shuffleWriteMetrics))
+    taskMetricsBuilder.build()
+  }
+
+  private def serializeInputMetrics(im: InputMetrics): StoreTypes.InputMetrics = {
+    StoreTypes.InputMetrics.newBuilder()
+      .setBytesRead(im.bytesRead)
+      .setRecordsRead(im.recordsRead)
+      .build()
+  }
+
+  private def serializeOutputMetrics(om: OutputMetrics): StoreTypes.OutputMetrics = {
+    StoreTypes.OutputMetrics.newBuilder()
+      .setBytesWritten(om.bytesWritten)
+      .setRecordsWritten(om.recordsWritten)
+      .build()
+  }
+
+  private def serializeShuffleReadMetrics(
+      srm: ShuffleReadMetrics): StoreTypes.ShuffleReadMetrics = {
+    StoreTypes.ShuffleReadMetrics.newBuilder()
+      .setRemoteBlocksFetched(srm.remoteBlocksFetched)
+      .setLocalBlocksFetched(srm.localBlocksFetched)
+      .setFetchWaitTime(srm.fetchWaitTime)
+      .setRemoteBytesRead(srm.remoteBytesRead)
+      .setRemoteBytesReadToDisk(srm.remoteBytesReadToDisk)
+      .setLocalBytesRead(srm.localBytesRead)
+      .setRecordsRead(srm.recordsRead)
+      .build()
+  }
+
+  private def serializeShuffleWriteMetrics(
+      swm: ShuffleWriteMetrics): StoreTypes.ShuffleWriteMetrics = {
+    StoreTypes.ShuffleWriteMetrics.newBuilder()
+      .setBytesWritten(swm.bytesWritten)
+      .setWriteTime(swm.writeTime)
+      .setRecordsWritten(swm.recordsWritten)
+      .build()
+  }
+
+  private def serializeSpeculationStageSummary(
+      sss: SpeculationStageSummary): StoreTypes.SpeculationStageSummary = {
+    StoreTypes.SpeculationStageSummary.newBuilder()
+      .setNumTasks(sss.numTasks)
+      .setNumActiveTasks(sss.numActiveTasks)
+      .setNumCompletedTasks(sss.numCompletedTasks)
+      .setNumFailedTasks(sss.numFailedTasks)
+      .setNumKilledTasks(sss.numKilledTasks)
+      .build()
+  }
+
+  private def serializeTaskMetricDistributions(
+      tmd: TaskMetricDistributions): StoreTypes.TaskMetricDistributions = {
+    val builder = StoreTypes.TaskMetricDistributions.newBuilder()
+    tmd.quantiles.foreach(q => builder.addQuantiles(q))
+    tmd.duration.foreach(d => builder.addDuration(d))
+    tmd.executorDeserializeTime.foreach(edt => builder.addExecutorDeserializeTime(edt))
+    tmd.executorDeserializeCpuTime.foreach(edct => builder.addExecutorDeserializeCpuTime(edct))
+    tmd.executorRunTime.foreach(ert => builder.addExecutorRunTime(ert))
+    tmd.executorCpuTime.foreach(ect => builder.addExecutorCpuTime(ect))
+    tmd.resultSize.foreach(rs => builder.addResultSize(rs))
+    tmd.jvmGcTime.foreach(jgt => builder.addJvmGcTime(jgt))
+    tmd.resultSerializationTime.foreach(rst => builder.addResultSerializationTime(rst))
+    tmd.gettingResultTime.foreach(grt => builder.addGettingResultTime(grt))
+    tmd.schedulerDelay.foreach(sd => builder.addSchedulerDelay(sd))
+    tmd.peakExecutionMemory.foreach(pem => builder.addPeakExecutionMemory(pem))
+    tmd.memoryBytesSpilled.foreach(mbs => builder.addMemoryBytesSpilled(mbs))
+    tmd.diskBytesSpilled.foreach(dbs => builder.addDiskBytesSpilled(dbs))
+    builder
+      .setInputMetrics(serializeInputMetricDistributions(tmd.inputMetrics))
+      .setOutputMetrics(serializeOutputMetricDistributions(tmd.outputMetrics))
+      .setShuffleReadMetrics(serializeShuffleReadMetricDistributions(tmd.shuffleReadMetrics))
+      .setShuffleWriteMetrics(serializeShuffleWriteMetricDistributions(tmd.shuffleWriteMetrics))
+      .build()
+  }
+
+  private def serializeInputMetricDistributions(
+      imd: InputMetricDistributions): StoreTypes.InputMetricDistributions = {
+    val builder = StoreTypes.InputMetricDistributions.newBuilder()
+    imd.bytesRead.foreach(br => builder.addBytesRead(br))
+    imd.recordsRead.foreach(rr => builder.addRecordsRead(rr))
+    builder.build()
+  }
+
+  private def serializeOutputMetricDistributions(
+      omd: OutputMetricDistributions): StoreTypes.OutputMetricDistributions = {
+    val builder = StoreTypes.OutputMetricDistributions.newBuilder()
+    omd.bytesWritten.foreach(bw => builder.addBytesWritten(bw))
+    omd.recordsWritten.foreach(rw => builder.addRecordsWritten(rw))
+    builder.build()
+  }
+
+  private def serializeShuffleReadMetricDistributions(
+      srmd: ShuffleReadMetricDistributions): StoreTypes.ShuffleReadMetricDistributions = {
+    val builder = StoreTypes.ShuffleReadMetricDistributions.newBuilder()
+    srmd.readBytes.foreach(rb => builder.addReadBytes(rb))
+    srmd.readRecords.foreach(rr => builder.addReadRecords(rr))
+    srmd.remoteBlocksFetched.foreach(rbf => builder.addRemoteBlocksFetched(rbf))
+    srmd.localBlocksFetched.foreach(lbf => builder.addLocalBlocksFetched(lbf))
+    srmd.fetchWaitTime.foreach(fwt => builder.addFetchWaitTime(fwt))
+    srmd.remoteBytesRead.foreach(rbr => builder.addRemoteBytesRead(rbr))
+    srmd.remoteBytesReadToDisk.foreach(rbrtd => builder.addRemoteBytesReadToDisk(rbrtd))
+    srmd.totalBlocksFetched.foreach(tbf => builder.addTotalBlocksFetched(tbf))
+    builder.build()
+  }
+
+  private def serializeShuffleWriteMetricDistributions(
+      swmd: ShuffleWriteMetricDistributions): StoreTypes.ShuffleWriteMetricDistributions = {
+    val builder = StoreTypes.ShuffleWriteMetricDistributions.newBuilder()
+    swmd.writeBytes.foreach(wb => builder.addWriteBytes(wb))
+    swmd.writeRecords.foreach(wr => builder.addWriteRecords(wr))
+    swmd.writeTime.foreach(wt => builder.addWriteTime(wt))
+    builder.build()
+  }
+
+  private def serializeExecutorMetricsDistributions(
+      emd: ExecutorMetricsDistributions): StoreTypes.ExecutorMetricsDistributions = {
+    val builder = StoreTypes.ExecutorMetricsDistributions.newBuilder()
+    emd.quantiles.foreach(q => builder.addQuantiles(q))
+    emd.taskTime.foreach(tt => builder.addTaskTime(tt))
+    emd.failedTasks.foreach(ft => builder.addFailedTasks(ft))
+    emd.succeededTasks.foreach(st => builder.addSucceededTasks(st))
+    emd.killedTasks.foreach(kt => builder.addKilledTasks(kt))
+    emd.inputBytes.foreach(ib => builder.addInputBytes(ib))
+    emd.inputRecords.foreach(ir => builder.addInputRecords(ir))
+    emd.outputBytes.foreach(ob => builder.addOutputBytes(ob))
+    emd.outputRecords.foreach(or => builder.addOutputRecords(or))
+    emd.shuffleRead.foreach(sr => builder.addShuffleRead(sr))
+    emd.shuffleReadRecords.foreach(srr => builder.addShuffleReadRecords(srr))
+    emd.shuffleWrite.foreach(sw => builder.addShuffleWrite(sw))
+    emd.shuffleWriteRecords.foreach(swr => builder.addShuffleWriteRecords(swr))
+    emd.memoryBytesSpilled.foreach(mbs => builder.addMemoryBytesSpilled(mbs))
+    emd.diskBytesSpilled.foreach(dbs => builder.addDiskBytesSpilled(dbs))
+    builder.setPeakMemoryMetrics(serializeExecutorPeakMetricsDistributions(emd.peakMemoryMetrics))
+    builder.build()
+  }
+
+  private def serializeExecutorPeakMetricsDistributions(
+      epmd: ExecutorPeakMetricsDistributions): StoreTypes.ExecutorPeakMetricsDistributions = {
+    val builder = StoreTypes.ExecutorPeakMetricsDistributions.newBuilder()
+    epmd.quantiles.foreach(q => builder.addQuantiles(q))
+    epmd.executorMetrics.foreach(em => builder.addExecutorMetrics(
+      ExecutorMetricsSerializer.serialize(em)))
+    builder.build()
+  }
+
+  override def deserialize(bytes: Array[Byte]): StageDataWrapper = {
+    val binary = StoreTypes.StageDataWrapper.parseFrom(bytes)
+    val info = deserializeStageData(binary.getInfo)
+    new StageDataWrapper(
+      info = info,
+      jobIds = binary.getJobIdsList.asScala.map(_.toInt).toSet,
+      locality = binary.getLocalityMap.asScala.mapValues(_.toLong).toMap
+    )
+  }
+
+  private def deserializeStageData(binary: StoreTypes.StageData): StageData = {
+    val status = StageStatus.valueOf(binary.getStatus.toString)
+    val submissionTime =
+      getOptional(binary.hasSubmissionTime, () => new Date(binary.getSubmissionTime))
+    val firstTaskLaunchedTime =
+      getOptional(binary.hasFirstTaskLaunchedTime, () => new Date(binary.getFirstTaskLaunchedTime))
+    val completionTime =
+      getOptional(binary.hasCompletionTime, () => new Date(binary.getCompletionTime))
+    val failureReason =
+      getOptional(binary.hasFailureReason, () => weakIntern(binary.getFailureReason))
+    val description =
+      getOptional(binary.hasDescription, () => weakIntern(binary.getDescription))
+    val accumulatorUpdates = AccumulableInfoSerializer.deserializeAccumulableInfos(
+      binary.getAccumulatorUpdatesList)
+    val tasks = MapUtils.isEmpty(binary.getTasksMap) match {
+      case true => None
+      case _ => Some(binary.getTasksMap.asScala.map(
+        entry => (entry._1.toLong, deserializeTaskData(entry._2))).toMap)
+    }
+    val executorSummary = MapUtils.isEmpty(binary.getExecutorSummaryMap) match {
+      case true => None
+      case _ => Some(binary.getExecutorSummaryMap.asScala.mapValues(
+          ExecutorStageSummarySerializer.deserializeExecutorStageSummary(_)).toMap
+      )
+    }
+    val speculationSummary =
+      getOptional(binary.hasSpeculationSummary,
+        () => deserializeSpeculationStageSummary(binary.getSpeculationSummary))
+    val peakExecutorMetrics =
+      getOptional(binary.hasPeakExecutorMetrics,
+        () => ExecutorMetricsSerializer.deserialize(binary.getPeakExecutorMetrics))
+    val taskMetricsDistributions =
+      getOptional(binary.hasTaskMetricsDistributions,
+        () => deserializeTaskMetricDistributions(binary.getTaskMetricsDistributions))
+    val executorMetricsDistributions =
+      getOptional(binary.hasExecutorMetricsDistributions,
+        () => deserializeExecutorMetricsDistributions(binary.getExecutorMetricsDistributions))
+    new StageData(
+      status = status,
+      stageId = binary.getStageId.toInt,
+      attemptId = binary.getAttemptId,
+      numTasks = binary.getNumTasks,
+      numActiveTasks = binary.getNumActiveTasks,
+      numCompleteTasks = binary.getNumCompleteTasks,
+      numFailedTasks = binary.getNumFailedTasks,
+      numKilledTasks = binary.getNumKilledTasks,
+      numCompletedIndices = binary.getNumCompletedIndices,
+      submissionTime = submissionTime,
+      firstTaskLaunchedTime = firstTaskLaunchedTime,
+      completionTime = completionTime,
+      failureReason = failureReason,
+      executorDeserializeTime = binary.getExecutorDeserializeTime,
+      executorDeserializeCpuTime = binary.getExecutorDeserializeCpuTime,
+      executorRunTime = binary.getExecutorRunTime,
+      executorCpuTime = binary.getExecutorCpuTime,
+      resultSize = binary.getResultSize,
+      jvmGcTime = binary.getJvmGcTime,
+      resultSerializationTime = binary.getResultSerializationTime,
+      memoryBytesSpilled = binary.getMemoryBytesSpilled,
+      diskBytesSpilled = binary.getDiskBytesSpilled,
+      peakExecutionMemory = binary.getPeakExecutionMemory,
+      inputBytes = binary.getInputBytes,
+      inputRecords = binary.getInputRecords,
+      outputBytes = binary.getOutputBytes,
+      outputRecords = binary.getOutputRecords,
+      shuffleRemoteBlocksFetched = binary.getShuffleRemoteBlocksFetched,
+      shuffleLocalBlocksFetched = binary.getShuffleLocalBlocksFetched,
+      shuffleFetchWaitTime = binary.getShuffleFetchWaitTime,
+      shuffleRemoteBytesRead = binary.getShuffleRemoteBytesRead,
+      shuffleRemoteBytesReadToDisk = binary.getShuffleRemoteBytesReadToDisk,
+      shuffleLocalBytesRead = binary.getShuffleLocalBytesRead,
+      shuffleReadBytes = binary.getShuffleReadBytes,
+      shuffleReadRecords = binary.getShuffleReadRecords,
+      shuffleWriteBytes = binary.getShuffleWriteBytes,
+      shuffleWriteTime = binary.getShuffleWriteTime,
+      shuffleWriteRecords = binary.getShuffleWriteRecords,
+      name = weakIntern(binary.getName),
+      description = description,
+      details = weakIntern(binary.getDetails),
+      schedulingPool = weakIntern(binary.getSchedulingPool),
+      rddIds = binary.getRddIdsList.asScala.map(_.toInt).toSeq,
+      accumulatorUpdates = accumulatorUpdates.toSeq,
+      tasks = tasks,
+      executorSummary = executorSummary,
+      speculationSummary = speculationSummary,
+      killedTasksSummary = binary.getKilledTasksSummaryMap.asScala.mapValues(_.toInt).toMap,
+      resourceProfileId = binary.getResourceProfileId,
+      peakExecutorMetrics = peakExecutorMetrics,
+      taskMetricsDistributions = taskMetricsDistributions,
+      executorMetricsDistributions = executorMetricsDistributions
+    )
+  }
+
+  private def deserializeSpeculationStageSummary(
+      binary: StoreTypes.SpeculationStageSummary): SpeculationStageSummary = {
+    new SpeculationStageSummary(
+      binary.getNumTasks,
+      binary.getNumActiveTasks,
+      binary.getNumCompletedTasks,
+      binary.getNumFailedTasks,
+      binary.getNumKilledTasks
+    )
+  }
+
+  private def deserializeTaskMetricDistributions(
+      binary: StoreTypes.TaskMetricDistributions): TaskMetricDistributions = {
+    new TaskMetricDistributions(
+      quantiles = binary.getQuantilesList.asScala.map(_.toDouble).toIndexedSeq,
+      duration = binary.getDurationList.asScala.map(_.toDouble).toIndexedSeq,
+      executorDeserializeTime =
+        binary.getExecutorDeserializeTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      executorDeserializeCpuTime =
+        binary.getExecutorDeserializeCpuTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      executorRunTime = binary.getExecutorRunTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      executorCpuTime = binary.getExecutorCpuTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      resultSize = binary.getResultSizeList.asScala.map(_.toDouble).toIndexedSeq,
+      jvmGcTime = binary.getJvmGcTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      resultSerializationTime =
+        binary.getResultSerializationTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      gettingResultTime = binary.getGettingResultTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      schedulerDelay = binary.getSchedulerDelayList.asScala.map(_.toDouble).toIndexedSeq,
+      peakExecutionMemory = binary.getPeakExecutionMemoryList.asScala.map(_.toDouble).toIndexedSeq,
+      memoryBytesSpilled = binary.getMemoryBytesSpilledList.asScala.map(_.toDouble).toIndexedSeq,
+      diskBytesSpilled = binary.getDiskBytesSpilledList.asScala.map(_.toDouble).toIndexedSeq,
+      inputMetrics = deserializeInputMetricDistributions(binary.getInputMetrics),
+      outputMetrics = deserializeOutputMetricDistributions(binary.getOutputMetrics),
+      shuffleReadMetrics = deserializeShuffleReadMetricDistributions(binary.getShuffleReadMetrics),
+      shuffleWriteMetrics =
+        deserializeShuffleWriteMetricDistributions(binary.getShuffleWriteMetrics)
+    )
+  }
+
+  private def deserializeInputMetricDistributions(
+      binary: StoreTypes.InputMetricDistributions): InputMetricDistributions = {
+    new InputMetricDistributions(
+      bytesRead = binary.getBytesReadList.asScala.map(_.toDouble).toIndexedSeq,
+      recordsRead = binary.getRecordsReadList.asScala.map(_.toDouble).toIndexedSeq
+    )
+  }
+
+  private def deserializeOutputMetricDistributions(
+      binary: StoreTypes.OutputMetricDistributions): OutputMetricDistributions = {
+    new OutputMetricDistributions(
+      bytesWritten = binary.getBytesWrittenList.asScala.map(_.toDouble).toIndexedSeq,
+      recordsWritten = binary.getRecordsWrittenList.asScala.map(_.toDouble).toIndexedSeq
+    )
+  }
+
+  private def deserializeShuffleReadMetricDistributions(
+      binary: StoreTypes.ShuffleReadMetricDistributions): ShuffleReadMetricDistributions = {
+    new ShuffleReadMetricDistributions(
+      readBytes = binary.getReadBytesList.asScala.map(_.toDouble).toIndexedSeq,
+      readRecords = binary.getReadRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      remoteBlocksFetched = binary.getRemoteBlocksFetchedList.asScala.map(_.toDouble).toIndexedSeq,
+      localBlocksFetched = binary.getLocalBlocksFetchedList.asScala.map(_.toDouble).toIndexedSeq,
+      fetchWaitTime = binary.getFetchWaitTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      remoteBytesRead = binary.getRemoteBytesReadList.asScala.map(_.toDouble).toIndexedSeq,
+      remoteBytesReadToDisk =
+        binary.getRemoteBytesReadToDiskList.asScala.map(_.toDouble).toIndexedSeq,
+      totalBlocksFetched = binary.getTotalBlocksFetchedList.asScala.map(_.toDouble).toIndexedSeq
+    )
+  }
+
+  private def deserializeShuffleWriteMetricDistributions(
+      binary: StoreTypes.ShuffleWriteMetricDistributions): ShuffleWriteMetricDistributions = {
+    new ShuffleWriteMetricDistributions(
+      writeBytes = binary.getWriteBytesList.asScala.map(_.toDouble).toIndexedSeq,
+      writeRecords = binary.getWriteRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      writeTime = binary.getWriteTimeList.asScala.map(_.toDouble).toIndexedSeq
+    )
+  }
+
+  private def deserializeExecutorMetricsDistributions(
+      binary: StoreTypes.ExecutorMetricsDistributions): ExecutorMetricsDistributions = {
+    new ExecutorMetricsDistributions(
+      quantiles = binary.getQuantilesList.asScala.map(_.toDouble).toIndexedSeq,
+      taskTime = binary.getTaskTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      failedTasks = binary.getFailedTasksList.asScala.map(_.toDouble).toIndexedSeq,
+      succeededTasks = binary.getSucceededTasksList.asScala.map(_.toDouble).toIndexedSeq,
+      killedTasks = binary.getKilledTasksList.asScala.map(_.toDouble).toIndexedSeq,
+      inputBytes = binary.getInputBytesList.asScala.map(_.toDouble).toIndexedSeq,
+      inputRecords = binary.getInputRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      outputBytes = binary.getOutputBytesList.asScala.map(_.toDouble).toIndexedSeq,
+      outputRecords = binary.getOutputRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      shuffleRead = binary.getShuffleReadList.asScala.map(_.toDouble).toIndexedSeq,
+      shuffleReadRecords = binary.getShuffleReadRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      shuffleWrite = binary.getShuffleWriteList.asScala.map(_.toDouble).toIndexedSeq,
+      shuffleWriteRecords = binary.getShuffleWriteRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      memoryBytesSpilled = binary.getMemoryBytesSpilledList.asScala.map(_.toDouble).toIndexedSeq,
+      diskBytesSpilled = binary.getDiskBytesSpilledList.asScala.map(_.toDouble).toIndexedSeq,
+      peakMemoryMetrics = deserializeExecutorPeakMetricsDistributions(binary.getPeakMemoryMetrics)
+    )
+  }
+
+  private def deserializeExecutorPeakMetricsDistributions(
+      binary: StoreTypes.ExecutorPeakMetricsDistributions): ExecutorPeakMetricsDistributions = {
+    new ExecutorPeakMetricsDistributions(
+      quantiles = binary.getQuantilesList.asScala.map(_.toDouble).toIndexedSeq,
+      executorMetrics = binary.getExecutorMetricsList.asScala.map(
+        ExecutorMetricsSerializer.deserialize(_)).toIndexedSeq

Review Comment:
   Done



##########
core/src/main/scala/org/apache/spark/status/protobuf/StageDataWrapperSerializer.scala:
##########
@@ -0,0 +1,627 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.Date
+
+import collection.JavaConverters._
+import org.apache.commons.collections4.MapUtils
+
+import org.apache.spark.status.StageDataWrapper
+import org.apache.spark.status.api.v1.{ExecutorMetricsDistributions, ExecutorPeakMetricsDistributions, InputMetricDistributions, InputMetrics, OutputMetricDistributions, OutputMetrics, ShuffleReadMetricDistributions, ShuffleReadMetrics, ShuffleWriteMetricDistributions, ShuffleWriteMetrics, SpeculationStageSummary, StageData, StageStatus, TaskData, TaskMetricDistributions, TaskMetrics}
+import org.apache.spark.status.protobuf.Utils.getOptional
+import org.apache.spark.util.Utils.weakIntern
+
+class StageDataWrapperSerializer extends ProtobufSerDe {
+
+  override val supportClass: Class[_] = classOf[StageDataWrapper]
+
+  override def serialize(input: Any): Array[Byte] =
+    serialize(input.asInstanceOf[StageDataWrapper])
+
+  private def serialize(s: StageDataWrapper): Array[Byte] = {
+    val builder = StoreTypes.StageDataWrapper.newBuilder()
+    builder.setInfo(serializeStageData(s.info))
+    s.jobIds.foreach(id => builder.addJobIds(id.toLong))
+    s.locality.foreach { entry =>
+      builder.putLocality(entry._1, entry._2)
+    }
+    builder.build().toByteArray
+  }
+
+  private def serializeStageData(stageData: StageData): StoreTypes.StageData = {
+    val stageDataBuilder = StoreTypes.StageData.newBuilder()
+    stageDataBuilder
+      .setStatus(serializeStageStatus(stageData.status))
+      .setStageId(stageData.stageId.toLong)
+      .setAttemptId(stageData.attemptId)
+      .setNumTasks(stageData.numTasks)
+      .setNumActiveTasks(stageData.numActiveTasks)
+      .setNumCompleteTasks(stageData.numCompleteTasks)
+      .setNumFailedTasks(stageData.numFailedTasks)
+      .setNumKilledTasks(stageData.numKilledTasks)
+      .setNumCompletedIndices(stageData.numCompletedIndices)
+      .setExecutorDeserializeTime(stageData.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(stageData.executorDeserializeCpuTime)
+      .setExecutorRunTime(stageData.executorRunTime)
+      .setExecutorCpuTime(stageData.executorCpuTime)
+      .setResultSize(stageData.resultSize)
+      .setJvmGcTime(stageData.jvmGcTime)
+      .setResultSerializationTime(stageData.resultSerializationTime)
+      .setMemoryBytesSpilled(stageData.memoryBytesSpilled)
+      .setDiskBytesSpilled(stageData.diskBytesSpilled)
+      .setPeakExecutionMemory(stageData.peakExecutionMemory)
+      .setInputBytes(stageData.inputBytes)
+      .setInputRecords(stageData.inputRecords)
+      .setOutputBytes(stageData.outputBytes)
+      .setOutputRecords(stageData.outputRecords)
+      .setShuffleRemoteBlocksFetched(stageData.shuffleRemoteBlocksFetched)
+      .setShuffleLocalBlocksFetched(stageData.shuffleLocalBlocksFetched)
+      .setShuffleFetchWaitTime(stageData.shuffleFetchWaitTime)
+      .setShuffleRemoteBytesRead(stageData.shuffleRemoteBytesRead)
+      .setShuffleRemoteBytesReadToDisk(stageData.shuffleRemoteBytesReadToDisk)
+      .setShuffleLocalBytesRead(stageData.shuffleLocalBytesRead)
+      .setShuffleReadBytes(stageData.shuffleReadBytes)
+      .setShuffleReadRecords(stageData.shuffleReadRecords)
+      .setShuffleWriteBytes(stageData.shuffleWriteBytes)
+      .setShuffleWriteTime(stageData.shuffleWriteTime)
+      .setShuffleWriteRecords(stageData.shuffleWriteRecords)
+      .setName(stageData.name)
+      .setDetails(stageData.details)
+      .setSchedulingPool(stageData.schedulingPool)
+      .setResourceProfileId(stageData.resourceProfileId)
+    stageData.submissionTime.foreach { d =>
+      stageDataBuilder.setSubmissionTime(d.getTime)
+    }
+    stageData.firstTaskLaunchedTime.foreach { d =>
+      stageDataBuilder.setFirstTaskLaunchedTime(d.getTime)
+    }
+    stageData.completionTime.foreach { d =>
+      stageDataBuilder.setCompletionTime(d.getTime)
+    }
+    stageData.failureReason.foreach { fr =>
+      stageDataBuilder.setFailureReason(fr)
+    }
+    stageData.description.foreach { d =>
+      stageDataBuilder.setDescription(d)
+    }
+    stageData.rddIds.foreach(id => stageDataBuilder.addRddIds(id.toLong))
+    stageData.accumulatorUpdates.foreach { update =>
+      stageDataBuilder.addAccumulatorUpdates(
+        AccumulableInfoSerializer.serializeAccumulableInfo(update))
+    }
+    stageData.tasks.foreach { t =>
+      t.foreach { entry =>
+        stageDataBuilder.putTasks(entry._1, serializeTaskData(entry._2))
+      }
+    }
+    stageData.executorSummary.foreach { es =>
+      es.foreach { entry =>
+        stageDataBuilder.putExecutorSummary(entry._1,
+          ExecutorStageSummarySerializer.serializeExecutorStageSummary(entry._2))
+      }
+    }
+    stageData.speculationSummary.foreach { ss =>
+      stageDataBuilder.setSpeculationSummary(serializeSpeculationStageSummary(ss))
+    }
+    stageData.killedTasksSummary.foreach { entry =>
+      stageDataBuilder.putKilledTasksSummary(entry._1, entry._2)
+    }
+    stageData.peakExecutorMetrics.foreach { pem =>
+      stageDataBuilder.setPeakExecutorMetrics(ExecutorMetricsSerializer.serialize(pem))
+    }
+    stageData.taskMetricsDistributions.foreach { tmd =>
+      stageDataBuilder.setTaskMetricsDistributions(serializeTaskMetricDistributions(tmd))
+    }
+    stageData.executorMetricsDistributions.foreach { emd =>
+      stageDataBuilder.setExecutorMetricsDistributions(serializeExecutorMetricsDistributions(emd))
+    }
+    stageDataBuilder.build()
+  }
+
+  private def serializeStageStatus(s: StageStatus): StoreTypes.StageData.StageStatus = {
+    StoreTypes.StageData.StageStatus.valueOf(s.toString)
+  }
+
+  private def serializeTaskData(t: TaskData): StoreTypes.TaskData = {
+    val taskDataBuilder = StoreTypes.TaskData.newBuilder()
+    taskDataBuilder
+      .setTaskId(t.taskId)
+      .setIndex(t.index)
+      .setAttempt(t.attempt)
+      .setPartitionId(t.partitionId)
+      .setLaunchTime(t.launchTime.getTime)
+      .setExecutorId(t.executorId)
+      .setHost(t.host)
+      .setStatus(t.status)
+      .setTaskLocality(t.taskLocality)
+      .setSpeculative(t.speculative)
+      .setSchedulerDelay(t.schedulerDelay)
+      .setGettingResultTime(t.gettingResultTime)
+    t.resultFetchStart.foreach { rfs =>
+      taskDataBuilder.setResultFetchStart(rfs.getTime)
+    }
+    t.duration.foreach { d =>
+      taskDataBuilder.setDuration(d)
+    }
+    t.accumulatorUpdates.foreach { update =>
+      taskDataBuilder.addAccumulatorUpdates(
+        AccumulableInfoSerializer.serializeAccumulableInfo(update))
+    }
+    t.errorMessage.foreach { em =>
+      taskDataBuilder.setErrorMessage(em)
+    }
+    t.taskMetrics.foreach { tm =>
+      taskDataBuilder.setTaskMetrics(serializeTaskMetrics(tm))
+    }
+    t.executorLogs.foreach { entry =>
+      taskDataBuilder.putExecutorLogs(entry._1, entry._2)
+    }
+    taskDataBuilder.build()
+  }
+
+  private def serializeTaskMetrics(tm: TaskMetrics): StoreTypes.TaskMetrics = {
+    val taskMetricsBuilder = StoreTypes.TaskMetrics.newBuilder()
+    taskMetricsBuilder
+      .setExecutorDeserializeTime(tm.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(tm.executorDeserializeCpuTime)
+      .setExecutorRunTime(tm.executorRunTime)
+      .setExecutorCpuTime(tm.executorCpuTime)
+      .setResultSize(tm.resultSize)
+      .setJvmGcTime(tm.jvmGcTime)
+      .setResultSerializationTime(tm.resultSerializationTime)
+      .setMemoryBytesSpilled(tm.memoryBytesSpilled)
+      .setDiskBytesSpilled(tm.diskBytesSpilled)
+      .setPeakExecutionMemory(tm.peakExecutionMemory)
+      .setInputMetrics(serializeInputMetrics(tm.inputMetrics))
+      .setOutputMetrics(serializeOutputMetrics(tm.outputMetrics))
+      .setShuffleReadMetrics(serializeShuffleReadMetrics(tm.shuffleReadMetrics))
+      .setShuffleWriteMetrics(serializeShuffleWriteMetrics(tm.shuffleWriteMetrics))
+    taskMetricsBuilder.build()
+  }
+
+  private def serializeInputMetrics(im: InputMetrics): StoreTypes.InputMetrics = {
+    StoreTypes.InputMetrics.newBuilder()
+      .setBytesRead(im.bytesRead)
+      .setRecordsRead(im.recordsRead)
+      .build()
+  }
+
+  private def serializeOutputMetrics(om: OutputMetrics): StoreTypes.OutputMetrics = {
+    StoreTypes.OutputMetrics.newBuilder()
+      .setBytesWritten(om.bytesWritten)
+      .setRecordsWritten(om.recordsWritten)
+      .build()
+  }
+
+  private def serializeShuffleReadMetrics(
+      srm: ShuffleReadMetrics): StoreTypes.ShuffleReadMetrics = {
+    StoreTypes.ShuffleReadMetrics.newBuilder()
+      .setRemoteBlocksFetched(srm.remoteBlocksFetched)
+      .setLocalBlocksFetched(srm.localBlocksFetched)
+      .setFetchWaitTime(srm.fetchWaitTime)
+      .setRemoteBytesRead(srm.remoteBytesRead)
+      .setRemoteBytesReadToDisk(srm.remoteBytesReadToDisk)
+      .setLocalBytesRead(srm.localBytesRead)
+      .setRecordsRead(srm.recordsRead)
+      .build()
+  }
+
+  private def serializeShuffleWriteMetrics(
+      swm: ShuffleWriteMetrics): StoreTypes.ShuffleWriteMetrics = {
+    StoreTypes.ShuffleWriteMetrics.newBuilder()
+      .setBytesWritten(swm.bytesWritten)
+      .setWriteTime(swm.writeTime)
+      .setRecordsWritten(swm.recordsWritten)
+      .build()
+  }
+
+  private def serializeSpeculationStageSummary(
+      sss: SpeculationStageSummary): StoreTypes.SpeculationStageSummary = {
+    StoreTypes.SpeculationStageSummary.newBuilder()
+      .setNumTasks(sss.numTasks)
+      .setNumActiveTasks(sss.numActiveTasks)
+      .setNumCompletedTasks(sss.numCompletedTasks)
+      .setNumFailedTasks(sss.numFailedTasks)
+      .setNumKilledTasks(sss.numKilledTasks)
+      .build()
+  }
+
+  private def serializeTaskMetricDistributions(
+      tmd: TaskMetricDistributions): StoreTypes.TaskMetricDistributions = {
+    val builder = StoreTypes.TaskMetricDistributions.newBuilder()
+    tmd.quantiles.foreach(q => builder.addQuantiles(q))
+    tmd.duration.foreach(d => builder.addDuration(d))
+    tmd.executorDeserializeTime.foreach(edt => builder.addExecutorDeserializeTime(edt))
+    tmd.executorDeserializeCpuTime.foreach(edct => builder.addExecutorDeserializeCpuTime(edct))
+    tmd.executorRunTime.foreach(ert => builder.addExecutorRunTime(ert))
+    tmd.executorCpuTime.foreach(ect => builder.addExecutorCpuTime(ect))
+    tmd.resultSize.foreach(rs => builder.addResultSize(rs))
+    tmd.jvmGcTime.foreach(jgt => builder.addJvmGcTime(jgt))
+    tmd.resultSerializationTime.foreach(rst => builder.addResultSerializationTime(rst))
+    tmd.gettingResultTime.foreach(grt => builder.addGettingResultTime(grt))
+    tmd.schedulerDelay.foreach(sd => builder.addSchedulerDelay(sd))
+    tmd.peakExecutionMemory.foreach(pem => builder.addPeakExecutionMemory(pem))
+    tmd.memoryBytesSpilled.foreach(mbs => builder.addMemoryBytesSpilled(mbs))
+    tmd.diskBytesSpilled.foreach(dbs => builder.addDiskBytesSpilled(dbs))
+    builder
+      .setInputMetrics(serializeInputMetricDistributions(tmd.inputMetrics))
+      .setOutputMetrics(serializeOutputMetricDistributions(tmd.outputMetrics))
+      .setShuffleReadMetrics(serializeShuffleReadMetricDistributions(tmd.shuffleReadMetrics))
+      .setShuffleWriteMetrics(serializeShuffleWriteMetricDistributions(tmd.shuffleWriteMetrics))
+      .build()
+  }
+
+  private def serializeInputMetricDistributions(
+      imd: InputMetricDistributions): StoreTypes.InputMetricDistributions = {
+    val builder = StoreTypes.InputMetricDistributions.newBuilder()
+    imd.bytesRead.foreach(br => builder.addBytesRead(br))
+    imd.recordsRead.foreach(rr => builder.addRecordsRead(rr))
+    builder.build()
+  }
+
+  private def serializeOutputMetricDistributions(
+      omd: OutputMetricDistributions): StoreTypes.OutputMetricDistributions = {
+    val builder = StoreTypes.OutputMetricDistributions.newBuilder()
+    omd.bytesWritten.foreach(bw => builder.addBytesWritten(bw))
+    omd.recordsWritten.foreach(rw => builder.addRecordsWritten(rw))
+    builder.build()
+  }
+
+  private def serializeShuffleReadMetricDistributions(
+      srmd: ShuffleReadMetricDistributions): StoreTypes.ShuffleReadMetricDistributions = {
+    val builder = StoreTypes.ShuffleReadMetricDistributions.newBuilder()
+    srmd.readBytes.foreach(rb => builder.addReadBytes(rb))
+    srmd.readRecords.foreach(rr => builder.addReadRecords(rr))
+    srmd.remoteBlocksFetched.foreach(rbf => builder.addRemoteBlocksFetched(rbf))
+    srmd.localBlocksFetched.foreach(lbf => builder.addLocalBlocksFetched(lbf))
+    srmd.fetchWaitTime.foreach(fwt => builder.addFetchWaitTime(fwt))
+    srmd.remoteBytesRead.foreach(rbr => builder.addRemoteBytesRead(rbr))
+    srmd.remoteBytesReadToDisk.foreach(rbrtd => builder.addRemoteBytesReadToDisk(rbrtd))
+    srmd.totalBlocksFetched.foreach(tbf => builder.addTotalBlocksFetched(tbf))
+    builder.build()
+  }
+
+  private def serializeShuffleWriteMetricDistributions(
+      swmd: ShuffleWriteMetricDistributions): StoreTypes.ShuffleWriteMetricDistributions = {
+    val builder = StoreTypes.ShuffleWriteMetricDistributions.newBuilder()
+    swmd.writeBytes.foreach(wb => builder.addWriteBytes(wb))
+    swmd.writeRecords.foreach(wr => builder.addWriteRecords(wr))
+    swmd.writeTime.foreach(wt => builder.addWriteTime(wt))
+    builder.build()
+  }
+
+  private def serializeExecutorMetricsDistributions(
+      emd: ExecutorMetricsDistributions): StoreTypes.ExecutorMetricsDistributions = {
+    val builder = StoreTypes.ExecutorMetricsDistributions.newBuilder()
+    emd.quantiles.foreach(q => builder.addQuantiles(q))
+    emd.taskTime.foreach(tt => builder.addTaskTime(tt))
+    emd.failedTasks.foreach(ft => builder.addFailedTasks(ft))
+    emd.succeededTasks.foreach(st => builder.addSucceededTasks(st))
+    emd.killedTasks.foreach(kt => builder.addKilledTasks(kt))
+    emd.inputBytes.foreach(ib => builder.addInputBytes(ib))
+    emd.inputRecords.foreach(ir => builder.addInputRecords(ir))
+    emd.outputBytes.foreach(ob => builder.addOutputBytes(ob))
+    emd.outputRecords.foreach(or => builder.addOutputRecords(or))
+    emd.shuffleRead.foreach(sr => builder.addShuffleRead(sr))
+    emd.shuffleReadRecords.foreach(srr => builder.addShuffleReadRecords(srr))
+    emd.shuffleWrite.foreach(sw => builder.addShuffleWrite(sw))
+    emd.shuffleWriteRecords.foreach(swr => builder.addShuffleWriteRecords(swr))
+    emd.memoryBytesSpilled.foreach(mbs => builder.addMemoryBytesSpilled(mbs))
+    emd.diskBytesSpilled.foreach(dbs => builder.addDiskBytesSpilled(dbs))
+    builder.setPeakMemoryMetrics(serializeExecutorPeakMetricsDistributions(emd.peakMemoryMetrics))
+    builder.build()
+  }
+
+  private def serializeExecutorPeakMetricsDistributions(
+      epmd: ExecutorPeakMetricsDistributions): StoreTypes.ExecutorPeakMetricsDistributions = {
+    val builder = StoreTypes.ExecutorPeakMetricsDistributions.newBuilder()
+    epmd.quantiles.foreach(q => builder.addQuantiles(q))
+    epmd.executorMetrics.foreach(em => builder.addExecutorMetrics(
+      ExecutorMetricsSerializer.serialize(em)))
+    builder.build()
+  }
+
+  override def deserialize(bytes: Array[Byte]): StageDataWrapper = {
+    val binary = StoreTypes.StageDataWrapper.parseFrom(bytes)
+    val info = deserializeStageData(binary.getInfo)
+    new StageDataWrapper(
+      info = info,
+      jobIds = binary.getJobIdsList.asScala.map(_.toInt).toSet,
+      locality = binary.getLocalityMap.asScala.mapValues(_.toLong).toMap
+    )
+  }
+
+  private def deserializeStageData(binary: StoreTypes.StageData): StageData = {
+    val status = StageStatus.valueOf(binary.getStatus.toString)
+    val submissionTime =
+      getOptional(binary.hasSubmissionTime, () => new Date(binary.getSubmissionTime))
+    val firstTaskLaunchedTime =
+      getOptional(binary.hasFirstTaskLaunchedTime, () => new Date(binary.getFirstTaskLaunchedTime))
+    val completionTime =
+      getOptional(binary.hasCompletionTime, () => new Date(binary.getCompletionTime))
+    val failureReason =
+      getOptional(binary.hasFailureReason, () => weakIntern(binary.getFailureReason))
+    val description =
+      getOptional(binary.hasDescription, () => weakIntern(binary.getDescription))
+    val accumulatorUpdates = AccumulableInfoSerializer.deserializeAccumulableInfos(
+      binary.getAccumulatorUpdatesList)
+    val tasks = MapUtils.isEmpty(binary.getTasksMap) match {
+      case true => None
+      case _ => Some(binary.getTasksMap.asScala.map(
+        entry => (entry._1.toLong, deserializeTaskData(entry._2))).toMap)
+    }
+    val executorSummary = MapUtils.isEmpty(binary.getExecutorSummaryMap) match {

Review Comment:
   Done



##########
core/src/main/scala/org/apache/spark/status/protobuf/StageDataWrapperSerializer.scala:
##########
@@ -0,0 +1,627 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.Date
+
+import collection.JavaConverters._
+import org.apache.commons.collections4.MapUtils
+
+import org.apache.spark.status.StageDataWrapper
+import org.apache.spark.status.api.v1.{ExecutorMetricsDistributions, ExecutorPeakMetricsDistributions, InputMetricDistributions, InputMetrics, OutputMetricDistributions, OutputMetrics, ShuffleReadMetricDistributions, ShuffleReadMetrics, ShuffleWriteMetricDistributions, ShuffleWriteMetrics, SpeculationStageSummary, StageData, StageStatus, TaskData, TaskMetricDistributions, TaskMetrics}
+import org.apache.spark.status.protobuf.Utils.getOptional
+import org.apache.spark.util.Utils.weakIntern
+
+class StageDataWrapperSerializer extends ProtobufSerDe {
+
+  override val supportClass: Class[_] = classOf[StageDataWrapper]
+
+  override def serialize(input: Any): Array[Byte] =
+    serialize(input.asInstanceOf[StageDataWrapper])
+
+  private def serialize(s: StageDataWrapper): Array[Byte] = {
+    val builder = StoreTypes.StageDataWrapper.newBuilder()
+    builder.setInfo(serializeStageData(s.info))
+    s.jobIds.foreach(id => builder.addJobIds(id.toLong))
+    s.locality.foreach { entry =>
+      builder.putLocality(entry._1, entry._2)
+    }
+    builder.build().toByteArray
+  }
+
+  private def serializeStageData(stageData: StageData): StoreTypes.StageData = {
+    val stageDataBuilder = StoreTypes.StageData.newBuilder()
+    stageDataBuilder
+      .setStatus(serializeStageStatus(stageData.status))
+      .setStageId(stageData.stageId.toLong)
+      .setAttemptId(stageData.attemptId)
+      .setNumTasks(stageData.numTasks)
+      .setNumActiveTasks(stageData.numActiveTasks)
+      .setNumCompleteTasks(stageData.numCompleteTasks)
+      .setNumFailedTasks(stageData.numFailedTasks)
+      .setNumKilledTasks(stageData.numKilledTasks)
+      .setNumCompletedIndices(stageData.numCompletedIndices)
+      .setExecutorDeserializeTime(stageData.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(stageData.executorDeserializeCpuTime)
+      .setExecutorRunTime(stageData.executorRunTime)
+      .setExecutorCpuTime(stageData.executorCpuTime)
+      .setResultSize(stageData.resultSize)
+      .setJvmGcTime(stageData.jvmGcTime)
+      .setResultSerializationTime(stageData.resultSerializationTime)
+      .setMemoryBytesSpilled(stageData.memoryBytesSpilled)
+      .setDiskBytesSpilled(stageData.diskBytesSpilled)
+      .setPeakExecutionMemory(stageData.peakExecutionMemory)
+      .setInputBytes(stageData.inputBytes)
+      .setInputRecords(stageData.inputRecords)
+      .setOutputBytes(stageData.outputBytes)
+      .setOutputRecords(stageData.outputRecords)
+      .setShuffleRemoteBlocksFetched(stageData.shuffleRemoteBlocksFetched)
+      .setShuffleLocalBlocksFetched(stageData.shuffleLocalBlocksFetched)
+      .setShuffleFetchWaitTime(stageData.shuffleFetchWaitTime)
+      .setShuffleRemoteBytesRead(stageData.shuffleRemoteBytesRead)
+      .setShuffleRemoteBytesReadToDisk(stageData.shuffleRemoteBytesReadToDisk)
+      .setShuffleLocalBytesRead(stageData.shuffleLocalBytesRead)
+      .setShuffleReadBytes(stageData.shuffleReadBytes)
+      .setShuffleReadRecords(stageData.shuffleReadRecords)
+      .setShuffleWriteBytes(stageData.shuffleWriteBytes)
+      .setShuffleWriteTime(stageData.shuffleWriteTime)
+      .setShuffleWriteRecords(stageData.shuffleWriteRecords)
+      .setName(stageData.name)
+      .setDetails(stageData.details)
+      .setSchedulingPool(stageData.schedulingPool)
+      .setResourceProfileId(stageData.resourceProfileId)
+    stageData.submissionTime.foreach { d =>
+      stageDataBuilder.setSubmissionTime(d.getTime)
+    }
+    stageData.firstTaskLaunchedTime.foreach { d =>
+      stageDataBuilder.setFirstTaskLaunchedTime(d.getTime)
+    }
+    stageData.completionTime.foreach { d =>
+      stageDataBuilder.setCompletionTime(d.getTime)
+    }
+    stageData.failureReason.foreach { fr =>
+      stageDataBuilder.setFailureReason(fr)
+    }
+    stageData.description.foreach { d =>
+      stageDataBuilder.setDescription(d)
+    }
+    stageData.rddIds.foreach(id => stageDataBuilder.addRddIds(id.toLong))
+    stageData.accumulatorUpdates.foreach { update =>
+      stageDataBuilder.addAccumulatorUpdates(
+        AccumulableInfoSerializer.serializeAccumulableInfo(update))
+    }
+    stageData.tasks.foreach { t =>
+      t.foreach { entry =>
+        stageDataBuilder.putTasks(entry._1, serializeTaskData(entry._2))
+      }
+    }
+    stageData.executorSummary.foreach { es =>
+      es.foreach { entry =>
+        stageDataBuilder.putExecutorSummary(entry._1,
+          ExecutorStageSummarySerializer.serializeExecutorStageSummary(entry._2))
+      }
+    }
+    stageData.speculationSummary.foreach { ss =>
+      stageDataBuilder.setSpeculationSummary(serializeSpeculationStageSummary(ss))
+    }
+    stageData.killedTasksSummary.foreach { entry =>
+      stageDataBuilder.putKilledTasksSummary(entry._1, entry._2)
+    }
+    stageData.peakExecutorMetrics.foreach { pem =>
+      stageDataBuilder.setPeakExecutorMetrics(ExecutorMetricsSerializer.serialize(pem))
+    }
+    stageData.taskMetricsDistributions.foreach { tmd =>
+      stageDataBuilder.setTaskMetricsDistributions(serializeTaskMetricDistributions(tmd))
+    }
+    stageData.executorMetricsDistributions.foreach { emd =>
+      stageDataBuilder.setExecutorMetricsDistributions(serializeExecutorMetricsDistributions(emd))
+    }
+    stageDataBuilder.build()
+  }
+
+  private def serializeStageStatus(s: StageStatus): StoreTypes.StageData.StageStatus = {
+    StoreTypes.StageData.StageStatus.valueOf(s.toString)
+  }
+
+  private def serializeTaskData(t: TaskData): StoreTypes.TaskData = {
+    val taskDataBuilder = StoreTypes.TaskData.newBuilder()
+    taskDataBuilder
+      .setTaskId(t.taskId)
+      .setIndex(t.index)
+      .setAttempt(t.attempt)
+      .setPartitionId(t.partitionId)
+      .setLaunchTime(t.launchTime.getTime)
+      .setExecutorId(t.executorId)
+      .setHost(t.host)
+      .setStatus(t.status)
+      .setTaskLocality(t.taskLocality)
+      .setSpeculative(t.speculative)
+      .setSchedulerDelay(t.schedulerDelay)
+      .setGettingResultTime(t.gettingResultTime)
+    t.resultFetchStart.foreach { rfs =>
+      taskDataBuilder.setResultFetchStart(rfs.getTime)
+    }
+    t.duration.foreach { d =>
+      taskDataBuilder.setDuration(d)
+    }
+    t.accumulatorUpdates.foreach { update =>
+      taskDataBuilder.addAccumulatorUpdates(
+        AccumulableInfoSerializer.serializeAccumulableInfo(update))
+    }
+    t.errorMessage.foreach { em =>
+      taskDataBuilder.setErrorMessage(em)
+    }
+    t.taskMetrics.foreach { tm =>
+      taskDataBuilder.setTaskMetrics(serializeTaskMetrics(tm))
+    }
+    t.executorLogs.foreach { entry =>
+      taskDataBuilder.putExecutorLogs(entry._1, entry._2)
+    }
+    taskDataBuilder.build()
+  }
+
+  private def serializeTaskMetrics(tm: TaskMetrics): StoreTypes.TaskMetrics = {
+    val taskMetricsBuilder = StoreTypes.TaskMetrics.newBuilder()
+    taskMetricsBuilder
+      .setExecutorDeserializeTime(tm.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(tm.executorDeserializeCpuTime)
+      .setExecutorRunTime(tm.executorRunTime)
+      .setExecutorCpuTime(tm.executorCpuTime)
+      .setResultSize(tm.resultSize)
+      .setJvmGcTime(tm.jvmGcTime)
+      .setResultSerializationTime(tm.resultSerializationTime)
+      .setMemoryBytesSpilled(tm.memoryBytesSpilled)
+      .setDiskBytesSpilled(tm.diskBytesSpilled)
+      .setPeakExecutionMemory(tm.peakExecutionMemory)
+      .setInputMetrics(serializeInputMetrics(tm.inputMetrics))
+      .setOutputMetrics(serializeOutputMetrics(tm.outputMetrics))
+      .setShuffleReadMetrics(serializeShuffleReadMetrics(tm.shuffleReadMetrics))
+      .setShuffleWriteMetrics(serializeShuffleWriteMetrics(tm.shuffleWriteMetrics))
+    taskMetricsBuilder.build()
+  }
+
+  private def serializeInputMetrics(im: InputMetrics): StoreTypes.InputMetrics = {
+    StoreTypes.InputMetrics.newBuilder()
+      .setBytesRead(im.bytesRead)
+      .setRecordsRead(im.recordsRead)
+      .build()
+  }
+
+  private def serializeOutputMetrics(om: OutputMetrics): StoreTypes.OutputMetrics = {
+    StoreTypes.OutputMetrics.newBuilder()
+      .setBytesWritten(om.bytesWritten)
+      .setRecordsWritten(om.recordsWritten)
+      .build()
+  }
+
+  private def serializeShuffleReadMetrics(
+      srm: ShuffleReadMetrics): StoreTypes.ShuffleReadMetrics = {
+    StoreTypes.ShuffleReadMetrics.newBuilder()
+      .setRemoteBlocksFetched(srm.remoteBlocksFetched)
+      .setLocalBlocksFetched(srm.localBlocksFetched)
+      .setFetchWaitTime(srm.fetchWaitTime)
+      .setRemoteBytesRead(srm.remoteBytesRead)
+      .setRemoteBytesReadToDisk(srm.remoteBytesReadToDisk)
+      .setLocalBytesRead(srm.localBytesRead)
+      .setRecordsRead(srm.recordsRead)
+      .build()
+  }
+
+  private def serializeShuffleWriteMetrics(
+      swm: ShuffleWriteMetrics): StoreTypes.ShuffleWriteMetrics = {
+    StoreTypes.ShuffleWriteMetrics.newBuilder()
+      .setBytesWritten(swm.bytesWritten)
+      .setWriteTime(swm.writeTime)
+      .setRecordsWritten(swm.recordsWritten)
+      .build()
+  }
+
+  private def serializeSpeculationStageSummary(
+      sss: SpeculationStageSummary): StoreTypes.SpeculationStageSummary = {
+    StoreTypes.SpeculationStageSummary.newBuilder()
+      .setNumTasks(sss.numTasks)
+      .setNumActiveTasks(sss.numActiveTasks)
+      .setNumCompletedTasks(sss.numCompletedTasks)
+      .setNumFailedTasks(sss.numFailedTasks)
+      .setNumKilledTasks(sss.numKilledTasks)
+      .build()
+  }
+
+  private def serializeTaskMetricDistributions(
+      tmd: TaskMetricDistributions): StoreTypes.TaskMetricDistributions = {
+    val builder = StoreTypes.TaskMetricDistributions.newBuilder()
+    tmd.quantiles.foreach(q => builder.addQuantiles(q))
+    tmd.duration.foreach(d => builder.addDuration(d))
+    tmd.executorDeserializeTime.foreach(edt => builder.addExecutorDeserializeTime(edt))
+    tmd.executorDeserializeCpuTime.foreach(edct => builder.addExecutorDeserializeCpuTime(edct))
+    tmd.executorRunTime.foreach(ert => builder.addExecutorRunTime(ert))
+    tmd.executorCpuTime.foreach(ect => builder.addExecutorCpuTime(ect))
+    tmd.resultSize.foreach(rs => builder.addResultSize(rs))
+    tmd.jvmGcTime.foreach(jgt => builder.addJvmGcTime(jgt))
+    tmd.resultSerializationTime.foreach(rst => builder.addResultSerializationTime(rst))
+    tmd.gettingResultTime.foreach(grt => builder.addGettingResultTime(grt))
+    tmd.schedulerDelay.foreach(sd => builder.addSchedulerDelay(sd))
+    tmd.peakExecutionMemory.foreach(pem => builder.addPeakExecutionMemory(pem))
+    tmd.memoryBytesSpilled.foreach(mbs => builder.addMemoryBytesSpilled(mbs))
+    tmd.diskBytesSpilled.foreach(dbs => builder.addDiskBytesSpilled(dbs))
+    builder
+      .setInputMetrics(serializeInputMetricDistributions(tmd.inputMetrics))
+      .setOutputMetrics(serializeOutputMetricDistributions(tmd.outputMetrics))
+      .setShuffleReadMetrics(serializeShuffleReadMetricDistributions(tmd.shuffleReadMetrics))
+      .setShuffleWriteMetrics(serializeShuffleWriteMetricDistributions(tmd.shuffleWriteMetrics))
+      .build()
+  }
+
+  private def serializeInputMetricDistributions(
+      imd: InputMetricDistributions): StoreTypes.InputMetricDistributions = {
+    val builder = StoreTypes.InputMetricDistributions.newBuilder()
+    imd.bytesRead.foreach(br => builder.addBytesRead(br))
+    imd.recordsRead.foreach(rr => builder.addRecordsRead(rr))
+    builder.build()
+  }
+
+  private def serializeOutputMetricDistributions(
+      omd: OutputMetricDistributions): StoreTypes.OutputMetricDistributions = {
+    val builder = StoreTypes.OutputMetricDistributions.newBuilder()
+    omd.bytesWritten.foreach(bw => builder.addBytesWritten(bw))
+    omd.recordsWritten.foreach(rw => builder.addRecordsWritten(rw))
+    builder.build()
+  }
+
+  private def serializeShuffleReadMetricDistributions(
+      srmd: ShuffleReadMetricDistributions): StoreTypes.ShuffleReadMetricDistributions = {
+    val builder = StoreTypes.ShuffleReadMetricDistributions.newBuilder()
+    srmd.readBytes.foreach(rb => builder.addReadBytes(rb))
+    srmd.readRecords.foreach(rr => builder.addReadRecords(rr))
+    srmd.remoteBlocksFetched.foreach(rbf => builder.addRemoteBlocksFetched(rbf))
+    srmd.localBlocksFetched.foreach(lbf => builder.addLocalBlocksFetched(lbf))
+    srmd.fetchWaitTime.foreach(fwt => builder.addFetchWaitTime(fwt))
+    srmd.remoteBytesRead.foreach(rbr => builder.addRemoteBytesRead(rbr))
+    srmd.remoteBytesReadToDisk.foreach(rbrtd => builder.addRemoteBytesReadToDisk(rbrtd))
+    srmd.totalBlocksFetched.foreach(tbf => builder.addTotalBlocksFetched(tbf))
+    builder.build()
+  }
+
+  private def serializeShuffleWriteMetricDistributions(
+      swmd: ShuffleWriteMetricDistributions): StoreTypes.ShuffleWriteMetricDistributions = {
+    val builder = StoreTypes.ShuffleWriteMetricDistributions.newBuilder()
+    swmd.writeBytes.foreach(wb => builder.addWriteBytes(wb))
+    swmd.writeRecords.foreach(wr => builder.addWriteRecords(wr))
+    swmd.writeTime.foreach(wt => builder.addWriteTime(wt))
+    builder.build()
+  }
+
+  private def serializeExecutorMetricsDistributions(
+      emd: ExecutorMetricsDistributions): StoreTypes.ExecutorMetricsDistributions = {
+    val builder = StoreTypes.ExecutorMetricsDistributions.newBuilder()
+    emd.quantiles.foreach(q => builder.addQuantiles(q))
+    emd.taskTime.foreach(tt => builder.addTaskTime(tt))
+    emd.failedTasks.foreach(ft => builder.addFailedTasks(ft))
+    emd.succeededTasks.foreach(st => builder.addSucceededTasks(st))
+    emd.killedTasks.foreach(kt => builder.addKilledTasks(kt))
+    emd.inputBytes.foreach(ib => builder.addInputBytes(ib))
+    emd.inputRecords.foreach(ir => builder.addInputRecords(ir))
+    emd.outputBytes.foreach(ob => builder.addOutputBytes(ob))
+    emd.outputRecords.foreach(or => builder.addOutputRecords(or))
+    emd.shuffleRead.foreach(sr => builder.addShuffleRead(sr))
+    emd.shuffleReadRecords.foreach(srr => builder.addShuffleReadRecords(srr))
+    emd.shuffleWrite.foreach(sw => builder.addShuffleWrite(sw))
+    emd.shuffleWriteRecords.foreach(swr => builder.addShuffleWriteRecords(swr))
+    emd.memoryBytesSpilled.foreach(mbs => builder.addMemoryBytesSpilled(mbs))
+    emd.diskBytesSpilled.foreach(dbs => builder.addDiskBytesSpilled(dbs))
+    builder.setPeakMemoryMetrics(serializeExecutorPeakMetricsDistributions(emd.peakMemoryMetrics))
+    builder.build()
+  }
+
+  private def serializeExecutorPeakMetricsDistributions(
+      epmd: ExecutorPeakMetricsDistributions): StoreTypes.ExecutorPeakMetricsDistributions = {
+    val builder = StoreTypes.ExecutorPeakMetricsDistributions.newBuilder()
+    epmd.quantiles.foreach(q => builder.addQuantiles(q))
+    epmd.executorMetrics.foreach(em => builder.addExecutorMetrics(
+      ExecutorMetricsSerializer.serialize(em)))
+    builder.build()
+  }
+
+  override def deserialize(bytes: Array[Byte]): StageDataWrapper = {
+    val binary = StoreTypes.StageDataWrapper.parseFrom(bytes)
+    val info = deserializeStageData(binary.getInfo)
+    new StageDataWrapper(
+      info = info,
+      jobIds = binary.getJobIdsList.asScala.map(_.toInt).toSet,
+      locality = binary.getLocalityMap.asScala.mapValues(_.toLong).toMap
+    )
+  }
+
+  private def deserializeStageData(binary: StoreTypes.StageData): StageData = {
+    val status = StageStatus.valueOf(binary.getStatus.toString)
+    val submissionTime =
+      getOptional(binary.hasSubmissionTime, () => new Date(binary.getSubmissionTime))
+    val firstTaskLaunchedTime =
+      getOptional(binary.hasFirstTaskLaunchedTime, () => new Date(binary.getFirstTaskLaunchedTime))
+    val completionTime =
+      getOptional(binary.hasCompletionTime, () => new Date(binary.getCompletionTime))
+    val failureReason =
+      getOptional(binary.hasFailureReason, () => weakIntern(binary.getFailureReason))
+    val description =
+      getOptional(binary.hasDescription, () => weakIntern(binary.getDescription))
+    val accumulatorUpdates = AccumulableInfoSerializer.deserializeAccumulableInfos(
+      binary.getAccumulatorUpdatesList)
+    val tasks = MapUtils.isEmpty(binary.getTasksMap) match {
+      case true => None
+      case _ => Some(binary.getTasksMap.asScala.map(
+        entry => (entry._1.toLong, deserializeTaskData(entry._2))).toMap)
+    }
+    val executorSummary = MapUtils.isEmpty(binary.getExecutorSummaryMap) match {
+      case true => None
+      case _ => Some(binary.getExecutorSummaryMap.asScala.mapValues(
+          ExecutorStageSummarySerializer.deserializeExecutorStageSummary(_)).toMap

Review Comment:
   Done



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
panbingkun commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1058002785


##########
core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto:
##########
@@ -390,3 +390,214 @@ message SQLExecutionUIData {
   repeated int64 stages = 11;
   map<int64, string> metric_values = 12;
 }
+
+message StageDataWrapper {
+  StageData info = 1;
+  repeated int64 job_ids = 2;
+  map<string, int64> locality = 3;
+}
+
+message TaskData {
+  int64 task_id = 1;
+  int32 index = 2;
+  int32 attempt = 3;
+  int32 partition_id = 4;
+  int64 launch_time = 5;
+  optional int64 result_fetch_start = 6;
+  optional int64 duration = 7;
+  string executor_id = 8;
+  string host = 9;
+  string status = 10;
+  string task_locality = 11;
+  bool speculative = 12;
+  repeated AccumulableInfo accumulator_updates = 13;
+  optional string error_message = 14;
+  optional TaskMetrics task_metrics = 15;
+  map<string, string> executor_logs = 16;
+  int64 scheduler_delay = 17;
+  int64 getting_result_time = 18;
+}
+
+message StageData {
+  enum StageStatus {

Review Comment:
   Similarly, the enum definition of JobExecutionStatus seems more reasonable in JobData ? 



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
panbingkun commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1058801116


##########
core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto:
##########
@@ -390,3 +390,214 @@ message SQLExecutionUIData {
   repeated int64 stages = 11;
   map<int64, string> metric_values = 12;
 }
+
+message StageDataWrapper {
+  StageData info = 1;
+  repeated int64 job_ids = 2;
+  map<string, int64> locality = 3;
+}
+
+message TaskData {
+  int64 task_id = 1;
+  int32 index = 2;
+  int32 attempt = 3;
+  int32 partition_id = 4;
+  int64 launch_time = 5;
+  optional int64 result_fetch_start = 6;
+  optional int64 duration = 7;
+  string executor_id = 8;
+  string host = 9;
+  string status = 10;
+  string task_locality = 11;
+  bool speculative = 12;
+  repeated AccumulableInfo accumulator_updates = 13;
+  optional string error_message = 14;
+  optional TaskMetrics task_metrics = 15;
+  map<string, string> executor_logs = 16;
+  int64 scheduler_delay = 17;
+  int64 getting_result_time = 18;
+}
+
+message StageData {
+  enum StageStatus {

Review Comment:
   A.If the enum definition of `JobExecutionStatus` in JobData
   <img width="224" alt="image" src="https://user-images.githubusercontent.com/15246973/209922039-617b3993-b6c2-48dc-a24f-fdb343585c68.png">
   
   SQLExecutionUIData can use it as follows:
   <img width="414" alt="image" src="https://user-images.githubusercontent.com/15246973/209922101-a2504145-87a3-46f7-b918-6d4363773d17.png">
   
   B.Other choices for the definition of `JobExecutionStatus` as follows:
   <img width="233" alt="image" src="https://user-images.githubusercontent.com/15246973/209922683-296a5c81-daa8-48ad-8bd2-8ea4682953e5.png">
   <img width="699" alt="image" src="https://user-images.githubusercontent.com/15246973/209922716-51945000-6654-4283-affb-37e2dbd5978e.png">
   <img width="390" alt="image" src="https://user-images.githubusercontent.com/15246973/209922853-d0d1affa-6c94-454a-883a-b87551c7ddb0.png">
   
   Otherwise, I'm afraid there are other enumeration names that will conflict(eg: StageStatus.FAILED). @gengliangwang @LuciferYang 



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
panbingkun commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1059217541


##########
core/src/test/scala/org/apache/spark/status/protobuf/KVStoreProtobufSerializerSuite.scala:
##########
@@ -773,4 +740,568 @@ class KVStoreProtobufSerializerSuite extends SparkFunSuite {
       assert(result.info.processLogs(k) == input.info.processLogs(k))
     }
   }
+
+  test("Stage Data") {
+    val accumulatorUpdates = Seq(
+      new AccumulableInfo(1L, "duration", Some("update"), "value1"),
+      new AccumulableInfo(2L, "duration2", None, "value2")
+    )
+    val inputMetrics = new InputMetrics(
+      bytesRead = 1L,
+      recordsRead = 2L)
+    val outputMetrics = new OutputMetrics(
+      bytesWritten = 1L,
+      recordsWritten = 2L
+    )
+    val shuffleReadMetrics = new ShuffleReadMetrics(
+      remoteBlocksFetched = 1L,
+      localBlocksFetched = 2L,
+      fetchWaitTime = 3L,
+      remoteBytesRead = 4L,
+      remoteBytesReadToDisk = 5L,
+      localBytesRead = 6L,
+      recordsRead = 7L
+    )
+    val shuffleWriteMetrics = new ShuffleWriteMetrics(
+      bytesWritten = 1L,
+      writeTime = 2L,
+      recordsWritten = 3L
+    )
+    val taskMetrics = new TaskMetrics(
+      executorDeserializeTime = 1L,
+      executorDeserializeCpuTime = 2L,
+      executorRunTime = 3L,
+      executorCpuTime = 4L,
+      resultSize = 5L,
+      jvmGcTime = 6L,
+      resultSerializationTime = 7L,
+      memoryBytesSpilled = 8L,
+      diskBytesSpilled = 9L,
+      peakExecutionMemory = 10L,
+      inputMetrics = inputMetrics,
+      outputMetrics = outputMetrics,
+      shuffleReadMetrics = shuffleReadMetrics,
+      shuffleWriteMetrics = shuffleWriteMetrics
+    )
+    val taskData1 = new TaskData(
+      taskId = 1L,
+      index = 2,
+      attempt = 3,
+      partitionId = 4,
+      launchTime = new Date(123456L),
+      resultFetchStart = Some(new Date(223456L)),
+      duration = Some(10000L),
+      executorId = "executor_id_1",
+      host = "host_name_1",
+      status = "SUCCESS",
+      taskLocality = "LOCAL",
+      speculative = true,
+      accumulatorUpdates = accumulatorUpdates,
+      errorMessage = Some("error_1"),
+      taskMetrics = Some(taskMetrics),
+      executorLogs = Map("executor_id_1" -> "executor_log_1"),
+      schedulerDelay = 5L,
+      gettingResultTime = 6L
+    )
+    val taskData2 = new TaskData(
+      taskId = 11L,
+      index = 12,
+      attempt = 13,
+      partitionId = 14,
+      launchTime = new Date(1123456L),
+      resultFetchStart = Some(new Date(1223456L)),
+      duration = Some(110000L),
+      executorId = "executor_id_2",
+      host = "host_name_2",
+      status = "SUCCESS",
+      taskLocality = "LOCAL",
+      speculative = false,
+      accumulatorUpdates = accumulatorUpdates,
+      errorMessage = Some("error_2"),
+      taskMetrics = Some(taskMetrics),
+      executorLogs = Map("executor_id_2" -> "executor_log_2"),
+      schedulerDelay = 15L,
+      gettingResultTime = 16L
+    )
+    val tasks = Some(
+      Map(1L -> taskData1, 2L -> taskData2)
+    )
+    val peakMemoryMetrics =
+      Some(new ExecutorMetrics(Array(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 1024L)))
+    val executorStageSummary1 = new ExecutorStageSummary(
+      taskTime = 1L,
+      failedTasks = 2,
+      succeededTasks = 3,
+      killedTasks = 4,
+      inputBytes = 5L,
+      inputRecords = 6L,
+      outputBytes = 7L,
+      outputRecords = 8L,
+      shuffleRead = 9L,
+      shuffleReadRecords = 10L,
+      shuffleWrite = 11L,
+      shuffleWriteRecords = 12L,
+      memoryBytesSpilled = 13L,
+      diskBytesSpilled = 14L,
+      isBlacklistedForStage = true,
+      peakMemoryMetrics = peakMemoryMetrics,
+      isExcludedForStage = false)
+    val executorStageSummary2 = new ExecutorStageSummary(
+      taskTime = 11L,
+      failedTasks = 12,
+      succeededTasks = 13,
+      killedTasks = 14,
+      inputBytes = 15L,
+      inputRecords = 16L,
+      outputBytes = 17L,
+      outputRecords = 18L,
+      shuffleRead = 19L,
+      shuffleReadRecords = 110L,
+      shuffleWrite = 111L,
+      shuffleWriteRecords = 112L,
+      memoryBytesSpilled = 113L,
+      diskBytesSpilled = 114L,
+      isBlacklistedForStage = false,
+      peakMemoryMetrics = peakMemoryMetrics,
+      isExcludedForStage = true)
+    val executorSummary = Some(
+      Map("executor_id_1" -> executorStageSummary1, "executor_id_2" -> executorStageSummary2)
+    )
+    val speculationStageSummary = new SpeculationStageSummary(
+      numTasks = 3,
+      numActiveTasks = 4,
+      numCompletedTasks = 5,
+      numFailedTasks = 6,
+      numKilledTasks = 7
+    )
+    val inputMetricDistributions = new InputMetricDistributions(
+      bytesRead = IndexedSeq(1.001D, 2.001D),
+      recordsRead = IndexedSeq(3.001D, 4.001D)
+    )
+    val outputMetricDistributions = new OutputMetricDistributions(
+      bytesWritten = IndexedSeq(1.001D, 2.001D),
+      recordsWritten = IndexedSeq(3.001D, 4.001D)
+    )
+    val shuffleReadMetricDistributions = new ShuffleReadMetricDistributions(
+      readBytes = IndexedSeq(1.001D, 2.001D),
+      readRecords = IndexedSeq(3.001D, 4.001D),
+      remoteBlocksFetched = IndexedSeq(5.001D, 6.001D),
+      localBlocksFetched = IndexedSeq(7.001D, 8.001D),
+      fetchWaitTime = IndexedSeq(9.001D, 10.001D),
+      remoteBytesRead = IndexedSeq(11.001D, 12.001D),
+      remoteBytesReadToDisk = IndexedSeq(13.001D, 14.001D),
+      totalBlocksFetched = IndexedSeq(15.001D, 16.001D)
+    )
+    val shuffleWriteMetricDistributions = new ShuffleWriteMetricDistributions(
+      writeBytes = IndexedSeq(1.001D, 2.001D),
+      writeRecords = IndexedSeq(3.001D, 4.001D),
+      writeTime = IndexedSeq(5.001D, 6.001D)
+    )
+    val taskMetricDistributions = new TaskMetricDistributions(
+      quantiles = IndexedSeq(1.001D, 2.001D),
+      duration = IndexedSeq(3.001D, 4.001D),
+      executorDeserializeTime = IndexedSeq(5.001D, 6.001D),
+      executorDeserializeCpuTime = IndexedSeq(7.001D, 8.001D),
+      executorRunTime = IndexedSeq(9.001D, 10.001D),
+      executorCpuTime = IndexedSeq(11.001D, 12.001D),
+      resultSize = IndexedSeq(13.001D, 14.001D),
+      jvmGcTime = IndexedSeq(15.001D, 16.001D),
+      resultSerializationTime = IndexedSeq(17.001D, 18.001D),
+      gettingResultTime = IndexedSeq(19.001D, 20.001D),
+      schedulerDelay = IndexedSeq(21.001D, 22.001D),
+      peakExecutionMemory = IndexedSeq(23.001D, 24.001D),
+      memoryBytesSpilled = IndexedSeq(25.001D, 26.001D),
+      diskBytesSpilled = IndexedSeq(27.001D, 28.001D),
+      inputMetrics = inputMetricDistributions,
+      outputMetrics = outputMetricDistributions,
+      shuffleReadMetrics = shuffleReadMetricDistributions,
+      shuffleWriteMetrics = shuffleWriteMetricDistributions
+    )
+    val executorPeakMetricsDistributions = new ExecutorPeakMetricsDistributions(
+      quantiles = IndexedSeq(1.001D, 2.001D),
+      executorMetrics = IndexedSeq(
+        new ExecutorMetrics(Array(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 1024L)))
+    )
+    val executorMetricsDistributions = new ExecutorMetricsDistributions(
+      quantiles = IndexedSeq(1.001D, 2.001D),
+      taskTime = IndexedSeq(3.001D, 4.001D),
+      failedTasks = IndexedSeq(5.001D, 6.001D),
+      succeededTasks = IndexedSeq(7.001D, 8.001D),
+      killedTasks = IndexedSeq(9.001D, 10.001D),
+      inputBytes = IndexedSeq(11.001D, 12.001D),
+      inputRecords = IndexedSeq(13.001D, 14.001D),
+      outputBytes = IndexedSeq(15.001D, 16.001D),
+      outputRecords = IndexedSeq(17.001D, 18.001D),
+      shuffleRead = IndexedSeq(19.001D, 20.001D),
+      shuffleReadRecords = IndexedSeq(21.001D, 22.001D),
+      shuffleWrite = IndexedSeq(23.001D, 24.001D),
+      shuffleWriteRecords = IndexedSeq(25.001D, 24.001D),
+      memoryBytesSpilled = IndexedSeq(27.001D, 28.001D),
+      diskBytesSpilled = IndexedSeq(29.001D, 30.001D),
+      peakMemoryMetrics = executorPeakMetricsDistributions
+    )
+    val info = new StageData(
+      status = StageStatus.COMPLETE,
+      stageId = 1,
+      attemptId = 2,
+      numTasks = 3,
+      numActiveTasks = 4,
+      numCompleteTasks = 5,
+      numFailedTasks = 6,
+      numKilledTasks = 7,
+      numCompletedIndices = 8,
+      submissionTime = Some(new Date(123456L)),
+      firstTaskLaunchedTime = Some(new Date(234567L)),
+      completionTime = Some(new Date(654321L)),
+      failureReason = Some("failure reason"),
+      executorDeserializeTime = 9L,
+      executorDeserializeCpuTime = 10L,
+      executorRunTime = 11L,
+      executorCpuTime = 12L,
+      resultSize = 13L,
+      jvmGcTime = 14L,
+      resultSerializationTime = 15L,
+      memoryBytesSpilled = 16L,
+      diskBytesSpilled = 17L,
+      peakExecutionMemory = 18L,
+      inputBytes = 19L,
+      inputRecords = 20L,
+      outputBytes = 21L,
+      outputRecords = 22L,
+      shuffleRemoteBlocksFetched = 23L,
+      shuffleLocalBlocksFetched = 24L,
+      shuffleFetchWaitTime = 25L,
+      shuffleRemoteBytesRead = 26L,
+      shuffleRemoteBytesReadToDisk = 27L,
+      shuffleLocalBytesRead = 28L,
+      shuffleReadBytes = 29L,
+      shuffleReadRecords = 30L,
+      shuffleWriteBytes = 31L,
+      shuffleWriteTime = 32L,
+      shuffleWriteRecords = 33L,
+      name = "name",
+      description = Some("test description"),
+      details = "test details",
+      schedulingPool = "test scheduling pool",
+      rddIds = Seq(1, 2, 3, 4, 5, 6),
+      accumulatorUpdates = accumulatorUpdates,
+      tasks = tasks,
+      executorSummary = executorSummary,
+      speculationSummary = Some(speculationStageSummary),
+      killedTasksSummary = Map("task_1" -> 1),
+      resourceProfileId = 34,
+      peakExecutorMetrics = peakMemoryMetrics,
+      taskMetricsDistributions = Some(taskMetricDistributions),
+      executorMetricsDistributions = Some(executorMetricsDistributions)
+    )
+    val input = new StageDataWrapper(
+      info = info,
+      jobIds = Set(1, 2, 3, 4),
+      locality = Map(
+        "PROCESS_LOCAL" -> 1L,
+        "NODE_LOCAL" -> 2L
+      )
+    )
+
+    val bytes = serializer.serialize(input)
+    val result = serializer.deserialize(bytes, classOf[StageDataWrapper])
+
+    assert(result.jobIds == input.jobIds)
+    assert(result.locality == input.locality)
+
+    assert(result.info.status == input.info.status)
+    assert(result.info.stageId == input.info.stageId)
+    assert(result.info.attemptId == input.info.attemptId)
+    assert(result.info.numTasks == input.info.numTasks)
+    assert(result.info.numActiveTasks == input.info.numActiveTasks)
+    assert(result.info.numCompleteTasks == input.info.numCompleteTasks)
+    assert(result.info.numFailedTasks == input.info.numFailedTasks)
+    assert(result.info.numKilledTasks == input.info.numKilledTasks)
+    assert(result.info.numCompletedIndices == input.info.numCompletedIndices)
+
+    assert(result.info.submissionTime == input.info.submissionTime)
+    assert(result.info.firstTaskLaunchedTime == input.info.firstTaskLaunchedTime)
+    assert(result.info.completionTime == input.info.completionTime)
+    assert(result.info.failureReason == input.info.failureReason)
+
+    assert(result.info.executorDeserializeTime == input.info.executorDeserializeTime)
+    assert(result.info.executorDeserializeCpuTime == input.info.executorDeserializeCpuTime)
+    assert(result.info.executorRunTime == input.info.executorRunTime)
+    assert(result.info.executorCpuTime == input.info.executorCpuTime)
+    assert(result.info.resultSize == input.info.resultSize)
+    assert(result.info.jvmGcTime == input.info.jvmGcTime)
+    assert(result.info.resultSerializationTime == input.info.resultSerializationTime)
+    assert(result.info.memoryBytesSpilled == input.info.memoryBytesSpilled)
+    assert(result.info.diskBytesSpilled == input.info.diskBytesSpilled)
+    assert(result.info.peakExecutionMemory == input.info.peakExecutionMemory)
+    assert(result.info.inputBytes == input.info.inputBytes)
+    assert(result.info.inputRecords == input.info.inputRecords)
+    assert(result.info.outputBytes == input.info.outputBytes)
+    assert(result.info.outputRecords == input.info.outputRecords)
+    assert(result.info.shuffleRemoteBlocksFetched == input.info.shuffleRemoteBlocksFetched)
+    assert(result.info.shuffleLocalBlocksFetched == input.info.shuffleLocalBlocksFetched)
+    assert(result.info.shuffleFetchWaitTime == input.info.shuffleFetchWaitTime)
+    assert(result.info.shuffleRemoteBytesRead == input.info.shuffleRemoteBytesRead)
+    assert(result.info.shuffleRemoteBytesReadToDisk == input.info.shuffleRemoteBytesReadToDisk)
+    assert(result.info.shuffleLocalBytesRead == input.info.shuffleLocalBytesRead)
+    assert(result.info.shuffleReadBytes == input.info.shuffleReadBytes)
+    assert(result.info.shuffleReadRecords == input.info.shuffleReadRecords)
+    assert(result.info.shuffleWriteBytes == input.info.shuffleWriteBytes)
+    assert(result.info.shuffleWriteTime == input.info.shuffleWriteTime)
+    assert(result.info.shuffleWriteRecords == input.info.shuffleWriteRecords)
+
+    assert(result.info.name == input.info.name)
+    assert(result.info.description == input.info.description)
+    assert(result.info.details == input.info.details)
+    assert(result.info.schedulingPool == input.info.schedulingPool)
+
+    assert(result.info.rddIds == input.info.rddIds)
+    assert(result.info.accumulatorUpdates, input.info.accumulatorUpdates)
+
+    assert(result.info.tasks.isDefined == input.info.tasks.isDefined)
+    if (result.info.tasks.isDefined && input.info.tasks.isDefined) {
+      assertIdTask(result.info.tasks.get, input.info.tasks.get)
+    }
+
+    assert(result.info.executorSummary.isDefined == input.info.executorSummary.isDefined)
+    if (result.info.executorSummary.isDefined && input.info.executorSummary.isDefined) {
+      assert(result.info.executorSummary.get, input.info.executorSummary.get)
+    }
+
+    assert(result.info.speculationSummary.isDefined == input.info.speculationSummary.isDefined)
+    if (result.info.speculationSummary.isDefined && input.info.speculationSummary.isDefined) {
+      assert(result.info.speculationSummary.get, input.info.speculationSummary.get)
+    }
+    assert(result.info.killedTasksSummary == input.info.killedTasksSummary)
+    assert(result.info.resourceProfileId == input.info.resourceProfileId)
+    assert(result.info.peakExecutorMetrics.isDefined == input.info.peakExecutorMetrics.isDefined)
+    if (result.info.peakExecutorMetrics.isDefined && input.info.peakExecutorMetrics.isDefined) {
+      assert(result.info.peakExecutorMetrics.get, input.info.peakExecutorMetrics.get)
+    }
+    assert(result.info.taskMetricsDistributions.isDefined ==
+      input.info.taskMetricsDistributions.isDefined)
+    if (result.info.taskMetricsDistributions.isDefined &&
+      input.info.taskMetricsDistributions.isDefined) {
+      assert(result.info.taskMetricsDistributions.get, input.info.taskMetricsDistributions.get)
+    }
+    assert(result.info.executorMetricsDistributions.isDefined ==
+      input.info.executorMetricsDistributions.isDefined)
+    if (result.info.executorMetricsDistributions.isDefined &&
+      input.info.executorMetricsDistributions.isDefined) {
+      assert(result.info.executorMetricsDistributions.get,
+        input.info.executorMetricsDistributions.get)
+    }
+  }
+
+  private def assert(result: TaskMetrics, input: TaskMetrics): Unit = {

Review Comment:
   Done



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] LuciferYang commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1058797949


##########
core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto:
##########
@@ -390,3 +390,214 @@ message SQLExecutionUIData {
   repeated int64 stages = 11;
   map<int64, string> metric_values = 12;
 }
+
+message StageDataWrapper {
+  StageData info = 1;
+  repeated int64 job_ids = 2;
+  map<string, int64> locality = 3;
+}
+
+message TaskData {
+  int64 task_id = 1;
+  int32 index = 2;
+  int32 attempt = 3;
+  int32 partition_id = 4;
+  int64 launch_time = 5;
+  optional int64 result_fetch_start = 6;
+  optional int64 duration = 7;
+  string executor_id = 8;
+  string host = 9;
+  string status = 10;
+  string task_locality = 11;
+  bool speculative = 12;
+  repeated AccumulableInfo accumulator_updates = 13;
+  optional string error_message = 14;
+  optional TaskMetrics task_metrics = 15;
+  map<string, string> executor_logs = 16;
+  int64 scheduler_delay = 17;
+  int64 getting_result_time = 18;
+}
+
+message StageData {
+  enum StageStatus {

Review Comment:
   As described in  https://github.com/apache/spark/pull/39270/files, `UNSPECIFIED` in `StageStatus`  should change to `STAGE_STATUS_UNSPECIFIED` and moved out of `StageData`
   
   
   
   



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
panbingkun commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1058801116


##########
core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto:
##########
@@ -390,3 +390,214 @@ message SQLExecutionUIData {
   repeated int64 stages = 11;
   map<int64, string> metric_values = 12;
 }
+
+message StageDataWrapper {
+  StageData info = 1;
+  repeated int64 job_ids = 2;
+  map<string, int64> locality = 3;
+}
+
+message TaskData {
+  int64 task_id = 1;
+  int32 index = 2;
+  int32 attempt = 3;
+  int32 partition_id = 4;
+  int64 launch_time = 5;
+  optional int64 result_fetch_start = 6;
+  optional int64 duration = 7;
+  string executor_id = 8;
+  string host = 9;
+  string status = 10;
+  string task_locality = 11;
+  bool speculative = 12;
+  repeated AccumulableInfo accumulator_updates = 13;
+  optional string error_message = 14;
+  optional TaskMetrics task_metrics = 15;
+  map<string, string> executor_logs = 16;
+  int64 scheduler_delay = 17;
+  int64 getting_result_time = 18;
+}
+
+message StageData {
+  enum StageStatus {

Review Comment:
   A.If the enum definition of `JobExecutionStatus` in JobData
   <img width="224" alt="image" src="https://user-images.githubusercontent.com/15246973/209922039-617b3993-b6c2-48dc-a24f-fdb343585c68.png">
   
   SQLExecutionUIData can use it as follows:
   <img width="414" alt="image" src="https://user-images.githubusercontent.com/15246973/209922101-a2504145-87a3-46f7-b918-6d4363773d17.png">
   
   B.Other choices for the definition of `JobExecutionStatus` as follows:
   <img width="233" alt="image" src="https://user-images.githubusercontent.com/15246973/209922683-296a5c81-daa8-48ad-8bd2-8ea4682953e5.png">
   <img width="699" alt="image" src="https://user-images.githubusercontent.com/15246973/209922716-51945000-6654-4283-affb-37e2dbd5978e.png">
   <img width="390" alt="image" src="https://user-images.githubusercontent.com/15246973/209922853-d0d1affa-6c94-454a-883a-b87551c7ddb0.png">
   
   Otherwise, I'm afraid there are other enumeration names that will conflict. @gengliangwang 



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
panbingkun commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1056822093


##########
core/src/main/scala/org/apache/spark/status/protobuf/StageDataWrapperSerializer.scala:
##########
@@ -0,0 +1,628 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.Date
+
+import collection.JavaConverters._
+
+import org.apache.commons.collections4.MapUtils
+
+import org.apache.spark.status.StageDataWrapper
+import org.apache.spark.status.api.v1.{ExecutorMetricsDistributions, ExecutorPeakMetricsDistributions, InputMetricDistributions, InputMetrics, OutputMetricDistributions, OutputMetrics, ShuffleReadMetricDistributions, ShuffleReadMetrics, ShuffleWriteMetricDistributions, ShuffleWriteMetrics, SpeculationStageSummary, StageData, StageStatus, TaskData, TaskMetricDistributions, TaskMetrics}
+import org.apache.spark.status.protobuf.Utils.getOptional
+import org.apache.spark.util.Utils.weakIntern
+
+class StageDataWrapperSerializer extends ProtobufSerDe {
+
+  override val supportClass: Class[_] = classOf[StageDataWrapper]
+
+  override def serialize(input: Any): Array[Byte] =
+    serialize(input.asInstanceOf[StageDataWrapper])
+
+  private def serialize(s: StageDataWrapper): Array[Byte] = {
+    val builder = StoreTypes.StageDataWrapper.newBuilder()
+    builder.setInfo(serializeStageData(s.info))
+    s.jobIds.foreach(id => builder.addJobIds(id.toLong))
+    s.locality.foreach { entry =>
+      builder.putLocality(entry._1, entry._2)
+    }
+    builder.build().toByteArray
+  }
+
+  private def serializeStageData(stageData: StageData): StoreTypes.StageData = {
+    val stageDataBuilder = StoreTypes.StageData.newBuilder()
+    stageDataBuilder
+      .setStatus(serializeStageStatus(stageData.status))
+      .setStageId(stageData.stageId.toLong)
+      .setAttemptId(stageData.attemptId)
+      .setNumTasks(stageData.numTasks)
+      .setNumActiveTasks(stageData.numActiveTasks)
+      .setNumCompleteTasks(stageData.numCompleteTasks)
+      .setNumFailedTasks(stageData.numFailedTasks)
+      .setNumKilledTasks(stageData.numKilledTasks)
+      .setNumCompletedIndices(stageData.numCompletedIndices)
+      .setExecutorDeserializeTime(stageData.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(stageData.executorDeserializeCpuTime)
+      .setExecutorRunTime(stageData.executorRunTime)
+      .setExecutorCpuTime(stageData.executorCpuTime)
+      .setResultSize(stageData.resultSize)
+      .setJvmGcTime(stageData.jvmGcTime)
+      .setResultSerializationTime(stageData.resultSerializationTime)
+      .setMemoryBytesSpilled(stageData.memoryBytesSpilled)
+      .setDiskBytesSpilled(stageData.diskBytesSpilled)
+      .setPeakExecutionMemory(stageData.peakExecutionMemory)
+      .setInputBytes(stageData.inputBytes)
+      .setInputRecords(stageData.inputRecords)
+      .setOutputBytes(stageData.outputBytes)
+      .setOutputRecords(stageData.outputRecords)
+      .setShuffleRemoteBlocksFetched(stageData.shuffleRemoteBlocksFetched)
+      .setShuffleLocalBlocksFetched(stageData.shuffleLocalBlocksFetched)
+      .setShuffleFetchWaitTime(stageData.shuffleFetchWaitTime)
+      .setShuffleRemoteBytesRead(stageData.shuffleRemoteBytesRead)
+      .setShuffleRemoteBytesReadToDisk(stageData.shuffleRemoteBytesReadToDisk)
+      .setShuffleLocalBytesRead(stageData.shuffleLocalBytesRead)
+      .setShuffleReadBytes(stageData.shuffleReadBytes)
+      .setShuffleReadRecords(stageData.shuffleReadRecords)
+      .setShuffleWriteBytes(stageData.shuffleWriteBytes)
+      .setShuffleWriteTime(stageData.shuffleWriteTime)
+      .setShuffleWriteRecords(stageData.shuffleWriteRecords)
+      .setName(stageData.name)
+      .setDetails(stageData.details)
+      .setSchedulingPool(stageData.schedulingPool)
+      .setResourceProfileId(stageData.resourceProfileId)
+
+    stageData.submissionTime.foreach { d =>
+      stageDataBuilder.setSubmissionTime(d.getTime)
+    }
+    stageData.firstTaskLaunchedTime.foreach { d =>
+      stageDataBuilder.setFirstTaskLaunchedTime(d.getTime)
+    }
+    stageData.completionTime.foreach { d =>
+      stageDataBuilder.setCompletionTime(d.getTime)
+    }
+    stageData.failureReason.foreach { fr =>
+      stageDataBuilder.setFailureReason(fr)
+    }
+    stageData.description.foreach { d =>
+      stageDataBuilder.setDescription(d)
+    }
+    stageData.rddIds.foreach(id => stageDataBuilder.addRddIds(id.toLong))
+    stageData.accumulatorUpdates.foreach { update =>
+      stageDataBuilder.addAccumulatorUpdates(Utils.serializeAccumulableInfo(update))
+    }
+    stageData.tasks.foreach { t =>
+      t.foreach { entry =>
+        stageDataBuilder.putTasks(entry._1, serializeTaskData(entry._2))
+      }
+    }
+    stageData.executorSummary.foreach { es =>
+      es.foreach { entry =>
+        stageDataBuilder.putExecutorSummary(entry._1, Utils.serializeExecutorStageSummary(entry._2))
+      }
+    }
+    stageData.speculationSummary.foreach { ss =>
+      stageDataBuilder.setSpeculationSummary(serializeSpeculationStageSummary(ss))
+    }
+    stageData.killedTasksSummary.foreach { entry =>
+      stageDataBuilder.putKilledTasksSummary(entry._1, entry._2)
+    }
+    stageData.peakExecutorMetrics.foreach { pem =>
+      stageDataBuilder.setPeakExecutorMetrics(ExecutorMetricsSerializer.serialize(pem))
+    }
+    stageData.taskMetricsDistributions.foreach { tmd =>
+      stageDataBuilder.setTaskMetricsDistributions(serializeTaskMetricDistributions(tmd))
+    }
+    stageData.executorMetricsDistributions.foreach { emd =>
+      stageDataBuilder.setExecutorMetricsDistributions(serializeExecutorMetricsDistributions(emd))
+    }
+
+    stageDataBuilder.build()
+  }
+
+  private def serializeStageStatus(s: StageStatus): StoreTypes.StageData.StageStatus = {
+    StoreTypes.StageData.StageStatus.valueOf(s.toString)
+  }
+
+  private def serializeTaskData(t: TaskData): StoreTypes.TaskData = {
+    val taskDataBuilder = StoreTypes.TaskData.newBuilder()
+    taskDataBuilder
+      .setTaskId(t.taskId)
+      .setIndex(t.index)
+      .setAttempt(t.attempt)
+      .setPartitionId(t.partitionId)
+      .setLaunchTime(t.launchTime.getTime)
+      .setExecutorId(t.executorId)
+      .setHost(t.host)
+      .setStatus(t.status)
+      .setTaskLocality(t.taskLocality)
+      .setSpeculative(t.speculative)
+      .setSchedulerDelay(t.schedulerDelay)
+      .setGettingResultTime(t.gettingResultTime)
+
+    t.resultFetchStart.foreach { rfs =>
+      taskDataBuilder.setResultFetchStart(rfs.getTime)
+    }
+    t.duration.foreach { d =>
+      taskDataBuilder.setDuration(d)
+    }
+    t.accumulatorUpdates.foreach { update =>
+      taskDataBuilder.addAccumulatorUpdates(Utils.serializeAccumulableInfo(update))
+    }
+    t.errorMessage.foreach { em =>
+      taskDataBuilder.setErrorMessage(em)
+    }
+    t.taskMetrics.foreach { tm =>
+      taskDataBuilder.setTaskMetrics(serializeTaskMetrics(tm))
+    }
+    t.executorLogs.foreach { entry =>
+      taskDataBuilder.putExecutorLogs(entry._1, entry._2)
+    }
+
+    taskDataBuilder.build()
+  }
+
+  private def serializeTaskMetrics(tm: TaskMetrics): StoreTypes.TaskMetrics = {
+    val taskMetricsBuilder = StoreTypes.TaskMetrics.newBuilder()
+    taskMetricsBuilder
+      .setExecutorDeserializeTime(tm.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(tm.executorDeserializeCpuTime)
+      .setExecutorRunTime(tm.executorRunTime)
+      .setExecutorCpuTime(tm.executorCpuTime)
+      .setResultSize(tm.resultSize)
+      .setJvmGcTime(tm.jvmGcTime)
+      .setResultSerializationTime(tm.resultSerializationTime)
+      .setMemoryBytesSpilled(tm.memoryBytesSpilled)
+      .setDiskBytesSpilled(tm.diskBytesSpilled)
+      .setPeakExecutionMemory(tm.peakExecutionMemory)
+      .setInputMetrics(serializeInputMetrics(tm.inputMetrics))
+      .setOutputMetrics(serializeOutputMetrics(tm.outputMetrics))
+      .setShuffleReadMetrics(serializeShuffleReadMetrics(tm.shuffleReadMetrics))
+      .setShuffleWriteMetrics(serializeShuffleWriteMetrics(tm.shuffleWriteMetrics))
+    taskMetricsBuilder.build()
+  }
+
+  private def serializeInputMetrics(im: InputMetrics): StoreTypes.InputMetrics = {
+    StoreTypes.InputMetrics.newBuilder()
+      .setBytesRead(im.bytesRead)
+      .setRecordsRead(im.recordsRead)
+      .build()
+  }
+
+  private def serializeOutputMetrics(om: OutputMetrics): StoreTypes.OutputMetrics = {
+    StoreTypes.OutputMetrics.newBuilder()
+      .setBytesWritten(om.bytesWritten)
+      .setRecordsWritten(om.recordsWritten)
+      .build()
+  }
+
+  private def serializeShuffleReadMetrics(
+      srm: ShuffleReadMetrics): StoreTypes.ShuffleReadMetrics = {
+    StoreTypes.ShuffleReadMetrics.newBuilder()
+      .setRemoteBlocksFetched(srm.remoteBlocksFetched)
+      .setLocalBlocksFetched(srm.localBlocksFetched)
+      .setFetchWaitTime(srm.fetchWaitTime)
+      .setRemoteBytesRead(srm.remoteBytesRead)
+      .setRemoteBytesReadToDisk(srm.remoteBytesReadToDisk)
+      .setLocalBytesRead(srm.localBytesRead)
+      .setRecordsRead(srm.recordsRead)
+      .build()
+  }
+
+  private def serializeShuffleWriteMetrics(
+      swm: ShuffleWriteMetrics): StoreTypes.ShuffleWriteMetrics = {
+    StoreTypes.ShuffleWriteMetrics.newBuilder()
+      .setBytesWritten(swm.bytesWritten)
+      .setWriteTime(swm.writeTime)
+      .setRecordsWritten(swm.recordsWritten)
+      .build()
+  }
+
+  private def serializeSpeculationStageSummary(
+      sss: SpeculationStageSummary): StoreTypes.SpeculationStageSummary = {
+    StoreTypes.SpeculationStageSummary.newBuilder()
+      .setNumTasks(sss.numTasks)
+      .setNumActiveTasks(sss.numActiveTasks)
+      .setNumCompletedTasks(sss.numCompletedTasks)
+      .setNumFailedTasks(sss.numFailedTasks)
+      .setNumKilledTasks(sss.numKilledTasks)
+      .build()
+  }
+
+  private def serializeTaskMetricDistributions(
+      tmd: TaskMetricDistributions): StoreTypes.TaskMetricDistributions = {
+    val builder = StoreTypes.TaskMetricDistributions.newBuilder()
+    tmd.quantiles.foreach(q => builder.addQuantiles(q))
+    tmd.duration.foreach(d => builder.addDuration(d))
+    tmd.executorDeserializeTime.foreach(edt => builder.addExecutorDeserializeTime(edt))
+    tmd.executorDeserializeCpuTime.foreach(edct => builder.addExecutorDeserializeCpuTime(edct))
+    tmd.executorRunTime.foreach(ert => builder.addExecutorRunTime(ert))
+    tmd.executorCpuTime.foreach(ect => builder.addExecutorCpuTime(ect))
+    tmd.resultSize.foreach(rs => builder.addResultSize(rs))
+    tmd.jvmGcTime.foreach(jgt => builder.addJvmGcTime(jgt))
+    tmd.resultSerializationTime.foreach(rst => builder.addResultSerializationTime(rst))
+    tmd.gettingResultTime.foreach(grt => builder.addGettingResultTime(grt))
+    tmd.schedulerDelay.foreach(sd => builder.addSchedulerDelay(sd))
+    tmd.peakExecutionMemory.foreach(pem => builder.addPeakExecutionMemory(pem))
+    tmd.memoryBytesSpilled.foreach(mbs => builder.addMemoryBytesSpilled(mbs))
+    tmd.diskBytesSpilled.foreach(dbs => builder.addDiskBytesSpilled(dbs))
+    builder
+      .setInputMetrics(serializeInputMetricDistributions(tmd.inputMetrics))
+      .setOutputMetrics(serializeOutputMetricDistributions(tmd.outputMetrics))
+      .setShuffleReadMetrics(serializeShuffleReadMetricDistributions(tmd.shuffleReadMetrics))
+      .setShuffleWriteMetrics(serializeShuffleWriteMetricDistributions(tmd.shuffleWriteMetrics))
+      .build()
+  }
+
+  private def serializeInputMetricDistributions(
+      imd: InputMetricDistributions): StoreTypes.InputMetricDistributions = {
+    val builder = StoreTypes.InputMetricDistributions.newBuilder()
+    imd.bytesRead.foreach(br => builder.addBytesRead(br))
+    imd.recordsRead.foreach(rr => builder.addRecordsRead(rr))
+    builder.build()
+  }
+
+  private def serializeOutputMetricDistributions(
+      omd: OutputMetricDistributions): StoreTypes.OutputMetricDistributions = {
+    val builder = StoreTypes.OutputMetricDistributions.newBuilder()
+    omd.bytesWritten.foreach(bw => builder.addBytesWritten(bw))
+    omd.recordsWritten.foreach(rw => builder.addRecordsWritten(rw))
+    builder.build()
+  }
+
+  private def serializeShuffleReadMetricDistributions(
+      srmd: ShuffleReadMetricDistributions): StoreTypes.ShuffleReadMetricDistributions = {
+    val builder = StoreTypes.ShuffleReadMetricDistributions.newBuilder()
+    srmd.readBytes.foreach(rb => builder.addReadBytes(rb))
+    srmd.readRecords.foreach(rr => builder.addReadRecords(rr))
+    srmd.remoteBlocksFetched.foreach(rbf => builder.addRemoteBlocksFetched(rbf))
+    srmd.localBlocksFetched.foreach(lbf => builder.addLocalBlocksFetched(lbf))
+    srmd.fetchWaitTime.foreach(fwt => builder.addFetchWaitTime(fwt))
+    srmd.remoteBytesRead.foreach(rbr => builder.addRemoteBytesRead(rbr))
+    srmd.remoteBytesReadToDisk.foreach(rbrtd => builder.addRemoteBytesReadToDisk(rbrtd))
+    srmd.totalBlocksFetched.foreach(tbf => builder.addTotalBlocksFetched(tbf))
+    builder.build()
+  }
+
+  private def serializeShuffleWriteMetricDistributions(
+      swmd: ShuffleWriteMetricDistributions): StoreTypes.ShuffleWriteMetricDistributions = {
+    val builder = StoreTypes.ShuffleWriteMetricDistributions.newBuilder()
+    swmd.writeBytes.foreach(wb => builder.addWriteBytes(wb))
+    swmd.writeRecords.foreach(wr => builder.addWriteRecords(wr))
+    swmd.writeTime.foreach(wt => builder.addWriteTime(wt))
+    builder.build()
+  }
+
+  private def serializeExecutorMetricsDistributions(
+      emd: ExecutorMetricsDistributions): StoreTypes.ExecutorMetricsDistributions = {
+    val builder = StoreTypes.ExecutorMetricsDistributions.newBuilder()
+    emd.quantiles.foreach(q => builder.addQuantiles(q))
+    emd.taskTime.foreach(tt => builder.addTaskTime(tt))
+    emd.failedTasks.foreach(ft => builder.addFailedTasks(ft))
+    emd.succeededTasks.foreach(st => builder.addSucceededTasks(st))
+    emd.killedTasks.foreach(kt => builder.addKilledTasks(kt))
+    emd.inputBytes.foreach(ib => builder.addInputBytes(ib))
+    emd.inputRecords.foreach(ir => builder.addInputRecords(ir))
+    emd.outputBytes.foreach(ob => builder.addOutputBytes(ob))
+    emd.outputRecords.foreach(or => builder.addOutputRecords(or))
+    emd.shuffleRead.foreach(sr => builder.addShuffleRead(sr))
+    emd.shuffleReadRecords.foreach(srr => builder.addShuffleReadRecords(srr))
+    emd.shuffleWrite.foreach(sw => builder.addShuffleWrite(sw))
+    emd.shuffleWriteRecords.foreach(swr => builder.addShuffleWriteRecords(swr))
+    emd.memoryBytesSpilled.foreach(mbs => builder.addMemoryBytesSpilled(mbs))
+    emd.diskBytesSpilled.foreach(dbs => builder.addDiskBytesSpilled(dbs))
+    builder.setPeakMemoryMetrics(serializeExecutorPeakMetricsDistributions(emd.peakMemoryMetrics))
+    builder.build()
+  }
+
+  private def serializeExecutorPeakMetricsDistributions(
+      epmd: ExecutorPeakMetricsDistributions): StoreTypes.ExecutorPeakMetricsDistributions = {
+    val builder = StoreTypes.ExecutorPeakMetricsDistributions.newBuilder()
+    epmd.quantiles.foreach(q => builder.addQuantiles(q))
+    epmd.executorMetrics.foreach(em => builder.addExecutorMetrics(
+      ExecutorMetricsSerializer.serialize(em)))
+    builder.build()
+  }
+
+  override def deserialize(bytes: Array[Byte]): StageDataWrapper = {
+    val binary = StoreTypes.StageDataWrapper.parseFrom(bytes)
+    val info = deserializeStageData(binary.getInfo)
+    new StageDataWrapper(
+      info = info,
+      jobIds = binary.getJobIdsList.asScala.map(_.toInt).toSet,
+      locality = binary.getLocalityMap.asScala.mapValues(_.toLong).toMap
+    )
+  }
+
+  private def deserializeStageData(binary: StoreTypes.StageData): StageData = {
+    val status = StageStatus.valueOf(binary.getStatus.toString)
+    val submissionTime =
+      getOptional(binary.hasSubmissionTime, () => new Date(binary.getSubmissionTime))
+    val firstTaskLaunchedTime =
+      getOptional(binary.hasFirstTaskLaunchedTime, () => new Date(binary.getFirstTaskLaunchedTime))
+    val completionTime =
+      getOptional(binary.hasCompletionTime, () => new Date(binary.getCompletionTime))
+    val failureReason =
+      getOptional(binary.hasFailureReason, () => weakIntern(binary.getFailureReason))
+    val description =
+      getOptional(binary.hasDescription, () => weakIntern(binary.getDescription))
+    val accumulatorUpdates = Utils.deserializeAccumulableInfos(binary.getAccumulatorUpdatesList)
+    val tasks = MapUtils.isEmpty(binary.getTasksMap) match {

Review Comment:
   `optional map` is not supported by pb



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
panbingkun commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1056822038


##########
core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto:
##########
@@ -390,3 +390,214 @@ message SQLExecutionUIData {
   repeated int64 stages = 11;
   map<int64, string> metric_values = 12;
 }
+
+message StageDataWrapper {
+  StageData info = 1;
+  repeated int64 job_ids = 2;
+  map<string, int64> locality = 3;
+}
+
+message TaskData {
+  int64 task_id = 1;
+  int32 index = 2;
+  int32 attempt = 3;
+  int32 partition_id = 4;
+  int64 launch_time = 5;
+  optional int64 result_fetch_start = 6;
+  optional int64 duration = 7;
+  string executor_id = 8;
+  string host = 9;
+  string status = 10;
+  string task_locality = 11;
+  bool speculative = 12;
+  repeated AccumulableInfo accumulator_updates = 13;
+  optional string error_message = 14;
+  optional TaskMetrics task_metrics = 15;
+  map<string, string> executor_logs = 16;
+  int64 scheduler_delay = 17;
+  int64 getting_result_time = 18;
+}
+
+message StageData {
+  enum StageStatus {
+    UNSPECIFIED = 0;
+    ACTIVE = 1;
+    COMPLETE = 2;
+    FAILED = 3;
+    PENDING = 4;
+    SKIPPED = 5;
+  }
+
+  StageStatus status = 1;
+  int64 stage_id = 2;
+  int32 attempt_id = 3;
+  int32 num_tasks = 4;
+  int32 num_active_tasks = 5;
+  int32 num_complete_tasks = 6;
+  int32 num_failed_tasks = 7;
+  int32 num_killed_tasks = 8;
+  int32 num_completed_indices = 9;
+
+  optional int64 submission_time = 10;
+  optional int64 first_task_launched_time = 11;
+  optional int64 completion_time = 12;
+  optional string failure_reason = 13;
+
+  int64 executor_deserialize_time = 14;
+  int64 executor_deserialize_cpu_time = 15;
+  int64 executor_run_time = 16;
+  int64 executor_cpu_time = 17;
+  int64 result_size = 18;
+  int64 jvm_gc_time = 19;
+  int64 result_serialization_time = 20;
+  int64 memory_bytes_spilled = 21;
+  int64 disk_bytes_spilled = 22;
+  int64 peak_execution_memory = 23;
+  int64 input_bytes = 24;
+  int64 input_records = 25;
+  int64 output_bytes = 26;
+  int64 output_records = 27;
+  int64 shuffle_remote_blocks_fetched = 28;
+  int64 shuffle_local_blocks_fetched = 29;
+  int64 shuffle_fetch_wait_time = 30;
+  int64 shuffle_remote_bytes_read = 31;
+  int64 shuffle_remote_bytes_read_to_disk = 32;
+  int64 shuffle_local_bytes_read = 33;
+  int64 shuffle_read_bytes = 34;
+  int64 shuffle_read_records = 35;
+  int64 shuffle_write_bytes = 36;
+  int64 shuffle_write_time = 37;
+  int64 shuffle_write_records = 38;
+
+  string name = 39;
+  optional string description = 40;
+  string details = 41;
+  string scheduling_pool = 42;
+
+  repeated int64 rdd_ids = 43;
+  repeated AccumulableInfo accumulator_updates = 44;
+  map<int64, TaskData> tasks = 45;

Review Comment:
   `optional map` is not supported by pb



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
panbingkun commented on PR #39192:
URL: https://github.com/apache/spark/pull/39192#issuecomment-1364821718

   cc @gengliangwang @LuciferYang 


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
panbingkun commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1058301782


##########
core/src/main/scala/org/apache/spark/status/protobuf/StageDataWrapperSerializer.scala:
##########
@@ -0,0 +1,627 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.Date
+
+import collection.JavaConverters._
+import org.apache.commons.collections4.MapUtils
+
+import org.apache.spark.status.StageDataWrapper
+import org.apache.spark.status.api.v1.{ExecutorMetricsDistributions, ExecutorPeakMetricsDistributions, InputMetricDistributions, InputMetrics, OutputMetricDistributions, OutputMetrics, ShuffleReadMetricDistributions, ShuffleReadMetrics, ShuffleWriteMetricDistributions, ShuffleWriteMetrics, SpeculationStageSummary, StageData, StageStatus, TaskData, TaskMetricDistributions, TaskMetrics}
+import org.apache.spark.status.protobuf.Utils.getOptional
+import org.apache.spark.util.Utils.weakIntern
+
+class StageDataWrapperSerializer extends ProtobufSerDe {
+
+  override val supportClass: Class[_] = classOf[StageDataWrapper]
+
+  override def serialize(input: Any): Array[Byte] =
+    serialize(input.asInstanceOf[StageDataWrapper])
+
+  private def serialize(s: StageDataWrapper): Array[Byte] = {
+    val builder = StoreTypes.StageDataWrapper.newBuilder()
+    builder.setInfo(serializeStageData(s.info))
+    s.jobIds.foreach(id => builder.addJobIds(id.toLong))
+    s.locality.foreach { entry =>
+      builder.putLocality(entry._1, entry._2)
+    }
+    builder.build().toByteArray
+  }
+
+  private def serializeStageData(stageData: StageData): StoreTypes.StageData = {
+    val stageDataBuilder = StoreTypes.StageData.newBuilder()
+    stageDataBuilder
+      .setStatus(serializeStageStatus(stageData.status))
+      .setStageId(stageData.stageId.toLong)
+      .setAttemptId(stageData.attemptId)
+      .setNumTasks(stageData.numTasks)
+      .setNumActiveTasks(stageData.numActiveTasks)
+      .setNumCompleteTasks(stageData.numCompleteTasks)
+      .setNumFailedTasks(stageData.numFailedTasks)
+      .setNumKilledTasks(stageData.numKilledTasks)
+      .setNumCompletedIndices(stageData.numCompletedIndices)
+      .setExecutorDeserializeTime(stageData.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(stageData.executorDeserializeCpuTime)
+      .setExecutorRunTime(stageData.executorRunTime)
+      .setExecutorCpuTime(stageData.executorCpuTime)
+      .setResultSize(stageData.resultSize)
+      .setJvmGcTime(stageData.jvmGcTime)
+      .setResultSerializationTime(stageData.resultSerializationTime)
+      .setMemoryBytesSpilled(stageData.memoryBytesSpilled)
+      .setDiskBytesSpilled(stageData.diskBytesSpilled)
+      .setPeakExecutionMemory(stageData.peakExecutionMemory)
+      .setInputBytes(stageData.inputBytes)
+      .setInputRecords(stageData.inputRecords)
+      .setOutputBytes(stageData.outputBytes)
+      .setOutputRecords(stageData.outputRecords)
+      .setShuffleRemoteBlocksFetched(stageData.shuffleRemoteBlocksFetched)
+      .setShuffleLocalBlocksFetched(stageData.shuffleLocalBlocksFetched)
+      .setShuffleFetchWaitTime(stageData.shuffleFetchWaitTime)
+      .setShuffleRemoteBytesRead(stageData.shuffleRemoteBytesRead)
+      .setShuffleRemoteBytesReadToDisk(stageData.shuffleRemoteBytesReadToDisk)
+      .setShuffleLocalBytesRead(stageData.shuffleLocalBytesRead)
+      .setShuffleReadBytes(stageData.shuffleReadBytes)
+      .setShuffleReadRecords(stageData.shuffleReadRecords)
+      .setShuffleWriteBytes(stageData.shuffleWriteBytes)
+      .setShuffleWriteTime(stageData.shuffleWriteTime)
+      .setShuffleWriteRecords(stageData.shuffleWriteRecords)
+      .setName(stageData.name)
+      .setDetails(stageData.details)
+      .setSchedulingPool(stageData.schedulingPool)
+      .setResourceProfileId(stageData.resourceProfileId)
+    stageData.submissionTime.foreach { d =>
+      stageDataBuilder.setSubmissionTime(d.getTime)
+    }
+    stageData.firstTaskLaunchedTime.foreach { d =>
+      stageDataBuilder.setFirstTaskLaunchedTime(d.getTime)
+    }
+    stageData.completionTime.foreach { d =>
+      stageDataBuilder.setCompletionTime(d.getTime)
+    }
+    stageData.failureReason.foreach { fr =>
+      stageDataBuilder.setFailureReason(fr)
+    }
+    stageData.description.foreach { d =>
+      stageDataBuilder.setDescription(d)
+    }
+    stageData.rddIds.foreach(id => stageDataBuilder.addRddIds(id.toLong))
+    stageData.accumulatorUpdates.foreach { update =>
+      stageDataBuilder.addAccumulatorUpdates(
+        AccumulableInfoSerializer.serializeAccumulableInfo(update))
+    }
+    stageData.tasks.foreach { t =>
+      t.foreach { entry =>
+        stageDataBuilder.putTasks(entry._1, serializeTaskData(entry._2))
+      }
+    }
+    stageData.executorSummary.foreach { es =>
+      es.foreach { entry =>
+        stageDataBuilder.putExecutorSummary(entry._1,
+          ExecutorStageSummarySerializer.serializeExecutorStageSummary(entry._2))
+      }
+    }
+    stageData.speculationSummary.foreach { ss =>
+      stageDataBuilder.setSpeculationSummary(serializeSpeculationStageSummary(ss))
+    }
+    stageData.killedTasksSummary.foreach { entry =>
+      stageDataBuilder.putKilledTasksSummary(entry._1, entry._2)
+    }
+    stageData.peakExecutorMetrics.foreach { pem =>
+      stageDataBuilder.setPeakExecutorMetrics(ExecutorMetricsSerializer.serialize(pem))
+    }
+    stageData.taskMetricsDistributions.foreach { tmd =>
+      stageDataBuilder.setTaskMetricsDistributions(serializeTaskMetricDistributions(tmd))
+    }
+    stageData.executorMetricsDistributions.foreach { emd =>
+      stageDataBuilder.setExecutorMetricsDistributions(serializeExecutorMetricsDistributions(emd))
+    }
+    stageDataBuilder.build()
+  }
+
+  private def serializeStageStatus(s: StageStatus): StoreTypes.StageData.StageStatus = {
+    StoreTypes.StageData.StageStatus.valueOf(s.toString)
+  }
+
+  private def serializeTaskData(t: TaskData): StoreTypes.TaskData = {
+    val taskDataBuilder = StoreTypes.TaskData.newBuilder()
+    taskDataBuilder
+      .setTaskId(t.taskId)
+      .setIndex(t.index)
+      .setAttempt(t.attempt)
+      .setPartitionId(t.partitionId)
+      .setLaunchTime(t.launchTime.getTime)
+      .setExecutorId(t.executorId)
+      .setHost(t.host)
+      .setStatus(t.status)
+      .setTaskLocality(t.taskLocality)
+      .setSpeculative(t.speculative)
+      .setSchedulerDelay(t.schedulerDelay)
+      .setGettingResultTime(t.gettingResultTime)
+    t.resultFetchStart.foreach { rfs =>
+      taskDataBuilder.setResultFetchStart(rfs.getTime)
+    }
+    t.duration.foreach { d =>
+      taskDataBuilder.setDuration(d)
+    }
+    t.accumulatorUpdates.foreach { update =>
+      taskDataBuilder.addAccumulatorUpdates(
+        AccumulableInfoSerializer.serializeAccumulableInfo(update))
+    }
+    t.errorMessage.foreach { em =>
+      taskDataBuilder.setErrorMessage(em)
+    }
+    t.taskMetrics.foreach { tm =>
+      taskDataBuilder.setTaskMetrics(serializeTaskMetrics(tm))
+    }
+    t.executorLogs.foreach { entry =>
+      taskDataBuilder.putExecutorLogs(entry._1, entry._2)
+    }
+    taskDataBuilder.build()
+  }
+
+  private def serializeTaskMetrics(tm: TaskMetrics): StoreTypes.TaskMetrics = {
+    val taskMetricsBuilder = StoreTypes.TaskMetrics.newBuilder()
+    taskMetricsBuilder
+      .setExecutorDeserializeTime(tm.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(tm.executorDeserializeCpuTime)
+      .setExecutorRunTime(tm.executorRunTime)
+      .setExecutorCpuTime(tm.executorCpuTime)
+      .setResultSize(tm.resultSize)
+      .setJvmGcTime(tm.jvmGcTime)
+      .setResultSerializationTime(tm.resultSerializationTime)
+      .setMemoryBytesSpilled(tm.memoryBytesSpilled)
+      .setDiskBytesSpilled(tm.diskBytesSpilled)
+      .setPeakExecutionMemory(tm.peakExecutionMemory)
+      .setInputMetrics(serializeInputMetrics(tm.inputMetrics))
+      .setOutputMetrics(serializeOutputMetrics(tm.outputMetrics))
+      .setShuffleReadMetrics(serializeShuffleReadMetrics(tm.shuffleReadMetrics))
+      .setShuffleWriteMetrics(serializeShuffleWriteMetrics(tm.shuffleWriteMetrics))
+    taskMetricsBuilder.build()
+  }
+
+  private def serializeInputMetrics(im: InputMetrics): StoreTypes.InputMetrics = {
+    StoreTypes.InputMetrics.newBuilder()
+      .setBytesRead(im.bytesRead)
+      .setRecordsRead(im.recordsRead)
+      .build()
+  }
+
+  private def serializeOutputMetrics(om: OutputMetrics): StoreTypes.OutputMetrics = {
+    StoreTypes.OutputMetrics.newBuilder()
+      .setBytesWritten(om.bytesWritten)
+      .setRecordsWritten(om.recordsWritten)
+      .build()
+  }
+
+  private def serializeShuffleReadMetrics(
+      srm: ShuffleReadMetrics): StoreTypes.ShuffleReadMetrics = {
+    StoreTypes.ShuffleReadMetrics.newBuilder()
+      .setRemoteBlocksFetched(srm.remoteBlocksFetched)
+      .setLocalBlocksFetched(srm.localBlocksFetched)
+      .setFetchWaitTime(srm.fetchWaitTime)
+      .setRemoteBytesRead(srm.remoteBytesRead)
+      .setRemoteBytesReadToDisk(srm.remoteBytesReadToDisk)
+      .setLocalBytesRead(srm.localBytesRead)
+      .setRecordsRead(srm.recordsRead)
+      .build()
+  }
+
+  private def serializeShuffleWriteMetrics(
+      swm: ShuffleWriteMetrics): StoreTypes.ShuffleWriteMetrics = {
+    StoreTypes.ShuffleWriteMetrics.newBuilder()
+      .setBytesWritten(swm.bytesWritten)
+      .setWriteTime(swm.writeTime)
+      .setRecordsWritten(swm.recordsWritten)
+      .build()
+  }
+
+  private def serializeSpeculationStageSummary(
+      sss: SpeculationStageSummary): StoreTypes.SpeculationStageSummary = {
+    StoreTypes.SpeculationStageSummary.newBuilder()
+      .setNumTasks(sss.numTasks)
+      .setNumActiveTasks(sss.numActiveTasks)
+      .setNumCompletedTasks(sss.numCompletedTasks)
+      .setNumFailedTasks(sss.numFailedTasks)
+      .setNumKilledTasks(sss.numKilledTasks)
+      .build()
+  }
+
+  private def serializeTaskMetricDistributions(
+      tmd: TaskMetricDistributions): StoreTypes.TaskMetricDistributions = {
+    val builder = StoreTypes.TaskMetricDistributions.newBuilder()
+    tmd.quantiles.foreach(q => builder.addQuantiles(q))
+    tmd.duration.foreach(d => builder.addDuration(d))
+    tmd.executorDeserializeTime.foreach(edt => builder.addExecutorDeserializeTime(edt))
+    tmd.executorDeserializeCpuTime.foreach(edct => builder.addExecutorDeserializeCpuTime(edct))
+    tmd.executorRunTime.foreach(ert => builder.addExecutorRunTime(ert))
+    tmd.executorCpuTime.foreach(ect => builder.addExecutorCpuTime(ect))
+    tmd.resultSize.foreach(rs => builder.addResultSize(rs))
+    tmd.jvmGcTime.foreach(jgt => builder.addJvmGcTime(jgt))
+    tmd.resultSerializationTime.foreach(rst => builder.addResultSerializationTime(rst))
+    tmd.gettingResultTime.foreach(grt => builder.addGettingResultTime(grt))
+    tmd.schedulerDelay.foreach(sd => builder.addSchedulerDelay(sd))
+    tmd.peakExecutionMemory.foreach(pem => builder.addPeakExecutionMemory(pem))
+    tmd.memoryBytesSpilled.foreach(mbs => builder.addMemoryBytesSpilled(mbs))
+    tmd.diskBytesSpilled.foreach(dbs => builder.addDiskBytesSpilled(dbs))
+    builder
+      .setInputMetrics(serializeInputMetricDistributions(tmd.inputMetrics))
+      .setOutputMetrics(serializeOutputMetricDistributions(tmd.outputMetrics))
+      .setShuffleReadMetrics(serializeShuffleReadMetricDistributions(tmd.shuffleReadMetrics))
+      .setShuffleWriteMetrics(serializeShuffleWriteMetricDistributions(tmd.shuffleWriteMetrics))
+      .build()
+  }
+
+  private def serializeInputMetricDistributions(
+      imd: InputMetricDistributions): StoreTypes.InputMetricDistributions = {
+    val builder = StoreTypes.InputMetricDistributions.newBuilder()
+    imd.bytesRead.foreach(br => builder.addBytesRead(br))
+    imd.recordsRead.foreach(rr => builder.addRecordsRead(rr))
+    builder.build()
+  }
+
+  private def serializeOutputMetricDistributions(
+      omd: OutputMetricDistributions): StoreTypes.OutputMetricDistributions = {
+    val builder = StoreTypes.OutputMetricDistributions.newBuilder()
+    omd.bytesWritten.foreach(bw => builder.addBytesWritten(bw))
+    omd.recordsWritten.foreach(rw => builder.addRecordsWritten(rw))
+    builder.build()
+  }
+
+  private def serializeShuffleReadMetricDistributions(
+      srmd: ShuffleReadMetricDistributions): StoreTypes.ShuffleReadMetricDistributions = {
+    val builder = StoreTypes.ShuffleReadMetricDistributions.newBuilder()
+    srmd.readBytes.foreach(rb => builder.addReadBytes(rb))
+    srmd.readRecords.foreach(rr => builder.addReadRecords(rr))
+    srmd.remoteBlocksFetched.foreach(rbf => builder.addRemoteBlocksFetched(rbf))
+    srmd.localBlocksFetched.foreach(lbf => builder.addLocalBlocksFetched(lbf))
+    srmd.fetchWaitTime.foreach(fwt => builder.addFetchWaitTime(fwt))
+    srmd.remoteBytesRead.foreach(rbr => builder.addRemoteBytesRead(rbr))
+    srmd.remoteBytesReadToDisk.foreach(rbrtd => builder.addRemoteBytesReadToDisk(rbrtd))
+    srmd.totalBlocksFetched.foreach(tbf => builder.addTotalBlocksFetched(tbf))
+    builder.build()
+  }
+
+  private def serializeShuffleWriteMetricDistributions(
+      swmd: ShuffleWriteMetricDistributions): StoreTypes.ShuffleWriteMetricDistributions = {
+    val builder = StoreTypes.ShuffleWriteMetricDistributions.newBuilder()
+    swmd.writeBytes.foreach(wb => builder.addWriteBytes(wb))
+    swmd.writeRecords.foreach(wr => builder.addWriteRecords(wr))
+    swmd.writeTime.foreach(wt => builder.addWriteTime(wt))
+    builder.build()
+  }
+
+  private def serializeExecutorMetricsDistributions(
+      emd: ExecutorMetricsDistributions): StoreTypes.ExecutorMetricsDistributions = {
+    val builder = StoreTypes.ExecutorMetricsDistributions.newBuilder()
+    emd.quantiles.foreach(q => builder.addQuantiles(q))
+    emd.taskTime.foreach(tt => builder.addTaskTime(tt))
+    emd.failedTasks.foreach(ft => builder.addFailedTasks(ft))
+    emd.succeededTasks.foreach(st => builder.addSucceededTasks(st))
+    emd.killedTasks.foreach(kt => builder.addKilledTasks(kt))
+    emd.inputBytes.foreach(ib => builder.addInputBytes(ib))
+    emd.inputRecords.foreach(ir => builder.addInputRecords(ir))
+    emd.outputBytes.foreach(ob => builder.addOutputBytes(ob))
+    emd.outputRecords.foreach(or => builder.addOutputRecords(or))
+    emd.shuffleRead.foreach(sr => builder.addShuffleRead(sr))
+    emd.shuffleReadRecords.foreach(srr => builder.addShuffleReadRecords(srr))
+    emd.shuffleWrite.foreach(sw => builder.addShuffleWrite(sw))
+    emd.shuffleWriteRecords.foreach(swr => builder.addShuffleWriteRecords(swr))
+    emd.memoryBytesSpilled.foreach(mbs => builder.addMemoryBytesSpilled(mbs))
+    emd.diskBytesSpilled.foreach(dbs => builder.addDiskBytesSpilled(dbs))
+    builder.setPeakMemoryMetrics(serializeExecutorPeakMetricsDistributions(emd.peakMemoryMetrics))
+    builder.build()
+  }
+
+  private def serializeExecutorPeakMetricsDistributions(
+      epmd: ExecutorPeakMetricsDistributions): StoreTypes.ExecutorPeakMetricsDistributions = {
+    val builder = StoreTypes.ExecutorPeakMetricsDistributions.newBuilder()
+    epmd.quantiles.foreach(q => builder.addQuantiles(q))
+    epmd.executorMetrics.foreach(em => builder.addExecutorMetrics(
+      ExecutorMetricsSerializer.serialize(em)))
+    builder.build()
+  }
+
+  override def deserialize(bytes: Array[Byte]): StageDataWrapper = {
+    val binary = StoreTypes.StageDataWrapper.parseFrom(bytes)
+    val info = deserializeStageData(binary.getInfo)
+    new StageDataWrapper(
+      info = info,
+      jobIds = binary.getJobIdsList.asScala.map(_.toInt).toSet,
+      locality = binary.getLocalityMap.asScala.mapValues(_.toLong).toMap
+    )
+  }
+
+  private def deserializeStageData(binary: StoreTypes.StageData): StageData = {
+    val status = StageStatus.valueOf(binary.getStatus.toString)
+    val submissionTime =
+      getOptional(binary.hasSubmissionTime, () => new Date(binary.getSubmissionTime))
+    val firstTaskLaunchedTime =
+      getOptional(binary.hasFirstTaskLaunchedTime, () => new Date(binary.getFirstTaskLaunchedTime))
+    val completionTime =
+      getOptional(binary.hasCompletionTime, () => new Date(binary.getCompletionTime))
+    val failureReason =
+      getOptional(binary.hasFailureReason, () => weakIntern(binary.getFailureReason))
+    val description =
+      getOptional(binary.hasDescription, () => weakIntern(binary.getDescription))
+    val accumulatorUpdates = AccumulableInfoSerializer.deserializeAccumulableInfos(
+      binary.getAccumulatorUpdatesList)
+    val tasks = MapUtils.isEmpty(binary.getTasksMap) match {
+      case true => None
+      case _ => Some(binary.getTasksMap.asScala.map(
+        entry => (entry._1.toLong, deserializeTaskData(entry._2))).toMap)
+    }
+    val executorSummary = MapUtils.isEmpty(binary.getExecutorSummaryMap) match {
+      case true => None
+      case _ => Some(binary.getExecutorSummaryMap.asScala.mapValues(
+          ExecutorStageSummarySerializer.deserializeExecutorStageSummary(_)).toMap
+      )
+    }
+    val speculationSummary =
+      getOptional(binary.hasSpeculationSummary,
+        () => deserializeSpeculationStageSummary(binary.getSpeculationSummary))
+    val peakExecutorMetrics =
+      getOptional(binary.hasPeakExecutorMetrics,
+        () => ExecutorMetricsSerializer.deserialize(binary.getPeakExecutorMetrics))
+    val taskMetricsDistributions =
+      getOptional(binary.hasTaskMetricsDistributions,
+        () => deserializeTaskMetricDistributions(binary.getTaskMetricsDistributions))
+    val executorMetricsDistributions =
+      getOptional(binary.hasExecutorMetricsDistributions,
+        () => deserializeExecutorMetricsDistributions(binary.getExecutorMetricsDistributions))
+    new StageData(
+      status = status,
+      stageId = binary.getStageId.toInt,
+      attemptId = binary.getAttemptId,
+      numTasks = binary.getNumTasks,
+      numActiveTasks = binary.getNumActiveTasks,
+      numCompleteTasks = binary.getNumCompleteTasks,
+      numFailedTasks = binary.getNumFailedTasks,
+      numKilledTasks = binary.getNumKilledTasks,
+      numCompletedIndices = binary.getNumCompletedIndices,
+      submissionTime = submissionTime,
+      firstTaskLaunchedTime = firstTaskLaunchedTime,
+      completionTime = completionTime,
+      failureReason = failureReason,
+      executorDeserializeTime = binary.getExecutorDeserializeTime,
+      executorDeserializeCpuTime = binary.getExecutorDeserializeCpuTime,
+      executorRunTime = binary.getExecutorRunTime,
+      executorCpuTime = binary.getExecutorCpuTime,
+      resultSize = binary.getResultSize,
+      jvmGcTime = binary.getJvmGcTime,
+      resultSerializationTime = binary.getResultSerializationTime,
+      memoryBytesSpilled = binary.getMemoryBytesSpilled,
+      diskBytesSpilled = binary.getDiskBytesSpilled,
+      peakExecutionMemory = binary.getPeakExecutionMemory,
+      inputBytes = binary.getInputBytes,
+      inputRecords = binary.getInputRecords,
+      outputBytes = binary.getOutputBytes,
+      outputRecords = binary.getOutputRecords,
+      shuffleRemoteBlocksFetched = binary.getShuffleRemoteBlocksFetched,
+      shuffleLocalBlocksFetched = binary.getShuffleLocalBlocksFetched,
+      shuffleFetchWaitTime = binary.getShuffleFetchWaitTime,
+      shuffleRemoteBytesRead = binary.getShuffleRemoteBytesRead,
+      shuffleRemoteBytesReadToDisk = binary.getShuffleRemoteBytesReadToDisk,
+      shuffleLocalBytesRead = binary.getShuffleLocalBytesRead,
+      shuffleReadBytes = binary.getShuffleReadBytes,
+      shuffleReadRecords = binary.getShuffleReadRecords,
+      shuffleWriteBytes = binary.getShuffleWriteBytes,
+      shuffleWriteTime = binary.getShuffleWriteTime,
+      shuffleWriteRecords = binary.getShuffleWriteRecords,
+      name = weakIntern(binary.getName),
+      description = description,
+      details = weakIntern(binary.getDetails),
+      schedulingPool = weakIntern(binary.getSchedulingPool),
+      rddIds = binary.getRddIdsList.asScala.map(_.toInt).toSeq,
+      accumulatorUpdates = accumulatorUpdates.toSeq,
+      tasks = tasks,
+      executorSummary = executorSummary,
+      speculationSummary = speculationSummary,
+      killedTasksSummary = binary.getKilledTasksSummaryMap.asScala.mapValues(_.toInt).toMap,
+      resourceProfileId = binary.getResourceProfileId,
+      peakExecutorMetrics = peakExecutorMetrics,
+      taskMetricsDistributions = taskMetricsDistributions,
+      executorMetricsDistributions = executorMetricsDistributions
+    )
+  }
+
+  private def deserializeSpeculationStageSummary(
+      binary: StoreTypes.SpeculationStageSummary): SpeculationStageSummary = {
+    new SpeculationStageSummary(
+      binary.getNumTasks,
+      binary.getNumActiveTasks,
+      binary.getNumCompletedTasks,
+      binary.getNumFailedTasks,
+      binary.getNumKilledTasks
+    )
+  }
+
+  private def deserializeTaskMetricDistributions(
+      binary: StoreTypes.TaskMetricDistributions): TaskMetricDistributions = {
+    new TaskMetricDistributions(
+      quantiles = binary.getQuantilesList.asScala.map(_.toDouble).toIndexedSeq,
+      duration = binary.getDurationList.asScala.map(_.toDouble).toIndexedSeq,
+      executorDeserializeTime =
+        binary.getExecutorDeserializeTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      executorDeserializeCpuTime =
+        binary.getExecutorDeserializeCpuTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      executorRunTime = binary.getExecutorRunTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      executorCpuTime = binary.getExecutorCpuTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      resultSize = binary.getResultSizeList.asScala.map(_.toDouble).toIndexedSeq,
+      jvmGcTime = binary.getJvmGcTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      resultSerializationTime =
+        binary.getResultSerializationTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      gettingResultTime = binary.getGettingResultTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      schedulerDelay = binary.getSchedulerDelayList.asScala.map(_.toDouble).toIndexedSeq,
+      peakExecutionMemory = binary.getPeakExecutionMemoryList.asScala.map(_.toDouble).toIndexedSeq,
+      memoryBytesSpilled = binary.getMemoryBytesSpilledList.asScala.map(_.toDouble).toIndexedSeq,
+      diskBytesSpilled = binary.getDiskBytesSpilledList.asScala.map(_.toDouble).toIndexedSeq,
+      inputMetrics = deserializeInputMetricDistributions(binary.getInputMetrics),
+      outputMetrics = deserializeOutputMetricDistributions(binary.getOutputMetrics),
+      shuffleReadMetrics = deserializeShuffleReadMetricDistributions(binary.getShuffleReadMetrics),
+      shuffleWriteMetrics =
+        deserializeShuffleWriteMetricDistributions(binary.getShuffleWriteMetrics)
+    )
+  }
+
+  private def deserializeInputMetricDistributions(
+      binary: StoreTypes.InputMetricDistributions): InputMetricDistributions = {
+    new InputMetricDistributions(
+      bytesRead = binary.getBytesReadList.asScala.map(_.toDouble).toIndexedSeq,
+      recordsRead = binary.getRecordsReadList.asScala.map(_.toDouble).toIndexedSeq
+    )
+  }
+
+  private def deserializeOutputMetricDistributions(
+      binary: StoreTypes.OutputMetricDistributions): OutputMetricDistributions = {
+    new OutputMetricDistributions(
+      bytesWritten = binary.getBytesWrittenList.asScala.map(_.toDouble).toIndexedSeq,
+      recordsWritten = binary.getRecordsWrittenList.asScala.map(_.toDouble).toIndexedSeq
+    )
+  }
+
+  private def deserializeShuffleReadMetricDistributions(
+      binary: StoreTypes.ShuffleReadMetricDistributions): ShuffleReadMetricDistributions = {
+    new ShuffleReadMetricDistributions(
+      readBytes = binary.getReadBytesList.asScala.map(_.toDouble).toIndexedSeq,
+      readRecords = binary.getReadRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      remoteBlocksFetched = binary.getRemoteBlocksFetchedList.asScala.map(_.toDouble).toIndexedSeq,
+      localBlocksFetched = binary.getLocalBlocksFetchedList.asScala.map(_.toDouble).toIndexedSeq,
+      fetchWaitTime = binary.getFetchWaitTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      remoteBytesRead = binary.getRemoteBytesReadList.asScala.map(_.toDouble).toIndexedSeq,
+      remoteBytesReadToDisk =
+        binary.getRemoteBytesReadToDiskList.asScala.map(_.toDouble).toIndexedSeq,
+      totalBlocksFetched = binary.getTotalBlocksFetchedList.asScala.map(_.toDouble).toIndexedSeq
+    )
+  }
+
+  private def deserializeShuffleWriteMetricDistributions(
+      binary: StoreTypes.ShuffleWriteMetricDistributions): ShuffleWriteMetricDistributions = {
+    new ShuffleWriteMetricDistributions(
+      writeBytes = binary.getWriteBytesList.asScala.map(_.toDouble).toIndexedSeq,
+      writeRecords = binary.getWriteRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      writeTime = binary.getWriteTimeList.asScala.map(_.toDouble).toIndexedSeq
+    )
+  }
+
+  private def deserializeExecutorMetricsDistributions(
+      binary: StoreTypes.ExecutorMetricsDistributions): ExecutorMetricsDistributions = {
+    new ExecutorMetricsDistributions(
+      quantiles = binary.getQuantilesList.asScala.map(_.toDouble).toIndexedSeq,
+      taskTime = binary.getTaskTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      failedTasks = binary.getFailedTasksList.asScala.map(_.toDouble).toIndexedSeq,
+      succeededTasks = binary.getSucceededTasksList.asScala.map(_.toDouble).toIndexedSeq,
+      killedTasks = binary.getKilledTasksList.asScala.map(_.toDouble).toIndexedSeq,
+      inputBytes = binary.getInputBytesList.asScala.map(_.toDouble).toIndexedSeq,
+      inputRecords = binary.getInputRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      outputBytes = binary.getOutputBytesList.asScala.map(_.toDouble).toIndexedSeq,
+      outputRecords = binary.getOutputRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      shuffleRead = binary.getShuffleReadList.asScala.map(_.toDouble).toIndexedSeq,
+      shuffleReadRecords = binary.getShuffleReadRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      shuffleWrite = binary.getShuffleWriteList.asScala.map(_.toDouble).toIndexedSeq,
+      shuffleWriteRecords = binary.getShuffleWriteRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      memoryBytesSpilled = binary.getMemoryBytesSpilledList.asScala.map(_.toDouble).toIndexedSeq,
+      diskBytesSpilled = binary.getDiskBytesSpilledList.asScala.map(_.toDouble).toIndexedSeq,
+      peakMemoryMetrics = deserializeExecutorPeakMetricsDistributions(binary.getPeakMemoryMetrics)
+    )
+  }
+
+  private def deserializeExecutorPeakMetricsDistributions(
+      binary: StoreTypes.ExecutorPeakMetricsDistributions): ExecutorPeakMetricsDistributions = {
+    new ExecutorPeakMetricsDistributions(
+      quantiles = binary.getQuantilesList.asScala.map(_.toDouble).toIndexedSeq,
+      executorMetrics = binary.getExecutorMetricsList.asScala.map(
+        ExecutorMetricsSerializer.deserialize(_)).toIndexedSeq
+    )
+  }
+
+  private def deserializeTaskData(binary: StoreTypes.TaskData): TaskData = {
+    val resultFetchStart = getOptional(binary.hasResultFetchStart,
+      () => new Date(binary.getResultFetchStart))
+    val duration = getOptional(binary.hasDuration, () => binary.getDuration)
+    val accumulatorUpdates = AccumulableInfoSerializer.deserializeAccumulableInfos(
+      binary.getAccumulatorUpdatesList)
+    val taskMetrics = getOptional(binary.hasTaskMetrics,
+      () => deserializeTaskMetrics(binary.getTaskMetrics))
+    new TaskData(
+      taskId = binary.getTaskId,
+      index = binary.getIndex,
+      attempt = binary.getAttempt,
+      partitionId = binary.getPartitionId,
+      launchTime = new Date(binary.getLaunchTime),
+      resultFetchStart = resultFetchStart,
+      duration = duration,
+      executorId = weakIntern(binary.getExecutorId),

Review Comment:
   For consistency, we use weak here, eg:
   https://github.com/apache/spark/blob/c4619b503e58da38c6223020a73c2ca2e0a8c0fa/core/src/main/scala/org/apache/spark/status/protobuf/TaskDataWrapperSerializer.scala#L100-L103
   
   As far as I know, when the field is of type string (not include map<string....>)



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gengliangwang commented on pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on PR #39192:
URL: https://github.com/apache/spark/pull/39192#issuecomment-1366229183

   This is a big one. @panbingkun Thanks for working on it!


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gengliangwang commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1058739922


##########
core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto:
##########
@@ -390,3 +390,214 @@ message SQLExecutionUIData {
   repeated int64 stages = 11;
   map<int64, string> metric_values = 12;
 }
+
+message StageDataWrapper {
+  StageData info = 1;
+  repeated int64 job_ids = 2;
+  map<string, int64> locality = 3;
+}
+
+message TaskData {
+  int64 task_id = 1;
+  int32 index = 2;
+  int32 attempt = 3;
+  int32 partition_id = 4;
+  int64 launch_time = 5;
+  optional int64 result_fetch_start = 6;
+  optional int64 duration = 7;
+  string executor_id = 8;
+  string host = 9;
+  string status = 10;
+  string task_locality = 11;
+  bool speculative = 12;
+  repeated AccumulableInfo accumulator_updates = 13;
+  optional string error_message = 14;
+  optional TaskMetrics task_metrics = 15;
+  map<string, string> executor_logs = 16;
+  int64 scheduler_delay = 17;
+  int64 getting_result_time = 18;
+}
+
+message StageData {
+  enum StageStatus {

Review Comment:
   JobExecutionStatus is used in `SQLExecutionUIData`. So it can't be moved into `JobData`



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] LuciferYang commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1057610747


##########
core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto:
##########
@@ -390,3 +390,214 @@ message SQLExecutionUIData {
   repeated int64 stages = 11;
   map<int64, string> metric_values = 12;
 }
+
+message StageDataWrapper {
+  StageData info = 1;
+  repeated int64 job_ids = 2;
+  map<string, int64> locality = 3;
+}
+
+message TaskData {
+  int64 task_id = 1;
+  int32 index = 2;
+  int32 attempt = 3;
+  int32 partition_id = 4;
+  int64 launch_time = 5;
+  optional int64 result_fetch_start = 6;
+  optional int64 duration = 7;
+  string executor_id = 8;
+  string host = 9;
+  string status = 10;
+  string task_locality = 11;
+  bool speculative = 12;
+  repeated AccumulableInfo accumulator_updates = 13;
+  optional string error_message = 14;
+  optional TaskMetrics task_metrics = 15;
+  map<string, string> executor_logs = 16;
+  int64 scheduler_delay = 17;
+  int64 getting_result_time = 18;
+}
+
+message StageData {
+  enum StageStatus {

Review Comment:
   Why `StageStatus` designed as `StageData` inside `enum` ?
   
   
   



##########
core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto:
##########
@@ -390,3 +390,214 @@ message SQLExecutionUIData {
   repeated int64 stages = 11;
   map<int64, string> metric_values = 12;
 }
+
+message StageDataWrapper {
+  StageData info = 1;
+  repeated int64 job_ids = 2;
+  map<string, int64> locality = 3;
+}
+
+message TaskData {
+  int64 task_id = 1;
+  int32 index = 2;
+  int32 attempt = 3;
+  int32 partition_id = 4;
+  int64 launch_time = 5;
+  optional int64 result_fetch_start = 6;
+  optional int64 duration = 7;
+  string executor_id = 8;
+  string host = 9;
+  string status = 10;
+  string task_locality = 11;
+  bool speculative = 12;
+  repeated AccumulableInfo accumulator_updates = 13;
+  optional string error_message = 14;
+  optional TaskMetrics task_metrics = 15;
+  map<string, string> executor_logs = 16;
+  int64 scheduler_delay = 17;
+  int64 getting_result_time = 18;
+}
+
+message StageData {
+  enum StageStatus {
+    UNSPECIFIED = 0;
+    ACTIVE = 1;
+    COMPLETE = 2;
+    FAILED = 3;
+    PENDING = 4;
+    SKIPPED = 5;
+  }
+
+  StageStatus status = 1;
+  int64 stage_id = 2;
+  int32 attempt_id = 3;
+  int32 num_tasks = 4;
+  int32 num_active_tasks = 5;
+  int32 num_complete_tasks = 6;
+  int32 num_failed_tasks = 7;
+  int32 num_killed_tasks = 8;
+  int32 num_completed_indices = 9;
+
+  optional int64 submission_time = 10;
+  optional int64 first_task_launched_time = 11;
+  optional int64 completion_time = 12;
+  optional string failure_reason = 13;
+
+  int64 executor_deserialize_time = 14;
+  int64 executor_deserialize_cpu_time = 15;
+  int64 executor_run_time = 16;
+  int64 executor_cpu_time = 17;
+  int64 result_size = 18;
+  int64 jvm_gc_time = 19;
+  int64 result_serialization_time = 20;
+  int64 memory_bytes_spilled = 21;
+  int64 disk_bytes_spilled = 22;
+  int64 peak_execution_memory = 23;
+  int64 input_bytes = 24;
+  int64 input_records = 25;
+  int64 output_bytes = 26;
+  int64 output_records = 27;
+  int64 shuffle_remote_blocks_fetched = 28;
+  int64 shuffle_local_blocks_fetched = 29;
+  int64 shuffle_fetch_wait_time = 30;
+  int64 shuffle_remote_bytes_read = 31;
+  int64 shuffle_remote_bytes_read_to_disk = 32;
+  int64 shuffle_local_bytes_read = 33;
+  int64 shuffle_read_bytes = 34;
+  int64 shuffle_read_records = 35;
+  int64 shuffle_write_bytes = 36;
+  int64 shuffle_write_time = 37;
+  int64 shuffle_write_records = 38;
+
+  string name = 39;
+  optional string description = 40;
+  string details = 41;
+  string scheduling_pool = 42;
+
+  repeated int64 rdd_ids = 43;
+  repeated AccumulableInfo accumulator_updates = 44;
+  map<int64, TaskData> tasks = 45;
+  map<string, ExecutorStageSummary> executor_summary = 46;

Review Comment:
   ditto



##########
core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto:
##########
@@ -390,3 +390,214 @@ message SQLExecutionUIData {
   repeated int64 stages = 11;
   map<int64, string> metric_values = 12;
 }
+
+message StageDataWrapper {
+  StageData info = 1;
+  repeated int64 job_ids = 2;
+  map<string, int64> locality = 3;
+}
+
+message TaskData {
+  int64 task_id = 1;
+  int32 index = 2;
+  int32 attempt = 3;
+  int32 partition_id = 4;
+  int64 launch_time = 5;
+  optional int64 result_fetch_start = 6;
+  optional int64 duration = 7;
+  string executor_id = 8;
+  string host = 9;
+  string status = 10;
+  string task_locality = 11;
+  bool speculative = 12;
+  repeated AccumulableInfo accumulator_updates = 13;
+  optional string error_message = 14;
+  optional TaskMetrics task_metrics = 15;
+  map<string, string> executor_logs = 16;
+  int64 scheduler_delay = 17;
+  int64 getting_result_time = 18;
+}
+
+message StageData {
+  enum StageStatus {
+    UNSPECIFIED = 0;
+    ACTIVE = 1;
+    COMPLETE = 2;
+    FAILED = 3;
+    PENDING = 4;
+    SKIPPED = 5;
+  }
+
+  StageStatus status = 1;
+  int64 stage_id = 2;
+  int32 attempt_id = 3;
+  int32 num_tasks = 4;
+  int32 num_active_tasks = 5;
+  int32 num_complete_tasks = 6;
+  int32 num_failed_tasks = 7;
+  int32 num_killed_tasks = 8;
+  int32 num_completed_indices = 9;
+
+  optional int64 submission_time = 10;
+  optional int64 first_task_launched_time = 11;
+  optional int64 completion_time = 12;
+  optional string failure_reason = 13;
+
+  int64 executor_deserialize_time = 14;
+  int64 executor_deserialize_cpu_time = 15;
+  int64 executor_run_time = 16;
+  int64 executor_cpu_time = 17;
+  int64 result_size = 18;
+  int64 jvm_gc_time = 19;
+  int64 result_serialization_time = 20;
+  int64 memory_bytes_spilled = 21;
+  int64 disk_bytes_spilled = 22;
+  int64 peak_execution_memory = 23;
+  int64 input_bytes = 24;
+  int64 input_records = 25;
+  int64 output_bytes = 26;
+  int64 output_records = 27;
+  int64 shuffle_remote_blocks_fetched = 28;
+  int64 shuffle_local_blocks_fetched = 29;
+  int64 shuffle_fetch_wait_time = 30;
+  int64 shuffle_remote_bytes_read = 31;
+  int64 shuffle_remote_bytes_read_to_disk = 32;
+  int64 shuffle_local_bytes_read = 33;
+  int64 shuffle_read_bytes = 34;
+  int64 shuffle_read_records = 35;
+  int64 shuffle_write_bytes = 36;
+  int64 shuffle_write_time = 37;
+  int64 shuffle_write_records = 38;
+
+  string name = 39;
+  optional string description = 40;
+  string details = 41;
+  string scheduling_pool = 42;
+
+  repeated int64 rdd_ids = 43;
+  repeated AccumulableInfo accumulator_updates = 44;
+  map<int64, TaskData> tasks = 45;

Review Comment:
   I see, hmm... should we encapsulate this map?
   
   such as 
   
   ```
   optional TaskMap tasks = 45;
   
   message TaskMap {
     map<int64, TaskData> tasks = 1;
   }
   ```
   
   also cc @gengliangwang 



##########
core/src/main/scala/org/apache/spark/status/protobuf/StageDataWrapperSerializer.scala:
##########
@@ -0,0 +1,622 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.Date
+
+import collection.JavaConverters._
+import org.apache.commons.collections4.MapUtils
+
+import org.apache.spark.status.StageDataWrapper
+import org.apache.spark.status.api.v1.{ExecutorMetricsDistributions, ExecutorPeakMetricsDistributions, InputMetricDistributions, InputMetrics, OutputMetricDistributions, OutputMetrics, ShuffleReadMetricDistributions, ShuffleReadMetrics, ShuffleWriteMetricDistributions, ShuffleWriteMetrics, SpeculationStageSummary, StageData, StageStatus, TaskData, TaskMetricDistributions, TaskMetrics}
+import org.apache.spark.status.protobuf.Utils.getOptional
+import org.apache.spark.util.Utils.weakIntern
+
+class StageDataWrapperSerializer extends ProtobufSerDe {
+
+  override val supportClass: Class[_] = classOf[StageDataWrapper]
+
+  override def serialize(input: Any): Array[Byte] =
+    serialize(input.asInstanceOf[StageDataWrapper])
+
+  private def serialize(s: StageDataWrapper): Array[Byte] = {
+    val builder = StoreTypes.StageDataWrapper.newBuilder()
+    builder.setInfo(serializeStageData(s.info))
+    s.jobIds.foreach(id => builder.addJobIds(id.toLong))
+    s.locality.foreach { entry =>
+      builder.putLocality(entry._1, entry._2)
+    }
+    builder.build().toByteArray
+  }
+
+  private def serializeStageData(stageData: StageData): StoreTypes.StageData = {
+    val stageDataBuilder = StoreTypes.StageData.newBuilder()
+    stageDataBuilder
+      .setStatus(serializeStageStatus(stageData.status))
+      .setStageId(stageData.stageId.toLong)
+      .setAttemptId(stageData.attemptId)
+      .setNumTasks(stageData.numTasks)
+      .setNumActiveTasks(stageData.numActiveTasks)
+      .setNumCompleteTasks(stageData.numCompleteTasks)
+      .setNumFailedTasks(stageData.numFailedTasks)
+      .setNumKilledTasks(stageData.numKilledTasks)
+      .setNumCompletedIndices(stageData.numCompletedIndices)
+      .setExecutorDeserializeTime(stageData.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(stageData.executorDeserializeCpuTime)
+      .setExecutorRunTime(stageData.executorRunTime)
+      .setExecutorCpuTime(stageData.executorCpuTime)
+      .setResultSize(stageData.resultSize)
+      .setJvmGcTime(stageData.jvmGcTime)
+      .setResultSerializationTime(stageData.resultSerializationTime)
+      .setMemoryBytesSpilled(stageData.memoryBytesSpilled)
+      .setDiskBytesSpilled(stageData.diskBytesSpilled)
+      .setPeakExecutionMemory(stageData.peakExecutionMemory)
+      .setInputBytes(stageData.inputBytes)
+      .setInputRecords(stageData.inputRecords)
+      .setOutputBytes(stageData.outputBytes)
+      .setOutputRecords(stageData.outputRecords)
+      .setShuffleRemoteBlocksFetched(stageData.shuffleRemoteBlocksFetched)
+      .setShuffleLocalBlocksFetched(stageData.shuffleLocalBlocksFetched)
+      .setShuffleFetchWaitTime(stageData.shuffleFetchWaitTime)
+      .setShuffleRemoteBytesRead(stageData.shuffleRemoteBytesRead)
+      .setShuffleRemoteBytesReadToDisk(stageData.shuffleRemoteBytesReadToDisk)
+      .setShuffleLocalBytesRead(stageData.shuffleLocalBytesRead)
+      .setShuffleReadBytes(stageData.shuffleReadBytes)
+      .setShuffleReadRecords(stageData.shuffleReadRecords)
+      .setShuffleWriteBytes(stageData.shuffleWriteBytes)
+      .setShuffleWriteTime(stageData.shuffleWriteTime)
+      .setShuffleWriteRecords(stageData.shuffleWriteRecords)
+      .setName(stageData.name)
+      .setDetails(stageData.details)
+      .setSchedulingPool(stageData.schedulingPool)
+      .setResourceProfileId(stageData.resourceProfileId)
+    stageData.submissionTime.foreach { d =>
+      stageDataBuilder.setSubmissionTime(d.getTime)
+    }
+    stageData.firstTaskLaunchedTime.foreach { d =>
+      stageDataBuilder.setFirstTaskLaunchedTime(d.getTime)
+    }
+    stageData.completionTime.foreach { d =>
+      stageDataBuilder.setCompletionTime(d.getTime)
+    }
+    stageData.failureReason.foreach { fr =>
+      stageDataBuilder.setFailureReason(fr)
+    }
+    stageData.description.foreach { d =>
+      stageDataBuilder.setDescription(d)
+    }
+    stageData.rddIds.foreach(id => stageDataBuilder.addRddIds(id.toLong))
+    stageData.accumulatorUpdates.foreach { update =>
+      stageDataBuilder.addAccumulatorUpdates(Utils.serializeAccumulableInfo(update))
+    }

Review Comment:
   I think there are 3 choices for the definition of `serializeAccumulableInfo`  function:
   
   1. Move it from class `TaskDataWrapperSerializer` to companion object `TaskDataWrapperSerializer`
   2. Move it from class `TaskDataWrapperSerializer` to object `AccumulableInfoSerializer`
   3. Keep the status quo and let `StageDataWrapperSerializer` hold a `TaskDataWrapperSerializer` instance 
   
   Similar suggestions for `deserializeAccumulableInfo\serializeExecutorStageSummary\deserializeExecutorStageSummary` and I think `Utils` should be a more general functions 



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
panbingkun commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1058009462


##########
core/src/main/scala/org/apache/spark/status/protobuf/StageDataWrapperSerializer.scala:
##########
@@ -0,0 +1,622 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.Date
+
+import collection.JavaConverters._
+import org.apache.commons.collections4.MapUtils
+
+import org.apache.spark.status.StageDataWrapper
+import org.apache.spark.status.api.v1.{ExecutorMetricsDistributions, ExecutorPeakMetricsDistributions, InputMetricDistributions, InputMetrics, OutputMetricDistributions, OutputMetrics, ShuffleReadMetricDistributions, ShuffleReadMetrics, ShuffleWriteMetricDistributions, ShuffleWriteMetrics, SpeculationStageSummary, StageData, StageStatus, TaskData, TaskMetricDistributions, TaskMetrics}
+import org.apache.spark.status.protobuf.Utils.getOptional
+import org.apache.spark.util.Utils.weakIntern
+
+class StageDataWrapperSerializer extends ProtobufSerDe {
+
+  override val supportClass: Class[_] = classOf[StageDataWrapper]
+
+  override def serialize(input: Any): Array[Byte] =
+    serialize(input.asInstanceOf[StageDataWrapper])
+
+  private def serialize(s: StageDataWrapper): Array[Byte] = {
+    val builder = StoreTypes.StageDataWrapper.newBuilder()
+    builder.setInfo(serializeStageData(s.info))
+    s.jobIds.foreach(id => builder.addJobIds(id.toLong))
+    s.locality.foreach { entry =>
+      builder.putLocality(entry._1, entry._2)
+    }
+    builder.build().toByteArray
+  }
+
+  private def serializeStageData(stageData: StageData): StoreTypes.StageData = {
+    val stageDataBuilder = StoreTypes.StageData.newBuilder()
+    stageDataBuilder
+      .setStatus(serializeStageStatus(stageData.status))
+      .setStageId(stageData.stageId.toLong)
+      .setAttemptId(stageData.attemptId)
+      .setNumTasks(stageData.numTasks)
+      .setNumActiveTasks(stageData.numActiveTasks)
+      .setNumCompleteTasks(stageData.numCompleteTasks)
+      .setNumFailedTasks(stageData.numFailedTasks)
+      .setNumKilledTasks(stageData.numKilledTasks)
+      .setNumCompletedIndices(stageData.numCompletedIndices)
+      .setExecutorDeserializeTime(stageData.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(stageData.executorDeserializeCpuTime)
+      .setExecutorRunTime(stageData.executorRunTime)
+      .setExecutorCpuTime(stageData.executorCpuTime)
+      .setResultSize(stageData.resultSize)
+      .setJvmGcTime(stageData.jvmGcTime)
+      .setResultSerializationTime(stageData.resultSerializationTime)
+      .setMemoryBytesSpilled(stageData.memoryBytesSpilled)
+      .setDiskBytesSpilled(stageData.diskBytesSpilled)
+      .setPeakExecutionMemory(stageData.peakExecutionMemory)
+      .setInputBytes(stageData.inputBytes)
+      .setInputRecords(stageData.inputRecords)
+      .setOutputBytes(stageData.outputBytes)
+      .setOutputRecords(stageData.outputRecords)
+      .setShuffleRemoteBlocksFetched(stageData.shuffleRemoteBlocksFetched)
+      .setShuffleLocalBlocksFetched(stageData.shuffleLocalBlocksFetched)
+      .setShuffleFetchWaitTime(stageData.shuffleFetchWaitTime)
+      .setShuffleRemoteBytesRead(stageData.shuffleRemoteBytesRead)
+      .setShuffleRemoteBytesReadToDisk(stageData.shuffleRemoteBytesReadToDisk)
+      .setShuffleLocalBytesRead(stageData.shuffleLocalBytesRead)
+      .setShuffleReadBytes(stageData.shuffleReadBytes)
+      .setShuffleReadRecords(stageData.shuffleReadRecords)
+      .setShuffleWriteBytes(stageData.shuffleWriteBytes)
+      .setShuffleWriteTime(stageData.shuffleWriteTime)
+      .setShuffleWriteRecords(stageData.shuffleWriteRecords)
+      .setName(stageData.name)
+      .setDetails(stageData.details)
+      .setSchedulingPool(stageData.schedulingPool)
+      .setResourceProfileId(stageData.resourceProfileId)
+    stageData.submissionTime.foreach { d =>
+      stageDataBuilder.setSubmissionTime(d.getTime)
+    }
+    stageData.firstTaskLaunchedTime.foreach { d =>
+      stageDataBuilder.setFirstTaskLaunchedTime(d.getTime)
+    }
+    stageData.completionTime.foreach { d =>
+      stageDataBuilder.setCompletionTime(d.getTime)
+    }
+    stageData.failureReason.foreach { fr =>
+      stageDataBuilder.setFailureReason(fr)
+    }
+    stageData.description.foreach { d =>
+      stageDataBuilder.setDescription(d)
+    }
+    stageData.rddIds.foreach(id => stageDataBuilder.addRddIds(id.toLong))
+    stageData.accumulatorUpdates.foreach { update =>
+      stageDataBuilder.addAccumulatorUpdates(Utils.serializeAccumulableInfo(update))
+    }

Review Comment:
   Done



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
panbingkun commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1059217493


##########
core/src/main/scala/org/apache/spark/status/protobuf/AccumulableInfoSerializer.scala:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.{List => JList}
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.status.api.v1.AccumulableInfo
+import org.apache.spark.status.protobuf.Utils.getOptional
+
+object AccumulableInfoSerializer {

Review Comment:
   Done



##########
core/src/main/scala/org/apache/spark/status/protobuf/ExecutorStageSummarySerializer.scala:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.spark.status.protobuf
+
+import org.apache.spark.status.api.v1.ExecutorStageSummary
+import org.apache.spark.status.protobuf.Utils.getOptional
+
+object ExecutorStageSummarySerializer {
+
+  private[protobuf] def serialize(input: ExecutorStageSummary): StoreTypes.ExecutorStageSummary = {

Review Comment:
   Done



##########
core/src/main/scala/org/apache/spark/status/protobuf/StageStatusSerializer.scala:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.spark.status.protobuf
+
+import org.apache.commons.lang3.StringUtils
+
+import org.apache.spark.status.api.v1.StageStatus
+
+object StageStatusSerializer {

Review Comment:
   Done



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] LuciferYang commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1058036782


##########
core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto:
##########
@@ -390,3 +390,214 @@ message SQLExecutionUIData {
   repeated int64 stages = 11;
   map<int64, string> metric_values = 12;
 }
+
+message StageDataWrapper {
+  StageData info = 1;
+  repeated int64 job_ids = 2;
+  map<string, int64> locality = 3;
+}
+
+message TaskData {
+  int64 task_id = 1;
+  int32 index = 2;
+  int32 attempt = 3;
+  int32 partition_id = 4;
+  int64 launch_time = 5;
+  optional int64 result_fetch_start = 6;
+  optional int64 duration = 7;
+  string executor_id = 8;
+  string host = 9;
+  string status = 10;
+  string task_locality = 11;
+  bool speculative = 12;
+  repeated AccumulableInfo accumulator_updates = 13;
+  optional string error_message = 14;
+  optional TaskMetrics task_metrics = 15;
+  map<string, string> executor_logs = 16;
+  int64 scheduler_delay = 17;
+  int64 getting_result_time = 18;
+}
+
+message StageData {
+  enum StageStatus {

Review Comment:
   fine to me



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
panbingkun commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1058910608


##########
core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto:
##########
@@ -390,3 +390,214 @@ message SQLExecutionUIData {
   repeated int64 stages = 11;
   map<int64, string> metric_values = 12;
 }
+
+message StageDataWrapper {
+  StageData info = 1;
+  repeated int64 job_ids = 2;
+  map<string, int64> locality = 3;
+}
+
+message TaskData {
+  int64 task_id = 1;
+  int32 index = 2;
+  int32 attempt = 3;
+  int32 partition_id = 4;
+  int64 launch_time = 5;
+  optional int64 result_fetch_start = 6;
+  optional int64 duration = 7;
+  string executor_id = 8;
+  string host = 9;
+  string status = 10;
+  string task_locality = 11;
+  bool speculative = 12;
+  repeated AccumulableInfo accumulator_updates = 13;
+  optional string error_message = 14;
+  optional TaskMetrics task_metrics = 15;
+  map<string, string> executor_logs = 16;
+  int64 scheduler_delay = 17;
+  int64 getting_result_time = 18;
+}
+
+message StageData {
+  enum StageStatus {

Review Comment:
   New pr for JobExecutionStatus: https://github.com/apache/spark/pull/39286
   @gengliangwang @LuciferYang 



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
panbingkun commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1058801116


##########
core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto:
##########
@@ -390,3 +390,214 @@ message SQLExecutionUIData {
   repeated int64 stages = 11;
   map<int64, string> metric_values = 12;
 }
+
+message StageDataWrapper {
+  StageData info = 1;
+  repeated int64 job_ids = 2;
+  map<string, int64> locality = 3;
+}
+
+message TaskData {
+  int64 task_id = 1;
+  int32 index = 2;
+  int32 attempt = 3;
+  int32 partition_id = 4;
+  int64 launch_time = 5;
+  optional int64 result_fetch_start = 6;
+  optional int64 duration = 7;
+  string executor_id = 8;
+  string host = 9;
+  string status = 10;
+  string task_locality = 11;
+  bool speculative = 12;
+  repeated AccumulableInfo accumulator_updates = 13;
+  optional string error_message = 14;
+  optional TaskMetrics task_metrics = 15;
+  map<string, string> executor_logs = 16;
+  int64 scheduler_delay = 17;
+  int64 getting_result_time = 18;
+}
+
+message StageData {
+  enum StageStatus {

Review Comment:
   A.If the enum definition of `JobExecutionStatus` in JobData
   <img width="224" alt="image" src="https://user-images.githubusercontent.com/15246973/209922039-617b3993-b6c2-48dc-a24f-fdb343585c68.png">
   
   SQLExecutionUIData can use it as follows:
   <img width="414" alt="image" src="https://user-images.githubusercontent.com/15246973/209922101-a2504145-87a3-46f7-b918-6d4363773d17.png">
   
   B.Other choices for the definition of `JobExecutionStatus` as follows:
   <img width="233" alt="image" src="https://user-images.githubusercontent.com/15246973/209922683-296a5c81-daa8-48ad-8bd2-8ea4682953e5.png">
   <img width="699" alt="image" src="https://user-images.githubusercontent.com/15246973/209922716-51945000-6654-4283-affb-37e2dbd5978e.png">
   <img width="390" alt="image" src="https://user-images.githubusercontent.com/15246973/209922853-d0d1affa-6c94-454a-883a-b87551c7ddb0.png">
   
   Otherwise, I'm afraid there are other enumeration names that will conflict(eg: StageStatus.FAILED VS JobExecutionStatus.FAILED). @gengliangwang @LuciferYang 



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
panbingkun commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1058004492


##########
core/src/main/scala/org/apache/spark/status/protobuf/StageDataWrapperSerializer.scala:
##########
@@ -0,0 +1,622 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.Date
+
+import collection.JavaConverters._
+import org.apache.commons.collections4.MapUtils
+
+import org.apache.spark.status.StageDataWrapper
+import org.apache.spark.status.api.v1.{ExecutorMetricsDistributions, ExecutorPeakMetricsDistributions, InputMetricDistributions, InputMetrics, OutputMetricDistributions, OutputMetrics, ShuffleReadMetricDistributions, ShuffleReadMetrics, ShuffleWriteMetricDistributions, ShuffleWriteMetrics, SpeculationStageSummary, StageData, StageStatus, TaskData, TaskMetricDistributions, TaskMetrics}
+import org.apache.spark.status.protobuf.Utils.getOptional
+import org.apache.spark.util.Utils.weakIntern
+
+class StageDataWrapperSerializer extends ProtobufSerDe {
+
+  override val supportClass: Class[_] = classOf[StageDataWrapper]
+
+  override def serialize(input: Any): Array[Byte] =
+    serialize(input.asInstanceOf[StageDataWrapper])
+
+  private def serialize(s: StageDataWrapper): Array[Byte] = {
+    val builder = StoreTypes.StageDataWrapper.newBuilder()
+    builder.setInfo(serializeStageData(s.info))
+    s.jobIds.foreach(id => builder.addJobIds(id.toLong))
+    s.locality.foreach { entry =>
+      builder.putLocality(entry._1, entry._2)
+    }
+    builder.build().toByteArray
+  }
+
+  private def serializeStageData(stageData: StageData): StoreTypes.StageData = {
+    val stageDataBuilder = StoreTypes.StageData.newBuilder()
+    stageDataBuilder
+      .setStatus(serializeStageStatus(stageData.status))
+      .setStageId(stageData.stageId.toLong)
+      .setAttemptId(stageData.attemptId)
+      .setNumTasks(stageData.numTasks)
+      .setNumActiveTasks(stageData.numActiveTasks)
+      .setNumCompleteTasks(stageData.numCompleteTasks)
+      .setNumFailedTasks(stageData.numFailedTasks)
+      .setNumKilledTasks(stageData.numKilledTasks)
+      .setNumCompletedIndices(stageData.numCompletedIndices)
+      .setExecutorDeserializeTime(stageData.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(stageData.executorDeserializeCpuTime)
+      .setExecutorRunTime(stageData.executorRunTime)
+      .setExecutorCpuTime(stageData.executorCpuTime)
+      .setResultSize(stageData.resultSize)
+      .setJvmGcTime(stageData.jvmGcTime)
+      .setResultSerializationTime(stageData.resultSerializationTime)
+      .setMemoryBytesSpilled(stageData.memoryBytesSpilled)
+      .setDiskBytesSpilled(stageData.diskBytesSpilled)
+      .setPeakExecutionMemory(stageData.peakExecutionMemory)
+      .setInputBytes(stageData.inputBytes)
+      .setInputRecords(stageData.inputRecords)
+      .setOutputBytes(stageData.outputBytes)
+      .setOutputRecords(stageData.outputRecords)
+      .setShuffleRemoteBlocksFetched(stageData.shuffleRemoteBlocksFetched)
+      .setShuffleLocalBlocksFetched(stageData.shuffleLocalBlocksFetched)
+      .setShuffleFetchWaitTime(stageData.shuffleFetchWaitTime)
+      .setShuffleRemoteBytesRead(stageData.shuffleRemoteBytesRead)
+      .setShuffleRemoteBytesReadToDisk(stageData.shuffleRemoteBytesReadToDisk)
+      .setShuffleLocalBytesRead(stageData.shuffleLocalBytesRead)
+      .setShuffleReadBytes(stageData.shuffleReadBytes)
+      .setShuffleReadRecords(stageData.shuffleReadRecords)
+      .setShuffleWriteBytes(stageData.shuffleWriteBytes)
+      .setShuffleWriteTime(stageData.shuffleWriteTime)
+      .setShuffleWriteRecords(stageData.shuffleWriteRecords)
+      .setName(stageData.name)
+      .setDetails(stageData.details)
+      .setSchedulingPool(stageData.schedulingPool)
+      .setResourceProfileId(stageData.resourceProfileId)
+    stageData.submissionTime.foreach { d =>
+      stageDataBuilder.setSubmissionTime(d.getTime)
+    }
+    stageData.firstTaskLaunchedTime.foreach { d =>
+      stageDataBuilder.setFirstTaskLaunchedTime(d.getTime)
+    }
+    stageData.completionTime.foreach { d =>
+      stageDataBuilder.setCompletionTime(d.getTime)
+    }
+    stageData.failureReason.foreach { fr =>
+      stageDataBuilder.setFailureReason(fr)
+    }
+    stageData.description.foreach { d =>
+      stageDataBuilder.setDescription(d)
+    }
+    stageData.rddIds.foreach(id => stageDataBuilder.addRddIds(id.toLong))
+    stageData.accumulatorUpdates.foreach { update =>
+      stageDataBuilder.addAccumulatorUpdates(Utils.serializeAccumulableInfo(update))
+    }

Review Comment:
   Ok, let me  do it.



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] LuciferYang commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1058205169


##########
core/src/main/scala/org/apache/spark/status/protobuf/StageDataWrapperSerializer.scala:
##########
@@ -0,0 +1,627 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.Date
+
+import collection.JavaConverters._
+import org.apache.commons.collections4.MapUtils
+
+import org.apache.spark.status.StageDataWrapper
+import org.apache.spark.status.api.v1.{ExecutorMetricsDistributions, ExecutorPeakMetricsDistributions, InputMetricDistributions, InputMetrics, OutputMetricDistributions, OutputMetrics, ShuffleReadMetricDistributions, ShuffleReadMetrics, ShuffleWriteMetricDistributions, ShuffleWriteMetrics, SpeculationStageSummary, StageData, StageStatus, TaskData, TaskMetricDistributions, TaskMetrics}
+import org.apache.spark.status.protobuf.Utils.getOptional
+import org.apache.spark.util.Utils.weakIntern
+
+class StageDataWrapperSerializer extends ProtobufSerDe {
+
+  override val supportClass: Class[_] = classOf[StageDataWrapper]
+
+  override def serialize(input: Any): Array[Byte] =
+    serialize(input.asInstanceOf[StageDataWrapper])
+
+  private def serialize(s: StageDataWrapper): Array[Byte] = {
+    val builder = StoreTypes.StageDataWrapper.newBuilder()
+    builder.setInfo(serializeStageData(s.info))
+    s.jobIds.foreach(id => builder.addJobIds(id.toLong))
+    s.locality.foreach { entry =>
+      builder.putLocality(entry._1, entry._2)
+    }
+    builder.build().toByteArray
+  }
+
+  private def serializeStageData(stageData: StageData): StoreTypes.StageData = {
+    val stageDataBuilder = StoreTypes.StageData.newBuilder()
+    stageDataBuilder
+      .setStatus(serializeStageStatus(stageData.status))
+      .setStageId(stageData.stageId.toLong)
+      .setAttemptId(stageData.attemptId)
+      .setNumTasks(stageData.numTasks)
+      .setNumActiveTasks(stageData.numActiveTasks)
+      .setNumCompleteTasks(stageData.numCompleteTasks)
+      .setNumFailedTasks(stageData.numFailedTasks)
+      .setNumKilledTasks(stageData.numKilledTasks)
+      .setNumCompletedIndices(stageData.numCompletedIndices)
+      .setExecutorDeserializeTime(stageData.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(stageData.executorDeserializeCpuTime)
+      .setExecutorRunTime(stageData.executorRunTime)
+      .setExecutorCpuTime(stageData.executorCpuTime)
+      .setResultSize(stageData.resultSize)
+      .setJvmGcTime(stageData.jvmGcTime)
+      .setResultSerializationTime(stageData.resultSerializationTime)
+      .setMemoryBytesSpilled(stageData.memoryBytesSpilled)
+      .setDiskBytesSpilled(stageData.diskBytesSpilled)
+      .setPeakExecutionMemory(stageData.peakExecutionMemory)
+      .setInputBytes(stageData.inputBytes)
+      .setInputRecords(stageData.inputRecords)
+      .setOutputBytes(stageData.outputBytes)
+      .setOutputRecords(stageData.outputRecords)
+      .setShuffleRemoteBlocksFetched(stageData.shuffleRemoteBlocksFetched)
+      .setShuffleLocalBlocksFetched(stageData.shuffleLocalBlocksFetched)
+      .setShuffleFetchWaitTime(stageData.shuffleFetchWaitTime)
+      .setShuffleRemoteBytesRead(stageData.shuffleRemoteBytesRead)
+      .setShuffleRemoteBytesReadToDisk(stageData.shuffleRemoteBytesReadToDisk)
+      .setShuffleLocalBytesRead(stageData.shuffleLocalBytesRead)
+      .setShuffleReadBytes(stageData.shuffleReadBytes)
+      .setShuffleReadRecords(stageData.shuffleReadRecords)
+      .setShuffleWriteBytes(stageData.shuffleWriteBytes)
+      .setShuffleWriteTime(stageData.shuffleWriteTime)
+      .setShuffleWriteRecords(stageData.shuffleWriteRecords)
+      .setName(stageData.name)
+      .setDetails(stageData.details)
+      .setSchedulingPool(stageData.schedulingPool)
+      .setResourceProfileId(stageData.resourceProfileId)
+    stageData.submissionTime.foreach { d =>
+      stageDataBuilder.setSubmissionTime(d.getTime)
+    }
+    stageData.firstTaskLaunchedTime.foreach { d =>
+      stageDataBuilder.setFirstTaskLaunchedTime(d.getTime)
+    }
+    stageData.completionTime.foreach { d =>
+      stageDataBuilder.setCompletionTime(d.getTime)
+    }
+    stageData.failureReason.foreach { fr =>
+      stageDataBuilder.setFailureReason(fr)
+    }
+    stageData.description.foreach { d =>
+      stageDataBuilder.setDescription(d)
+    }
+    stageData.rddIds.foreach(id => stageDataBuilder.addRddIds(id.toLong))
+    stageData.accumulatorUpdates.foreach { update =>
+      stageDataBuilder.addAccumulatorUpdates(
+        AccumulableInfoSerializer.serializeAccumulableInfo(update))
+    }
+    stageData.tasks.foreach { t =>
+      t.foreach { entry =>
+        stageDataBuilder.putTasks(entry._1, serializeTaskData(entry._2))
+      }
+    }
+    stageData.executorSummary.foreach { es =>
+      es.foreach { entry =>
+        stageDataBuilder.putExecutorSummary(entry._1,
+          ExecutorStageSummarySerializer.serializeExecutorStageSummary(entry._2))
+      }
+    }
+    stageData.speculationSummary.foreach { ss =>
+      stageDataBuilder.setSpeculationSummary(serializeSpeculationStageSummary(ss))
+    }
+    stageData.killedTasksSummary.foreach { entry =>
+      stageDataBuilder.putKilledTasksSummary(entry._1, entry._2)
+    }
+    stageData.peakExecutorMetrics.foreach { pem =>
+      stageDataBuilder.setPeakExecutorMetrics(ExecutorMetricsSerializer.serialize(pem))
+    }
+    stageData.taskMetricsDistributions.foreach { tmd =>
+      stageDataBuilder.setTaskMetricsDistributions(serializeTaskMetricDistributions(tmd))
+    }
+    stageData.executorMetricsDistributions.foreach { emd =>
+      stageDataBuilder.setExecutorMetricsDistributions(serializeExecutorMetricsDistributions(emd))
+    }
+    stageDataBuilder.build()
+  }
+
+  private def serializeStageStatus(s: StageStatus): StoreTypes.StageData.StageStatus = {
+    StoreTypes.StageData.StageStatus.valueOf(s.toString)
+  }
+
+  private def serializeTaskData(t: TaskData): StoreTypes.TaskData = {
+    val taskDataBuilder = StoreTypes.TaskData.newBuilder()
+    taskDataBuilder
+      .setTaskId(t.taskId)
+      .setIndex(t.index)
+      .setAttempt(t.attempt)
+      .setPartitionId(t.partitionId)
+      .setLaunchTime(t.launchTime.getTime)
+      .setExecutorId(t.executorId)
+      .setHost(t.host)
+      .setStatus(t.status)
+      .setTaskLocality(t.taskLocality)
+      .setSpeculative(t.speculative)
+      .setSchedulerDelay(t.schedulerDelay)
+      .setGettingResultTime(t.gettingResultTime)
+    t.resultFetchStart.foreach { rfs =>
+      taskDataBuilder.setResultFetchStart(rfs.getTime)
+    }
+    t.duration.foreach { d =>
+      taskDataBuilder.setDuration(d)
+    }
+    t.accumulatorUpdates.foreach { update =>
+      taskDataBuilder.addAccumulatorUpdates(
+        AccumulableInfoSerializer.serializeAccumulableInfo(update))
+    }
+    t.errorMessage.foreach { em =>
+      taskDataBuilder.setErrorMessage(em)
+    }
+    t.taskMetrics.foreach { tm =>
+      taskDataBuilder.setTaskMetrics(serializeTaskMetrics(tm))
+    }
+    t.executorLogs.foreach { entry =>
+      taskDataBuilder.putExecutorLogs(entry._1, entry._2)
+    }
+    taskDataBuilder.build()
+  }
+
+  private def serializeTaskMetrics(tm: TaskMetrics): StoreTypes.TaskMetrics = {
+    val taskMetricsBuilder = StoreTypes.TaskMetrics.newBuilder()
+    taskMetricsBuilder
+      .setExecutorDeserializeTime(tm.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(tm.executorDeserializeCpuTime)
+      .setExecutorRunTime(tm.executorRunTime)
+      .setExecutorCpuTime(tm.executorCpuTime)
+      .setResultSize(tm.resultSize)
+      .setJvmGcTime(tm.jvmGcTime)
+      .setResultSerializationTime(tm.resultSerializationTime)
+      .setMemoryBytesSpilled(tm.memoryBytesSpilled)
+      .setDiskBytesSpilled(tm.diskBytesSpilled)
+      .setPeakExecutionMemory(tm.peakExecutionMemory)
+      .setInputMetrics(serializeInputMetrics(tm.inputMetrics))
+      .setOutputMetrics(serializeOutputMetrics(tm.outputMetrics))
+      .setShuffleReadMetrics(serializeShuffleReadMetrics(tm.shuffleReadMetrics))
+      .setShuffleWriteMetrics(serializeShuffleWriteMetrics(tm.shuffleWriteMetrics))
+    taskMetricsBuilder.build()
+  }
+
+  private def serializeInputMetrics(im: InputMetrics): StoreTypes.InputMetrics = {
+    StoreTypes.InputMetrics.newBuilder()
+      .setBytesRead(im.bytesRead)
+      .setRecordsRead(im.recordsRead)
+      .build()
+  }
+
+  private def serializeOutputMetrics(om: OutputMetrics): StoreTypes.OutputMetrics = {
+    StoreTypes.OutputMetrics.newBuilder()
+      .setBytesWritten(om.bytesWritten)
+      .setRecordsWritten(om.recordsWritten)
+      .build()
+  }
+
+  private def serializeShuffleReadMetrics(
+      srm: ShuffleReadMetrics): StoreTypes.ShuffleReadMetrics = {
+    StoreTypes.ShuffleReadMetrics.newBuilder()
+      .setRemoteBlocksFetched(srm.remoteBlocksFetched)
+      .setLocalBlocksFetched(srm.localBlocksFetched)
+      .setFetchWaitTime(srm.fetchWaitTime)
+      .setRemoteBytesRead(srm.remoteBytesRead)
+      .setRemoteBytesReadToDisk(srm.remoteBytesReadToDisk)
+      .setLocalBytesRead(srm.localBytesRead)
+      .setRecordsRead(srm.recordsRead)
+      .build()
+  }
+
+  private def serializeShuffleWriteMetrics(
+      swm: ShuffleWriteMetrics): StoreTypes.ShuffleWriteMetrics = {
+    StoreTypes.ShuffleWriteMetrics.newBuilder()
+      .setBytesWritten(swm.bytesWritten)
+      .setWriteTime(swm.writeTime)
+      .setRecordsWritten(swm.recordsWritten)
+      .build()
+  }
+
+  private def serializeSpeculationStageSummary(
+      sss: SpeculationStageSummary): StoreTypes.SpeculationStageSummary = {
+    StoreTypes.SpeculationStageSummary.newBuilder()
+      .setNumTasks(sss.numTasks)
+      .setNumActiveTasks(sss.numActiveTasks)
+      .setNumCompletedTasks(sss.numCompletedTasks)
+      .setNumFailedTasks(sss.numFailedTasks)
+      .setNumKilledTasks(sss.numKilledTasks)
+      .build()
+  }
+
+  private def serializeTaskMetricDistributions(
+      tmd: TaskMetricDistributions): StoreTypes.TaskMetricDistributions = {
+    val builder = StoreTypes.TaskMetricDistributions.newBuilder()
+    tmd.quantiles.foreach(q => builder.addQuantiles(q))
+    tmd.duration.foreach(d => builder.addDuration(d))
+    tmd.executorDeserializeTime.foreach(edt => builder.addExecutorDeserializeTime(edt))
+    tmd.executorDeserializeCpuTime.foreach(edct => builder.addExecutorDeserializeCpuTime(edct))
+    tmd.executorRunTime.foreach(ert => builder.addExecutorRunTime(ert))
+    tmd.executorCpuTime.foreach(ect => builder.addExecutorCpuTime(ect))
+    tmd.resultSize.foreach(rs => builder.addResultSize(rs))
+    tmd.jvmGcTime.foreach(jgt => builder.addJvmGcTime(jgt))
+    tmd.resultSerializationTime.foreach(rst => builder.addResultSerializationTime(rst))
+    tmd.gettingResultTime.foreach(grt => builder.addGettingResultTime(grt))
+    tmd.schedulerDelay.foreach(sd => builder.addSchedulerDelay(sd))
+    tmd.peakExecutionMemory.foreach(pem => builder.addPeakExecutionMemory(pem))
+    tmd.memoryBytesSpilled.foreach(mbs => builder.addMemoryBytesSpilled(mbs))
+    tmd.diskBytesSpilled.foreach(dbs => builder.addDiskBytesSpilled(dbs))
+    builder
+      .setInputMetrics(serializeInputMetricDistributions(tmd.inputMetrics))
+      .setOutputMetrics(serializeOutputMetricDistributions(tmd.outputMetrics))
+      .setShuffleReadMetrics(serializeShuffleReadMetricDistributions(tmd.shuffleReadMetrics))
+      .setShuffleWriteMetrics(serializeShuffleWriteMetricDistributions(tmd.shuffleWriteMetrics))
+      .build()
+  }
+
+  private def serializeInputMetricDistributions(
+      imd: InputMetricDistributions): StoreTypes.InputMetricDistributions = {
+    val builder = StoreTypes.InputMetricDistributions.newBuilder()
+    imd.bytesRead.foreach(br => builder.addBytesRead(br))
+    imd.recordsRead.foreach(rr => builder.addRecordsRead(rr))
+    builder.build()
+  }
+
+  private def serializeOutputMetricDistributions(
+      omd: OutputMetricDistributions): StoreTypes.OutputMetricDistributions = {
+    val builder = StoreTypes.OutputMetricDistributions.newBuilder()
+    omd.bytesWritten.foreach(bw => builder.addBytesWritten(bw))
+    omd.recordsWritten.foreach(rw => builder.addRecordsWritten(rw))
+    builder.build()
+  }
+
+  private def serializeShuffleReadMetricDistributions(
+      srmd: ShuffleReadMetricDistributions): StoreTypes.ShuffleReadMetricDistributions = {
+    val builder = StoreTypes.ShuffleReadMetricDistributions.newBuilder()
+    srmd.readBytes.foreach(rb => builder.addReadBytes(rb))
+    srmd.readRecords.foreach(rr => builder.addReadRecords(rr))
+    srmd.remoteBlocksFetched.foreach(rbf => builder.addRemoteBlocksFetched(rbf))
+    srmd.localBlocksFetched.foreach(lbf => builder.addLocalBlocksFetched(lbf))
+    srmd.fetchWaitTime.foreach(fwt => builder.addFetchWaitTime(fwt))
+    srmd.remoteBytesRead.foreach(rbr => builder.addRemoteBytesRead(rbr))
+    srmd.remoteBytesReadToDisk.foreach(rbrtd => builder.addRemoteBytesReadToDisk(rbrtd))
+    srmd.totalBlocksFetched.foreach(tbf => builder.addTotalBlocksFetched(tbf))
+    builder.build()
+  }
+
+  private def serializeShuffleWriteMetricDistributions(
+      swmd: ShuffleWriteMetricDistributions): StoreTypes.ShuffleWriteMetricDistributions = {
+    val builder = StoreTypes.ShuffleWriteMetricDistributions.newBuilder()
+    swmd.writeBytes.foreach(wb => builder.addWriteBytes(wb))
+    swmd.writeRecords.foreach(wr => builder.addWriteRecords(wr))
+    swmd.writeTime.foreach(wt => builder.addWriteTime(wt))
+    builder.build()
+  }
+
+  private def serializeExecutorMetricsDistributions(
+      emd: ExecutorMetricsDistributions): StoreTypes.ExecutorMetricsDistributions = {
+    val builder = StoreTypes.ExecutorMetricsDistributions.newBuilder()
+    emd.quantiles.foreach(q => builder.addQuantiles(q))
+    emd.taskTime.foreach(tt => builder.addTaskTime(tt))
+    emd.failedTasks.foreach(ft => builder.addFailedTasks(ft))
+    emd.succeededTasks.foreach(st => builder.addSucceededTasks(st))
+    emd.killedTasks.foreach(kt => builder.addKilledTasks(kt))
+    emd.inputBytes.foreach(ib => builder.addInputBytes(ib))
+    emd.inputRecords.foreach(ir => builder.addInputRecords(ir))
+    emd.outputBytes.foreach(ob => builder.addOutputBytes(ob))
+    emd.outputRecords.foreach(or => builder.addOutputRecords(or))
+    emd.shuffleRead.foreach(sr => builder.addShuffleRead(sr))
+    emd.shuffleReadRecords.foreach(srr => builder.addShuffleReadRecords(srr))
+    emd.shuffleWrite.foreach(sw => builder.addShuffleWrite(sw))
+    emd.shuffleWriteRecords.foreach(swr => builder.addShuffleWriteRecords(swr))
+    emd.memoryBytesSpilled.foreach(mbs => builder.addMemoryBytesSpilled(mbs))
+    emd.diskBytesSpilled.foreach(dbs => builder.addDiskBytesSpilled(dbs))
+    builder.setPeakMemoryMetrics(serializeExecutorPeakMetricsDistributions(emd.peakMemoryMetrics))
+    builder.build()
+  }
+
+  private def serializeExecutorPeakMetricsDistributions(
+      epmd: ExecutorPeakMetricsDistributions): StoreTypes.ExecutorPeakMetricsDistributions = {
+    val builder = StoreTypes.ExecutorPeakMetricsDistributions.newBuilder()
+    epmd.quantiles.foreach(q => builder.addQuantiles(q))
+    epmd.executorMetrics.foreach(em => builder.addExecutorMetrics(
+      ExecutorMetricsSerializer.serialize(em)))
+    builder.build()
+  }
+
+  override def deserialize(bytes: Array[Byte]): StageDataWrapper = {
+    val binary = StoreTypes.StageDataWrapper.parseFrom(bytes)
+    val info = deserializeStageData(binary.getInfo)
+    new StageDataWrapper(
+      info = info,
+      jobIds = binary.getJobIdsList.asScala.map(_.toInt).toSet,
+      locality = binary.getLocalityMap.asScala.mapValues(_.toLong).toMap
+    )
+  }
+
+  private def deserializeStageData(binary: StoreTypes.StageData): StageData = {
+    val status = StageStatus.valueOf(binary.getStatus.toString)
+    val submissionTime =
+      getOptional(binary.hasSubmissionTime, () => new Date(binary.getSubmissionTime))
+    val firstTaskLaunchedTime =
+      getOptional(binary.hasFirstTaskLaunchedTime, () => new Date(binary.getFirstTaskLaunchedTime))
+    val completionTime =
+      getOptional(binary.hasCompletionTime, () => new Date(binary.getCompletionTime))
+    val failureReason =
+      getOptional(binary.hasFailureReason, () => weakIntern(binary.getFailureReason))
+    val description =
+      getOptional(binary.hasDescription, () => weakIntern(binary.getDescription))
+    val accumulatorUpdates = AccumulableInfoSerializer.deserializeAccumulableInfos(
+      binary.getAccumulatorUpdatesList)
+    val tasks = MapUtils.isEmpty(binary.getTasksMap) match {
+      case true => None
+      case _ => Some(binary.getTasksMap.asScala.map(
+        entry => (entry._1.toLong, deserializeTaskData(entry._2))).toMap)
+    }
+    val executorSummary = MapUtils.isEmpty(binary.getExecutorSummaryMap) match {
+      case true => None
+      case _ => Some(binary.getExecutorSummaryMap.asScala.mapValues(
+          ExecutorStageSummarySerializer.deserializeExecutorStageSummary(_)).toMap
+      )
+    }
+    val speculationSummary =
+      getOptional(binary.hasSpeculationSummary,
+        () => deserializeSpeculationStageSummary(binary.getSpeculationSummary))
+    val peakExecutorMetrics =
+      getOptional(binary.hasPeakExecutorMetrics,
+        () => ExecutorMetricsSerializer.deserialize(binary.getPeakExecutorMetrics))
+    val taskMetricsDistributions =
+      getOptional(binary.hasTaskMetricsDistributions,
+        () => deserializeTaskMetricDistributions(binary.getTaskMetricsDistributions))
+    val executorMetricsDistributions =
+      getOptional(binary.hasExecutorMetricsDistributions,
+        () => deserializeExecutorMetricsDistributions(binary.getExecutorMetricsDistributions))
+    new StageData(
+      status = status,
+      stageId = binary.getStageId.toInt,
+      attemptId = binary.getAttemptId,
+      numTasks = binary.getNumTasks,
+      numActiveTasks = binary.getNumActiveTasks,
+      numCompleteTasks = binary.getNumCompleteTasks,
+      numFailedTasks = binary.getNumFailedTasks,
+      numKilledTasks = binary.getNumKilledTasks,
+      numCompletedIndices = binary.getNumCompletedIndices,
+      submissionTime = submissionTime,
+      firstTaskLaunchedTime = firstTaskLaunchedTime,
+      completionTime = completionTime,
+      failureReason = failureReason,
+      executorDeserializeTime = binary.getExecutorDeserializeTime,
+      executorDeserializeCpuTime = binary.getExecutorDeserializeCpuTime,
+      executorRunTime = binary.getExecutorRunTime,
+      executorCpuTime = binary.getExecutorCpuTime,
+      resultSize = binary.getResultSize,
+      jvmGcTime = binary.getJvmGcTime,
+      resultSerializationTime = binary.getResultSerializationTime,
+      memoryBytesSpilled = binary.getMemoryBytesSpilled,
+      diskBytesSpilled = binary.getDiskBytesSpilled,
+      peakExecutionMemory = binary.getPeakExecutionMemory,
+      inputBytes = binary.getInputBytes,
+      inputRecords = binary.getInputRecords,
+      outputBytes = binary.getOutputBytes,
+      outputRecords = binary.getOutputRecords,
+      shuffleRemoteBlocksFetched = binary.getShuffleRemoteBlocksFetched,
+      shuffleLocalBlocksFetched = binary.getShuffleLocalBlocksFetched,
+      shuffleFetchWaitTime = binary.getShuffleFetchWaitTime,
+      shuffleRemoteBytesRead = binary.getShuffleRemoteBytesRead,
+      shuffleRemoteBytesReadToDisk = binary.getShuffleRemoteBytesReadToDisk,
+      shuffleLocalBytesRead = binary.getShuffleLocalBytesRead,
+      shuffleReadBytes = binary.getShuffleReadBytes,
+      shuffleReadRecords = binary.getShuffleReadRecords,
+      shuffleWriteBytes = binary.getShuffleWriteBytes,
+      shuffleWriteTime = binary.getShuffleWriteTime,
+      shuffleWriteRecords = binary.getShuffleWriteRecords,
+      name = weakIntern(binary.getName),
+      description = description,
+      details = weakIntern(binary.getDetails),
+      schedulingPool = weakIntern(binary.getSchedulingPool),
+      rddIds = binary.getRddIdsList.asScala.map(_.toInt).toSeq,
+      accumulatorUpdates = accumulatorUpdates.toSeq,
+      tasks = tasks,
+      executorSummary = executorSummary,
+      speculationSummary = speculationSummary,
+      killedTasksSummary = binary.getKilledTasksSummaryMap.asScala.mapValues(_.toInt).toMap,
+      resourceProfileId = binary.getResourceProfileId,
+      peakExecutorMetrics = peakExecutorMetrics,
+      taskMetricsDistributions = taskMetricsDistributions,
+      executorMetricsDistributions = executorMetricsDistributions
+    )
+  }
+
+  private def deserializeSpeculationStageSummary(
+      binary: StoreTypes.SpeculationStageSummary): SpeculationStageSummary = {
+    new SpeculationStageSummary(
+      binary.getNumTasks,
+      binary.getNumActiveTasks,
+      binary.getNumCompletedTasks,
+      binary.getNumFailedTasks,
+      binary.getNumKilledTasks
+    )
+  }
+
+  private def deserializeTaskMetricDistributions(
+      binary: StoreTypes.TaskMetricDistributions): TaskMetricDistributions = {
+    new TaskMetricDistributions(
+      quantiles = binary.getQuantilesList.asScala.map(_.toDouble).toIndexedSeq,
+      duration = binary.getDurationList.asScala.map(_.toDouble).toIndexedSeq,
+      executorDeserializeTime =
+        binary.getExecutorDeserializeTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      executorDeserializeCpuTime =
+        binary.getExecutorDeserializeCpuTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      executorRunTime = binary.getExecutorRunTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      executorCpuTime = binary.getExecutorCpuTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      resultSize = binary.getResultSizeList.asScala.map(_.toDouble).toIndexedSeq,
+      jvmGcTime = binary.getJvmGcTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      resultSerializationTime =
+        binary.getResultSerializationTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      gettingResultTime = binary.getGettingResultTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      schedulerDelay = binary.getSchedulerDelayList.asScala.map(_.toDouble).toIndexedSeq,
+      peakExecutionMemory = binary.getPeakExecutionMemoryList.asScala.map(_.toDouble).toIndexedSeq,
+      memoryBytesSpilled = binary.getMemoryBytesSpilledList.asScala.map(_.toDouble).toIndexedSeq,
+      diskBytesSpilled = binary.getDiskBytesSpilledList.asScala.map(_.toDouble).toIndexedSeq,
+      inputMetrics = deserializeInputMetricDistributions(binary.getInputMetrics),
+      outputMetrics = deserializeOutputMetricDistributions(binary.getOutputMetrics),
+      shuffleReadMetrics = deserializeShuffleReadMetricDistributions(binary.getShuffleReadMetrics),
+      shuffleWriteMetrics =
+        deserializeShuffleWriteMetricDistributions(binary.getShuffleWriteMetrics)
+    )
+  }
+
+  private def deserializeInputMetricDistributions(
+      binary: StoreTypes.InputMetricDistributions): InputMetricDistributions = {
+    new InputMetricDistributions(
+      bytesRead = binary.getBytesReadList.asScala.map(_.toDouble).toIndexedSeq,
+      recordsRead = binary.getRecordsReadList.asScala.map(_.toDouble).toIndexedSeq
+    )
+  }
+
+  private def deserializeOutputMetricDistributions(
+      binary: StoreTypes.OutputMetricDistributions): OutputMetricDistributions = {
+    new OutputMetricDistributions(
+      bytesWritten = binary.getBytesWrittenList.asScala.map(_.toDouble).toIndexedSeq,
+      recordsWritten = binary.getRecordsWrittenList.asScala.map(_.toDouble).toIndexedSeq
+    )
+  }
+
+  private def deserializeShuffleReadMetricDistributions(
+      binary: StoreTypes.ShuffleReadMetricDistributions): ShuffleReadMetricDistributions = {
+    new ShuffleReadMetricDistributions(
+      readBytes = binary.getReadBytesList.asScala.map(_.toDouble).toIndexedSeq,
+      readRecords = binary.getReadRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      remoteBlocksFetched = binary.getRemoteBlocksFetchedList.asScala.map(_.toDouble).toIndexedSeq,
+      localBlocksFetched = binary.getLocalBlocksFetchedList.asScala.map(_.toDouble).toIndexedSeq,
+      fetchWaitTime = binary.getFetchWaitTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      remoteBytesRead = binary.getRemoteBytesReadList.asScala.map(_.toDouble).toIndexedSeq,
+      remoteBytesReadToDisk =
+        binary.getRemoteBytesReadToDiskList.asScala.map(_.toDouble).toIndexedSeq,
+      totalBlocksFetched = binary.getTotalBlocksFetchedList.asScala.map(_.toDouble).toIndexedSeq
+    )
+  }
+
+  private def deserializeShuffleWriteMetricDistributions(
+      binary: StoreTypes.ShuffleWriteMetricDistributions): ShuffleWriteMetricDistributions = {
+    new ShuffleWriteMetricDistributions(
+      writeBytes = binary.getWriteBytesList.asScala.map(_.toDouble).toIndexedSeq,
+      writeRecords = binary.getWriteRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      writeTime = binary.getWriteTimeList.asScala.map(_.toDouble).toIndexedSeq
+    )
+  }
+
+  private def deserializeExecutorMetricsDistributions(
+      binary: StoreTypes.ExecutorMetricsDistributions): ExecutorMetricsDistributions = {
+    new ExecutorMetricsDistributions(
+      quantiles = binary.getQuantilesList.asScala.map(_.toDouble).toIndexedSeq,
+      taskTime = binary.getTaskTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      failedTasks = binary.getFailedTasksList.asScala.map(_.toDouble).toIndexedSeq,
+      succeededTasks = binary.getSucceededTasksList.asScala.map(_.toDouble).toIndexedSeq,
+      killedTasks = binary.getKilledTasksList.asScala.map(_.toDouble).toIndexedSeq,
+      inputBytes = binary.getInputBytesList.asScala.map(_.toDouble).toIndexedSeq,
+      inputRecords = binary.getInputRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      outputBytes = binary.getOutputBytesList.asScala.map(_.toDouble).toIndexedSeq,
+      outputRecords = binary.getOutputRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      shuffleRead = binary.getShuffleReadList.asScala.map(_.toDouble).toIndexedSeq,
+      shuffleReadRecords = binary.getShuffleReadRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      shuffleWrite = binary.getShuffleWriteList.asScala.map(_.toDouble).toIndexedSeq,
+      shuffleWriteRecords = binary.getShuffleWriteRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      memoryBytesSpilled = binary.getMemoryBytesSpilledList.asScala.map(_.toDouble).toIndexedSeq,
+      diskBytesSpilled = binary.getDiskBytesSpilledList.asScala.map(_.toDouble).toIndexedSeq,
+      peakMemoryMetrics = deserializeExecutorPeakMetricsDistributions(binary.getPeakMemoryMetrics)
+    )
+  }
+
+  private def deserializeExecutorPeakMetricsDistributions(
+      binary: StoreTypes.ExecutorPeakMetricsDistributions): ExecutorPeakMetricsDistributions = {
+    new ExecutorPeakMetricsDistributions(
+      quantiles = binary.getQuantilesList.asScala.map(_.toDouble).toIndexedSeq,
+      executorMetrics = binary.getExecutorMetricsList.asScala.map(
+        ExecutorMetricsSerializer.deserialize(_)).toIndexedSeq
+    )
+  }
+
+  private def deserializeTaskData(binary: StoreTypes.TaskData): TaskData = {
+    val resultFetchStart = getOptional(binary.hasResultFetchStart,
+      () => new Date(binary.getResultFetchStart))
+    val duration = getOptional(binary.hasDuration, () => binary.getDuration)
+    val accumulatorUpdates = AccumulableInfoSerializer.deserializeAccumulableInfos(
+      binary.getAccumulatorUpdatesList)
+    val taskMetrics = getOptional(binary.hasTaskMetrics,
+      () => deserializeTaskMetrics(binary.getTaskMetrics))
+    new TaskData(
+      taskId = binary.getTaskId,
+      index = binary.getIndex,
+      attempt = binary.getAttempt,
+      partitionId = binary.getPartitionId,
+      launchTime = new Date(binary.getLaunchTime),
+      resultFetchStart = resultFetchStart,
+      duration = duration,
+      executorId = weakIntern(binary.getExecutorId),

Review Comment:
   When should we use `weakIntern`? Seems not all serializers use it, will this affect performance?
   
   For example, when `new AccumulableInfo` in `AccumulableInfoSerializer`, we didn't use `weakIntern`
   
   
   
   
   
   



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] LuciferYang commented on pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on PR #39192:
URL: https://github.com/apache/spark/pull/39192#issuecomment-1364964272

   also cc @techaddict 


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gengliangwang closed pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
gengliangwang closed pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper
URL: https://github.com/apache/spark/pull/39192


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
panbingkun commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1058002785


##########
core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto:
##########
@@ -390,3 +390,214 @@ message SQLExecutionUIData {
   repeated int64 stages = 11;
   map<int64, string> metric_values = 12;
 }
+
+message StageDataWrapper {
+  StageData info = 1;
+  repeated int64 job_ids = 2;
+  map<string, int64> locality = 3;
+}
+
+message TaskData {
+  int64 task_id = 1;
+  int32 index = 2;
+  int32 attempt = 3;
+  int32 partition_id = 4;
+  int64 launch_time = 5;
+  optional int64 result_fetch_start = 6;
+  optional int64 duration = 7;
+  string executor_id = 8;
+  string host = 9;
+  string status = 10;
+  string task_locality = 11;
+  bool speculative = 12;
+  repeated AccumulableInfo accumulator_updates = 13;
+  optional string error_message = 14;
+  optional TaskMetrics task_metrics = 15;
+  map<string, string> executor_logs = 16;
+  int64 scheduler_delay = 17;
+  int64 getting_result_time = 18;
+}
+
+message StageData {
+  enum StageStatus {

Review Comment:
   In fact, the enum definition of JobExecutionStatus seems more reasonable in JobData ? 



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
panbingkun commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1058002256


##########
core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto:
##########
@@ -390,3 +390,214 @@ message SQLExecutionUIData {
   repeated int64 stages = 11;
   map<int64, string> metric_values = 12;
 }
+
+message StageDataWrapper {
+  StageData info = 1;
+  repeated int64 job_ids = 2;
+  map<string, int64> locality = 3;
+}
+
+message TaskData {
+  int64 task_id = 1;
+  int32 index = 2;
+  int32 attempt = 3;
+  int32 partition_id = 4;
+  int64 launch_time = 5;
+  optional int64 result_fetch_start = 6;
+  optional int64 duration = 7;
+  string executor_id = 8;
+  string host = 9;
+  string status = 10;
+  string task_locality = 11;
+  bool speculative = 12;
+  repeated AccumulableInfo accumulator_updates = 13;
+  optional string error_message = 14;
+  optional TaskMetrics task_metrics = 15;
+  map<string, string> executor_logs = 16;
+  int64 scheduler_delay = 17;
+  int64 getting_result_time = 18;
+}
+
+message StageData {
+  enum StageStatus {

Review Comment:
   If `StageStatus` is defined outside, the error message is as follows:
   <img width="508" alt="image" src="https://user-images.githubusercontent.com/15246973/209742331-cf5e5af7-3321-41e2-b094-28193e397274.png">
   
   Then If `StageStatus` is defined as follows:
   enum StageStatus {
     STAGE_STATUS_UNSPECIFIED = 0;
     STAGE_STATUS_ACTIVE = 1;
     STAGE_STATUS_COMPLETE = 2;
     STAGE_STATUS_FAILED = 3;
     STAGE_STATUS_PENDING = 4;
     STAGE_STATUS_SKIPPED = 5;
   }
   
   The Code of `Serializer` and `Deerializer` will be very ugly!
   Will have to handle the operations of adding prefix and deleting prefix.
   



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] LuciferYang commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1058190933


##########
core/src/main/scala/org/apache/spark/status/protobuf/StageDataWrapperSerializer.scala:
##########
@@ -0,0 +1,627 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.Date
+
+import collection.JavaConverters._
+import org.apache.commons.collections4.MapUtils
+
+import org.apache.spark.status.StageDataWrapper
+import org.apache.spark.status.api.v1.{ExecutorMetricsDistributions, ExecutorPeakMetricsDistributions, InputMetricDistributions, InputMetrics, OutputMetricDistributions, OutputMetrics, ShuffleReadMetricDistributions, ShuffleReadMetrics, ShuffleWriteMetricDistributions, ShuffleWriteMetrics, SpeculationStageSummary, StageData, StageStatus, TaskData, TaskMetricDistributions, TaskMetrics}
+import org.apache.spark.status.protobuf.Utils.getOptional
+import org.apache.spark.util.Utils.weakIntern
+
+class StageDataWrapperSerializer extends ProtobufSerDe {
+
+  override val supportClass: Class[_] = classOf[StageDataWrapper]
+
+  override def serialize(input: Any): Array[Byte] =

Review Comment:
   we can merge the two `serialize` to one



##########
core/src/main/scala/org/apache/spark/status/protobuf/AccumulableInfoSerializer.scala:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.{List => JList}
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.status.api.v1.AccumulableInfo
+import org.apache.spark.status.protobuf.Utils.getOptional
+
+object AccumulableInfoSerializer {
+
+  private[protobuf] def serializeAccumulableInfo(

Review Comment:
   `serializeAccumulableInfo`  -> `serialize`



##########
core/src/main/scala/org/apache/spark/status/protobuf/AccumulableInfoSerializer.scala:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.{List => JList}
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.status.api.v1.AccumulableInfo
+import org.apache.spark.status.protobuf.Utils.getOptional
+
+object AccumulableInfoSerializer {
+
+  private[protobuf] def serializeAccumulableInfo(
+      input: AccumulableInfo): StoreTypes.AccumulableInfo = {
+    val builder = StoreTypes.AccumulableInfo.newBuilder()
+      .setId(input.id)
+      .setName(input.name)
+      .setValue(input.value)
+    input.update.foreach(builder.setUpdate)
+    builder.build()
+  }
+
+  private[protobuf] def deserializeAccumulableInfos(
+      updates: JList[StoreTypes.AccumulableInfo]): ArrayBuffer[AccumulableInfo] = {
+    val accumulatorUpdates = new ArrayBuffer[AccumulableInfo]()

Review Comment:
   with a `initialSize` ?



##########
core/src/main/scala/org/apache/spark/status/protobuf/StageDataWrapperSerializer.scala:
##########
@@ -0,0 +1,627 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.Date
+
+import collection.JavaConverters._
+import org.apache.commons.collections4.MapUtils
+
+import org.apache.spark.status.StageDataWrapper
+import org.apache.spark.status.api.v1.{ExecutorMetricsDistributions, ExecutorPeakMetricsDistributions, InputMetricDistributions, InputMetrics, OutputMetricDistributions, OutputMetrics, ShuffleReadMetricDistributions, ShuffleReadMetrics, ShuffleWriteMetricDistributions, ShuffleWriteMetrics, SpeculationStageSummary, StageData, StageStatus, TaskData, TaskMetricDistributions, TaskMetrics}
+import org.apache.spark.status.protobuf.Utils.getOptional
+import org.apache.spark.util.Utils.weakIntern
+
+class StageDataWrapperSerializer extends ProtobufSerDe {
+
+  override val supportClass: Class[_] = classOf[StageDataWrapper]
+
+  override def serialize(input: Any): Array[Byte] =
+    serialize(input.asInstanceOf[StageDataWrapper])
+
+  private def serialize(s: StageDataWrapper): Array[Byte] = {
+    val builder = StoreTypes.StageDataWrapper.newBuilder()
+    builder.setInfo(serializeStageData(s.info))
+    s.jobIds.foreach(id => builder.addJobIds(id.toLong))
+    s.locality.foreach { entry =>
+      builder.putLocality(entry._1, entry._2)
+    }
+    builder.build().toByteArray
+  }
+
+  private def serializeStageData(stageData: StageData): StoreTypes.StageData = {
+    val stageDataBuilder = StoreTypes.StageData.newBuilder()
+    stageDataBuilder
+      .setStatus(serializeStageStatus(stageData.status))
+      .setStageId(stageData.stageId.toLong)
+      .setAttemptId(stageData.attemptId)
+      .setNumTasks(stageData.numTasks)
+      .setNumActiveTasks(stageData.numActiveTasks)
+      .setNumCompleteTasks(stageData.numCompleteTasks)
+      .setNumFailedTasks(stageData.numFailedTasks)
+      .setNumKilledTasks(stageData.numKilledTasks)
+      .setNumCompletedIndices(stageData.numCompletedIndices)
+      .setExecutorDeserializeTime(stageData.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(stageData.executorDeserializeCpuTime)
+      .setExecutorRunTime(stageData.executorRunTime)
+      .setExecutorCpuTime(stageData.executorCpuTime)
+      .setResultSize(stageData.resultSize)
+      .setJvmGcTime(stageData.jvmGcTime)
+      .setResultSerializationTime(stageData.resultSerializationTime)
+      .setMemoryBytesSpilled(stageData.memoryBytesSpilled)
+      .setDiskBytesSpilled(stageData.diskBytesSpilled)
+      .setPeakExecutionMemory(stageData.peakExecutionMemory)
+      .setInputBytes(stageData.inputBytes)
+      .setInputRecords(stageData.inputRecords)
+      .setOutputBytes(stageData.outputBytes)
+      .setOutputRecords(stageData.outputRecords)
+      .setShuffleRemoteBlocksFetched(stageData.shuffleRemoteBlocksFetched)
+      .setShuffleLocalBlocksFetched(stageData.shuffleLocalBlocksFetched)
+      .setShuffleFetchWaitTime(stageData.shuffleFetchWaitTime)
+      .setShuffleRemoteBytesRead(stageData.shuffleRemoteBytesRead)
+      .setShuffleRemoteBytesReadToDisk(stageData.shuffleRemoteBytesReadToDisk)
+      .setShuffleLocalBytesRead(stageData.shuffleLocalBytesRead)
+      .setShuffleReadBytes(stageData.shuffleReadBytes)
+      .setShuffleReadRecords(stageData.shuffleReadRecords)
+      .setShuffleWriteBytes(stageData.shuffleWriteBytes)
+      .setShuffleWriteTime(stageData.shuffleWriteTime)
+      .setShuffleWriteRecords(stageData.shuffleWriteRecords)
+      .setName(stageData.name)
+      .setDetails(stageData.details)
+      .setSchedulingPool(stageData.schedulingPool)
+      .setResourceProfileId(stageData.resourceProfileId)
+    stageData.submissionTime.foreach { d =>
+      stageDataBuilder.setSubmissionTime(d.getTime)
+    }
+    stageData.firstTaskLaunchedTime.foreach { d =>
+      stageDataBuilder.setFirstTaskLaunchedTime(d.getTime)
+    }
+    stageData.completionTime.foreach { d =>
+      stageDataBuilder.setCompletionTime(d.getTime)
+    }
+    stageData.failureReason.foreach { fr =>
+      stageDataBuilder.setFailureReason(fr)
+    }
+    stageData.description.foreach { d =>
+      stageDataBuilder.setDescription(d)
+    }
+    stageData.rddIds.foreach(id => stageDataBuilder.addRddIds(id.toLong))
+    stageData.accumulatorUpdates.foreach { update =>
+      stageDataBuilder.addAccumulatorUpdates(
+        AccumulableInfoSerializer.serializeAccumulableInfo(update))
+    }
+    stageData.tasks.foreach { t =>
+      t.foreach { entry =>
+        stageDataBuilder.putTasks(entry._1, serializeTaskData(entry._2))
+      }
+    }
+    stageData.executorSummary.foreach { es =>
+      es.foreach { entry =>
+        stageDataBuilder.putExecutorSummary(entry._1,
+          ExecutorStageSummarySerializer.serializeExecutorStageSummary(entry._2))
+      }
+    }
+    stageData.speculationSummary.foreach { ss =>
+      stageDataBuilder.setSpeculationSummary(serializeSpeculationStageSummary(ss))
+    }
+    stageData.killedTasksSummary.foreach { entry =>
+      stageDataBuilder.putKilledTasksSummary(entry._1, entry._2)
+    }
+    stageData.peakExecutorMetrics.foreach { pem =>
+      stageDataBuilder.setPeakExecutorMetrics(ExecutorMetricsSerializer.serialize(pem))
+    }
+    stageData.taskMetricsDistributions.foreach { tmd =>
+      stageDataBuilder.setTaskMetricsDistributions(serializeTaskMetricDistributions(tmd))
+    }
+    stageData.executorMetricsDistributions.foreach { emd =>
+      stageDataBuilder.setExecutorMetricsDistributions(serializeExecutorMetricsDistributions(emd))
+    }
+    stageDataBuilder.build()
+  }
+
+  private def serializeStageStatus(s: StageStatus): StoreTypes.StageData.StageStatus = {
+    StoreTypes.StageData.StageStatus.valueOf(s.toString)
+  }
+
+  private def serializeTaskData(t: TaskData): StoreTypes.TaskData = {
+    val taskDataBuilder = StoreTypes.TaskData.newBuilder()
+    taskDataBuilder
+      .setTaskId(t.taskId)
+      .setIndex(t.index)
+      .setAttempt(t.attempt)
+      .setPartitionId(t.partitionId)
+      .setLaunchTime(t.launchTime.getTime)
+      .setExecutorId(t.executorId)
+      .setHost(t.host)
+      .setStatus(t.status)
+      .setTaskLocality(t.taskLocality)
+      .setSpeculative(t.speculative)
+      .setSchedulerDelay(t.schedulerDelay)
+      .setGettingResultTime(t.gettingResultTime)
+    t.resultFetchStart.foreach { rfs =>
+      taskDataBuilder.setResultFetchStart(rfs.getTime)
+    }
+    t.duration.foreach { d =>
+      taskDataBuilder.setDuration(d)
+    }
+    t.accumulatorUpdates.foreach { update =>
+      taskDataBuilder.addAccumulatorUpdates(
+        AccumulableInfoSerializer.serializeAccumulableInfo(update))
+    }
+    t.errorMessage.foreach { em =>
+      taskDataBuilder.setErrorMessage(em)
+    }
+    t.taskMetrics.foreach { tm =>
+      taskDataBuilder.setTaskMetrics(serializeTaskMetrics(tm))
+    }
+    t.executorLogs.foreach { entry =>
+      taskDataBuilder.putExecutorLogs(entry._1, entry._2)
+    }
+    taskDataBuilder.build()
+  }
+
+  private def serializeTaskMetrics(tm: TaskMetrics): StoreTypes.TaskMetrics = {
+    val taskMetricsBuilder = StoreTypes.TaskMetrics.newBuilder()
+    taskMetricsBuilder
+      .setExecutorDeserializeTime(tm.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(tm.executorDeserializeCpuTime)
+      .setExecutorRunTime(tm.executorRunTime)
+      .setExecutorCpuTime(tm.executorCpuTime)
+      .setResultSize(tm.resultSize)
+      .setJvmGcTime(tm.jvmGcTime)
+      .setResultSerializationTime(tm.resultSerializationTime)
+      .setMemoryBytesSpilled(tm.memoryBytesSpilled)
+      .setDiskBytesSpilled(tm.diskBytesSpilled)
+      .setPeakExecutionMemory(tm.peakExecutionMemory)
+      .setInputMetrics(serializeInputMetrics(tm.inputMetrics))
+      .setOutputMetrics(serializeOutputMetrics(tm.outputMetrics))
+      .setShuffleReadMetrics(serializeShuffleReadMetrics(tm.shuffleReadMetrics))
+      .setShuffleWriteMetrics(serializeShuffleWriteMetrics(tm.shuffleWriteMetrics))
+    taskMetricsBuilder.build()
+  }
+
+  private def serializeInputMetrics(im: InputMetrics): StoreTypes.InputMetrics = {
+    StoreTypes.InputMetrics.newBuilder()
+      .setBytesRead(im.bytesRead)
+      .setRecordsRead(im.recordsRead)
+      .build()
+  }
+
+  private def serializeOutputMetrics(om: OutputMetrics): StoreTypes.OutputMetrics = {
+    StoreTypes.OutputMetrics.newBuilder()
+      .setBytesWritten(om.bytesWritten)
+      .setRecordsWritten(om.recordsWritten)
+      .build()
+  }
+
+  private def serializeShuffleReadMetrics(
+      srm: ShuffleReadMetrics): StoreTypes.ShuffleReadMetrics = {
+    StoreTypes.ShuffleReadMetrics.newBuilder()
+      .setRemoteBlocksFetched(srm.remoteBlocksFetched)
+      .setLocalBlocksFetched(srm.localBlocksFetched)
+      .setFetchWaitTime(srm.fetchWaitTime)
+      .setRemoteBytesRead(srm.remoteBytesRead)
+      .setRemoteBytesReadToDisk(srm.remoteBytesReadToDisk)
+      .setLocalBytesRead(srm.localBytesRead)
+      .setRecordsRead(srm.recordsRead)
+      .build()
+  }
+
+  private def serializeShuffleWriteMetrics(
+      swm: ShuffleWriteMetrics): StoreTypes.ShuffleWriteMetrics = {
+    StoreTypes.ShuffleWriteMetrics.newBuilder()
+      .setBytesWritten(swm.bytesWritten)
+      .setWriteTime(swm.writeTime)
+      .setRecordsWritten(swm.recordsWritten)
+      .build()
+  }
+
+  private def serializeSpeculationStageSummary(
+      sss: SpeculationStageSummary): StoreTypes.SpeculationStageSummary = {
+    StoreTypes.SpeculationStageSummary.newBuilder()
+      .setNumTasks(sss.numTasks)
+      .setNumActiveTasks(sss.numActiveTasks)
+      .setNumCompletedTasks(sss.numCompletedTasks)
+      .setNumFailedTasks(sss.numFailedTasks)
+      .setNumKilledTasks(sss.numKilledTasks)
+      .build()
+  }
+
+  private def serializeTaskMetricDistributions(
+      tmd: TaskMetricDistributions): StoreTypes.TaskMetricDistributions = {
+    val builder = StoreTypes.TaskMetricDistributions.newBuilder()
+    tmd.quantiles.foreach(q => builder.addQuantiles(q))
+    tmd.duration.foreach(d => builder.addDuration(d))
+    tmd.executorDeserializeTime.foreach(edt => builder.addExecutorDeserializeTime(edt))
+    tmd.executorDeserializeCpuTime.foreach(edct => builder.addExecutorDeserializeCpuTime(edct))
+    tmd.executorRunTime.foreach(ert => builder.addExecutorRunTime(ert))
+    tmd.executorCpuTime.foreach(ect => builder.addExecutorCpuTime(ect))
+    tmd.resultSize.foreach(rs => builder.addResultSize(rs))
+    tmd.jvmGcTime.foreach(jgt => builder.addJvmGcTime(jgt))
+    tmd.resultSerializationTime.foreach(rst => builder.addResultSerializationTime(rst))
+    tmd.gettingResultTime.foreach(grt => builder.addGettingResultTime(grt))
+    tmd.schedulerDelay.foreach(sd => builder.addSchedulerDelay(sd))
+    tmd.peakExecutionMemory.foreach(pem => builder.addPeakExecutionMemory(pem))
+    tmd.memoryBytesSpilled.foreach(mbs => builder.addMemoryBytesSpilled(mbs))
+    tmd.diskBytesSpilled.foreach(dbs => builder.addDiskBytesSpilled(dbs))
+    builder
+      .setInputMetrics(serializeInputMetricDistributions(tmd.inputMetrics))
+      .setOutputMetrics(serializeOutputMetricDistributions(tmd.outputMetrics))
+      .setShuffleReadMetrics(serializeShuffleReadMetricDistributions(tmd.shuffleReadMetrics))
+      .setShuffleWriteMetrics(serializeShuffleWriteMetricDistributions(tmd.shuffleWriteMetrics))
+      .build()
+  }
+
+  private def serializeInputMetricDistributions(
+      imd: InputMetricDistributions): StoreTypes.InputMetricDistributions = {
+    val builder = StoreTypes.InputMetricDistributions.newBuilder()
+    imd.bytesRead.foreach(br => builder.addBytesRead(br))
+    imd.recordsRead.foreach(rr => builder.addRecordsRead(rr))
+    builder.build()
+  }
+
+  private def serializeOutputMetricDistributions(
+      omd: OutputMetricDistributions): StoreTypes.OutputMetricDistributions = {
+    val builder = StoreTypes.OutputMetricDistributions.newBuilder()
+    omd.bytesWritten.foreach(bw => builder.addBytesWritten(bw))
+    omd.recordsWritten.foreach(rw => builder.addRecordsWritten(rw))
+    builder.build()
+  }
+
+  private def serializeShuffleReadMetricDistributions(
+      srmd: ShuffleReadMetricDistributions): StoreTypes.ShuffleReadMetricDistributions = {
+    val builder = StoreTypes.ShuffleReadMetricDistributions.newBuilder()
+    srmd.readBytes.foreach(rb => builder.addReadBytes(rb))
+    srmd.readRecords.foreach(rr => builder.addReadRecords(rr))
+    srmd.remoteBlocksFetched.foreach(rbf => builder.addRemoteBlocksFetched(rbf))
+    srmd.localBlocksFetched.foreach(lbf => builder.addLocalBlocksFetched(lbf))
+    srmd.fetchWaitTime.foreach(fwt => builder.addFetchWaitTime(fwt))
+    srmd.remoteBytesRead.foreach(rbr => builder.addRemoteBytesRead(rbr))
+    srmd.remoteBytesReadToDisk.foreach(rbrtd => builder.addRemoteBytesReadToDisk(rbrtd))
+    srmd.totalBlocksFetched.foreach(tbf => builder.addTotalBlocksFetched(tbf))
+    builder.build()
+  }
+
+  private def serializeShuffleWriteMetricDistributions(
+      swmd: ShuffleWriteMetricDistributions): StoreTypes.ShuffleWriteMetricDistributions = {
+    val builder = StoreTypes.ShuffleWriteMetricDistributions.newBuilder()
+    swmd.writeBytes.foreach(wb => builder.addWriteBytes(wb))
+    swmd.writeRecords.foreach(wr => builder.addWriteRecords(wr))
+    swmd.writeTime.foreach(wt => builder.addWriteTime(wt))
+    builder.build()
+  }
+
+  private def serializeExecutorMetricsDistributions(
+      emd: ExecutorMetricsDistributions): StoreTypes.ExecutorMetricsDistributions = {
+    val builder = StoreTypes.ExecutorMetricsDistributions.newBuilder()
+    emd.quantiles.foreach(q => builder.addQuantiles(q))
+    emd.taskTime.foreach(tt => builder.addTaskTime(tt))
+    emd.failedTasks.foreach(ft => builder.addFailedTasks(ft))
+    emd.succeededTasks.foreach(st => builder.addSucceededTasks(st))
+    emd.killedTasks.foreach(kt => builder.addKilledTasks(kt))
+    emd.inputBytes.foreach(ib => builder.addInputBytes(ib))
+    emd.inputRecords.foreach(ir => builder.addInputRecords(ir))
+    emd.outputBytes.foreach(ob => builder.addOutputBytes(ob))
+    emd.outputRecords.foreach(or => builder.addOutputRecords(or))
+    emd.shuffleRead.foreach(sr => builder.addShuffleRead(sr))
+    emd.shuffleReadRecords.foreach(srr => builder.addShuffleReadRecords(srr))
+    emd.shuffleWrite.foreach(sw => builder.addShuffleWrite(sw))
+    emd.shuffleWriteRecords.foreach(swr => builder.addShuffleWriteRecords(swr))
+    emd.memoryBytesSpilled.foreach(mbs => builder.addMemoryBytesSpilled(mbs))
+    emd.diskBytesSpilled.foreach(dbs => builder.addDiskBytesSpilled(dbs))
+    builder.setPeakMemoryMetrics(serializeExecutorPeakMetricsDistributions(emd.peakMemoryMetrics))
+    builder.build()
+  }
+
+  private def serializeExecutorPeakMetricsDistributions(
+      epmd: ExecutorPeakMetricsDistributions): StoreTypes.ExecutorPeakMetricsDistributions = {
+    val builder = StoreTypes.ExecutorPeakMetricsDistributions.newBuilder()
+    epmd.quantiles.foreach(q => builder.addQuantiles(q))
+    epmd.executorMetrics.foreach(em => builder.addExecutorMetrics(
+      ExecutorMetricsSerializer.serialize(em)))
+    builder.build()
+  }
+
+  override def deserialize(bytes: Array[Byte]): StageDataWrapper = {
+    val binary = StoreTypes.StageDataWrapper.parseFrom(bytes)
+    val info = deserializeStageData(binary.getInfo)
+    new StageDataWrapper(
+      info = info,
+      jobIds = binary.getJobIdsList.asScala.map(_.toInt).toSet,
+      locality = binary.getLocalityMap.asScala.mapValues(_.toLong).toMap
+    )
+  }
+
+  private def deserializeStageData(binary: StoreTypes.StageData): StageData = {
+    val status = StageStatus.valueOf(binary.getStatus.toString)
+    val submissionTime =
+      getOptional(binary.hasSubmissionTime, () => new Date(binary.getSubmissionTime))
+    val firstTaskLaunchedTime =
+      getOptional(binary.hasFirstTaskLaunchedTime, () => new Date(binary.getFirstTaskLaunchedTime))
+    val completionTime =
+      getOptional(binary.hasCompletionTime, () => new Date(binary.getCompletionTime))
+    val failureReason =
+      getOptional(binary.hasFailureReason, () => weakIntern(binary.getFailureReason))
+    val description =
+      getOptional(binary.hasDescription, () => weakIntern(binary.getDescription))
+    val accumulatorUpdates = AccumulableInfoSerializer.deserializeAccumulableInfos(
+      binary.getAccumulatorUpdatesList)
+    val tasks = MapUtils.isEmpty(binary.getTasksMap) match {
+      case true => None
+      case _ => Some(binary.getTasksMap.asScala.map(
+        entry => (entry._1.toLong, deserializeTaskData(entry._2))).toMap)
+    }
+    val executorSummary = MapUtils.isEmpty(binary.getExecutorSummaryMap) match {

Review Comment:
   just true and false, I prefer to `if  {} else {}`



##########
core/src/main/scala/org/apache/spark/status/protobuf/AccumulableInfoSerializer.scala:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.{List => JList}
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.status.api.v1.AccumulableInfo
+import org.apache.spark.status.protobuf.Utils.getOptional
+
+object AccumulableInfoSerializer {
+
+  private[protobuf] def serializeAccumulableInfo(
+      input: AccumulableInfo): StoreTypes.AccumulableInfo = {
+    val builder = StoreTypes.AccumulableInfo.newBuilder()
+      .setId(input.id)
+      .setName(input.name)
+      .setValue(input.value)
+    input.update.foreach(builder.setUpdate)
+    builder.build()
+  }
+
+  private[protobuf] def deserializeAccumulableInfos(

Review Comment:
   `deserializeAccumulableInfos` -> `deserialize`, 
   
   nit: I prefer to  `deserialize(info AccumulableInfo)`, looks more generic, but now is also ok
   
   



##########
core/src/main/scala/org/apache/spark/status/protobuf/StageDataWrapperSerializer.scala:
##########
@@ -0,0 +1,627 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.Date
+
+import collection.JavaConverters._
+import org.apache.commons.collections4.MapUtils
+
+import org.apache.spark.status.StageDataWrapper
+import org.apache.spark.status.api.v1.{ExecutorMetricsDistributions, ExecutorPeakMetricsDistributions, InputMetricDistributions, InputMetrics, OutputMetricDistributions, OutputMetrics, ShuffleReadMetricDistributions, ShuffleReadMetrics, ShuffleWriteMetricDistributions, ShuffleWriteMetrics, SpeculationStageSummary, StageData, StageStatus, TaskData, TaskMetricDistributions, TaskMetrics}
+import org.apache.spark.status.protobuf.Utils.getOptional
+import org.apache.spark.util.Utils.weakIntern
+
+class StageDataWrapperSerializer extends ProtobufSerDe {
+
+  override val supportClass: Class[_] = classOf[StageDataWrapper]
+
+  override def serialize(input: Any): Array[Byte] =
+    serialize(input.asInstanceOf[StageDataWrapper])
+
+  private def serialize(s: StageDataWrapper): Array[Byte] = {
+    val builder = StoreTypes.StageDataWrapper.newBuilder()
+    builder.setInfo(serializeStageData(s.info))
+    s.jobIds.foreach(id => builder.addJobIds(id.toLong))
+    s.locality.foreach { entry =>
+      builder.putLocality(entry._1, entry._2)
+    }
+    builder.build().toByteArray
+  }
+
+  private def serializeStageData(stageData: StageData): StoreTypes.StageData = {
+    val stageDataBuilder = StoreTypes.StageData.newBuilder()
+    stageDataBuilder
+      .setStatus(serializeStageStatus(stageData.status))
+      .setStageId(stageData.stageId.toLong)
+      .setAttemptId(stageData.attemptId)
+      .setNumTasks(stageData.numTasks)
+      .setNumActiveTasks(stageData.numActiveTasks)
+      .setNumCompleteTasks(stageData.numCompleteTasks)
+      .setNumFailedTasks(stageData.numFailedTasks)
+      .setNumKilledTasks(stageData.numKilledTasks)
+      .setNumCompletedIndices(stageData.numCompletedIndices)
+      .setExecutorDeserializeTime(stageData.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(stageData.executorDeserializeCpuTime)
+      .setExecutorRunTime(stageData.executorRunTime)
+      .setExecutorCpuTime(stageData.executorCpuTime)
+      .setResultSize(stageData.resultSize)
+      .setJvmGcTime(stageData.jvmGcTime)
+      .setResultSerializationTime(stageData.resultSerializationTime)
+      .setMemoryBytesSpilled(stageData.memoryBytesSpilled)
+      .setDiskBytesSpilled(stageData.diskBytesSpilled)
+      .setPeakExecutionMemory(stageData.peakExecutionMemory)
+      .setInputBytes(stageData.inputBytes)
+      .setInputRecords(stageData.inputRecords)
+      .setOutputBytes(stageData.outputBytes)
+      .setOutputRecords(stageData.outputRecords)
+      .setShuffleRemoteBlocksFetched(stageData.shuffleRemoteBlocksFetched)
+      .setShuffleLocalBlocksFetched(stageData.shuffleLocalBlocksFetched)
+      .setShuffleFetchWaitTime(stageData.shuffleFetchWaitTime)
+      .setShuffleRemoteBytesRead(stageData.shuffleRemoteBytesRead)
+      .setShuffleRemoteBytesReadToDisk(stageData.shuffleRemoteBytesReadToDisk)
+      .setShuffleLocalBytesRead(stageData.shuffleLocalBytesRead)
+      .setShuffleReadBytes(stageData.shuffleReadBytes)
+      .setShuffleReadRecords(stageData.shuffleReadRecords)
+      .setShuffleWriteBytes(stageData.shuffleWriteBytes)
+      .setShuffleWriteTime(stageData.shuffleWriteTime)
+      .setShuffleWriteRecords(stageData.shuffleWriteRecords)
+      .setName(stageData.name)
+      .setDetails(stageData.details)
+      .setSchedulingPool(stageData.schedulingPool)
+      .setResourceProfileId(stageData.resourceProfileId)
+    stageData.submissionTime.foreach { d =>
+      stageDataBuilder.setSubmissionTime(d.getTime)
+    }
+    stageData.firstTaskLaunchedTime.foreach { d =>
+      stageDataBuilder.setFirstTaskLaunchedTime(d.getTime)
+    }
+    stageData.completionTime.foreach { d =>
+      stageDataBuilder.setCompletionTime(d.getTime)
+    }
+    stageData.failureReason.foreach { fr =>
+      stageDataBuilder.setFailureReason(fr)
+    }
+    stageData.description.foreach { d =>
+      stageDataBuilder.setDescription(d)
+    }
+    stageData.rddIds.foreach(id => stageDataBuilder.addRddIds(id.toLong))
+    stageData.accumulatorUpdates.foreach { update =>
+      stageDataBuilder.addAccumulatorUpdates(
+        AccumulableInfoSerializer.serializeAccumulableInfo(update))
+    }
+    stageData.tasks.foreach { t =>
+      t.foreach { entry =>
+        stageDataBuilder.putTasks(entry._1, serializeTaskData(entry._2))
+      }
+    }
+    stageData.executorSummary.foreach { es =>
+      es.foreach { entry =>
+        stageDataBuilder.putExecutorSummary(entry._1,
+          ExecutorStageSummarySerializer.serializeExecutorStageSummary(entry._2))
+      }
+    }
+    stageData.speculationSummary.foreach { ss =>
+      stageDataBuilder.setSpeculationSummary(serializeSpeculationStageSummary(ss))
+    }
+    stageData.killedTasksSummary.foreach { entry =>
+      stageDataBuilder.putKilledTasksSummary(entry._1, entry._2)
+    }
+    stageData.peakExecutorMetrics.foreach { pem =>
+      stageDataBuilder.setPeakExecutorMetrics(ExecutorMetricsSerializer.serialize(pem))
+    }
+    stageData.taskMetricsDistributions.foreach { tmd =>
+      stageDataBuilder.setTaskMetricsDistributions(serializeTaskMetricDistributions(tmd))
+    }
+    stageData.executorMetricsDistributions.foreach { emd =>
+      stageDataBuilder.setExecutorMetricsDistributions(serializeExecutorMetricsDistributions(emd))
+    }
+    stageDataBuilder.build()
+  }
+
+  private def serializeStageStatus(s: StageStatus): StoreTypes.StageData.StageStatus = {
+    StoreTypes.StageData.StageStatus.valueOf(s.toString)
+  }
+
+  private def serializeTaskData(t: TaskData): StoreTypes.TaskData = {
+    val taskDataBuilder = StoreTypes.TaskData.newBuilder()
+    taskDataBuilder
+      .setTaskId(t.taskId)
+      .setIndex(t.index)
+      .setAttempt(t.attempt)
+      .setPartitionId(t.partitionId)
+      .setLaunchTime(t.launchTime.getTime)
+      .setExecutorId(t.executorId)
+      .setHost(t.host)
+      .setStatus(t.status)
+      .setTaskLocality(t.taskLocality)
+      .setSpeculative(t.speculative)
+      .setSchedulerDelay(t.schedulerDelay)
+      .setGettingResultTime(t.gettingResultTime)
+    t.resultFetchStart.foreach { rfs =>
+      taskDataBuilder.setResultFetchStart(rfs.getTime)
+    }
+    t.duration.foreach { d =>
+      taskDataBuilder.setDuration(d)
+    }
+    t.accumulatorUpdates.foreach { update =>
+      taskDataBuilder.addAccumulatorUpdates(
+        AccumulableInfoSerializer.serializeAccumulableInfo(update))
+    }
+    t.errorMessage.foreach { em =>
+      taskDataBuilder.setErrorMessage(em)
+    }
+    t.taskMetrics.foreach { tm =>
+      taskDataBuilder.setTaskMetrics(serializeTaskMetrics(tm))
+    }
+    t.executorLogs.foreach { entry =>
+      taskDataBuilder.putExecutorLogs(entry._1, entry._2)
+    }
+    taskDataBuilder.build()
+  }
+
+  private def serializeTaskMetrics(tm: TaskMetrics): StoreTypes.TaskMetrics = {
+    val taskMetricsBuilder = StoreTypes.TaskMetrics.newBuilder()
+    taskMetricsBuilder
+      .setExecutorDeserializeTime(tm.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(tm.executorDeserializeCpuTime)
+      .setExecutorRunTime(tm.executorRunTime)
+      .setExecutorCpuTime(tm.executorCpuTime)
+      .setResultSize(tm.resultSize)
+      .setJvmGcTime(tm.jvmGcTime)
+      .setResultSerializationTime(tm.resultSerializationTime)
+      .setMemoryBytesSpilled(tm.memoryBytesSpilled)
+      .setDiskBytesSpilled(tm.diskBytesSpilled)
+      .setPeakExecutionMemory(tm.peakExecutionMemory)
+      .setInputMetrics(serializeInputMetrics(tm.inputMetrics))
+      .setOutputMetrics(serializeOutputMetrics(tm.outputMetrics))
+      .setShuffleReadMetrics(serializeShuffleReadMetrics(tm.shuffleReadMetrics))
+      .setShuffleWriteMetrics(serializeShuffleWriteMetrics(tm.shuffleWriteMetrics))
+    taskMetricsBuilder.build()
+  }
+
+  private def serializeInputMetrics(im: InputMetrics): StoreTypes.InputMetrics = {
+    StoreTypes.InputMetrics.newBuilder()
+      .setBytesRead(im.bytesRead)
+      .setRecordsRead(im.recordsRead)
+      .build()
+  }
+
+  private def serializeOutputMetrics(om: OutputMetrics): StoreTypes.OutputMetrics = {
+    StoreTypes.OutputMetrics.newBuilder()
+      .setBytesWritten(om.bytesWritten)
+      .setRecordsWritten(om.recordsWritten)
+      .build()
+  }
+
+  private def serializeShuffleReadMetrics(
+      srm: ShuffleReadMetrics): StoreTypes.ShuffleReadMetrics = {
+    StoreTypes.ShuffleReadMetrics.newBuilder()
+      .setRemoteBlocksFetched(srm.remoteBlocksFetched)
+      .setLocalBlocksFetched(srm.localBlocksFetched)
+      .setFetchWaitTime(srm.fetchWaitTime)
+      .setRemoteBytesRead(srm.remoteBytesRead)
+      .setRemoteBytesReadToDisk(srm.remoteBytesReadToDisk)
+      .setLocalBytesRead(srm.localBytesRead)
+      .setRecordsRead(srm.recordsRead)
+      .build()
+  }
+
+  private def serializeShuffleWriteMetrics(
+      swm: ShuffleWriteMetrics): StoreTypes.ShuffleWriteMetrics = {
+    StoreTypes.ShuffleWriteMetrics.newBuilder()
+      .setBytesWritten(swm.bytesWritten)
+      .setWriteTime(swm.writeTime)
+      .setRecordsWritten(swm.recordsWritten)
+      .build()
+  }
+
+  private def serializeSpeculationStageSummary(
+      sss: SpeculationStageSummary): StoreTypes.SpeculationStageSummary = {
+    StoreTypes.SpeculationStageSummary.newBuilder()
+      .setNumTasks(sss.numTasks)
+      .setNumActiveTasks(sss.numActiveTasks)
+      .setNumCompletedTasks(sss.numCompletedTasks)
+      .setNumFailedTasks(sss.numFailedTasks)
+      .setNumKilledTasks(sss.numKilledTasks)
+      .build()
+  }
+
+  private def serializeTaskMetricDistributions(
+      tmd: TaskMetricDistributions): StoreTypes.TaskMetricDistributions = {
+    val builder = StoreTypes.TaskMetricDistributions.newBuilder()
+    tmd.quantiles.foreach(q => builder.addQuantiles(q))
+    tmd.duration.foreach(d => builder.addDuration(d))
+    tmd.executorDeserializeTime.foreach(edt => builder.addExecutorDeserializeTime(edt))
+    tmd.executorDeserializeCpuTime.foreach(edct => builder.addExecutorDeserializeCpuTime(edct))
+    tmd.executorRunTime.foreach(ert => builder.addExecutorRunTime(ert))
+    tmd.executorCpuTime.foreach(ect => builder.addExecutorCpuTime(ect))
+    tmd.resultSize.foreach(rs => builder.addResultSize(rs))
+    tmd.jvmGcTime.foreach(jgt => builder.addJvmGcTime(jgt))
+    tmd.resultSerializationTime.foreach(rst => builder.addResultSerializationTime(rst))
+    tmd.gettingResultTime.foreach(grt => builder.addGettingResultTime(grt))
+    tmd.schedulerDelay.foreach(sd => builder.addSchedulerDelay(sd))
+    tmd.peakExecutionMemory.foreach(pem => builder.addPeakExecutionMemory(pem))
+    tmd.memoryBytesSpilled.foreach(mbs => builder.addMemoryBytesSpilled(mbs))
+    tmd.diskBytesSpilled.foreach(dbs => builder.addDiskBytesSpilled(dbs))
+    builder
+      .setInputMetrics(serializeInputMetricDistributions(tmd.inputMetrics))
+      .setOutputMetrics(serializeOutputMetricDistributions(tmd.outputMetrics))
+      .setShuffleReadMetrics(serializeShuffleReadMetricDistributions(tmd.shuffleReadMetrics))
+      .setShuffleWriteMetrics(serializeShuffleWriteMetricDistributions(tmd.shuffleWriteMetrics))
+      .build()
+  }
+
+  private def serializeInputMetricDistributions(
+      imd: InputMetricDistributions): StoreTypes.InputMetricDistributions = {
+    val builder = StoreTypes.InputMetricDistributions.newBuilder()
+    imd.bytesRead.foreach(br => builder.addBytesRead(br))
+    imd.recordsRead.foreach(rr => builder.addRecordsRead(rr))
+    builder.build()
+  }
+
+  private def serializeOutputMetricDistributions(
+      omd: OutputMetricDistributions): StoreTypes.OutputMetricDistributions = {
+    val builder = StoreTypes.OutputMetricDistributions.newBuilder()
+    omd.bytesWritten.foreach(bw => builder.addBytesWritten(bw))
+    omd.recordsWritten.foreach(rw => builder.addRecordsWritten(rw))
+    builder.build()
+  }
+
+  private def serializeShuffleReadMetricDistributions(
+      srmd: ShuffleReadMetricDistributions): StoreTypes.ShuffleReadMetricDistributions = {
+    val builder = StoreTypes.ShuffleReadMetricDistributions.newBuilder()
+    srmd.readBytes.foreach(rb => builder.addReadBytes(rb))
+    srmd.readRecords.foreach(rr => builder.addReadRecords(rr))
+    srmd.remoteBlocksFetched.foreach(rbf => builder.addRemoteBlocksFetched(rbf))
+    srmd.localBlocksFetched.foreach(lbf => builder.addLocalBlocksFetched(lbf))
+    srmd.fetchWaitTime.foreach(fwt => builder.addFetchWaitTime(fwt))
+    srmd.remoteBytesRead.foreach(rbr => builder.addRemoteBytesRead(rbr))
+    srmd.remoteBytesReadToDisk.foreach(rbrtd => builder.addRemoteBytesReadToDisk(rbrtd))
+    srmd.totalBlocksFetched.foreach(tbf => builder.addTotalBlocksFetched(tbf))
+    builder.build()
+  }
+
+  private def serializeShuffleWriteMetricDistributions(
+      swmd: ShuffleWriteMetricDistributions): StoreTypes.ShuffleWriteMetricDistributions = {
+    val builder = StoreTypes.ShuffleWriteMetricDistributions.newBuilder()
+    swmd.writeBytes.foreach(wb => builder.addWriteBytes(wb))
+    swmd.writeRecords.foreach(wr => builder.addWriteRecords(wr))
+    swmd.writeTime.foreach(wt => builder.addWriteTime(wt))
+    builder.build()
+  }
+
+  private def serializeExecutorMetricsDistributions(
+      emd: ExecutorMetricsDistributions): StoreTypes.ExecutorMetricsDistributions = {
+    val builder = StoreTypes.ExecutorMetricsDistributions.newBuilder()
+    emd.quantiles.foreach(q => builder.addQuantiles(q))
+    emd.taskTime.foreach(tt => builder.addTaskTime(tt))
+    emd.failedTasks.foreach(ft => builder.addFailedTasks(ft))
+    emd.succeededTasks.foreach(st => builder.addSucceededTasks(st))
+    emd.killedTasks.foreach(kt => builder.addKilledTasks(kt))
+    emd.inputBytes.foreach(ib => builder.addInputBytes(ib))
+    emd.inputRecords.foreach(ir => builder.addInputRecords(ir))
+    emd.outputBytes.foreach(ob => builder.addOutputBytes(ob))
+    emd.outputRecords.foreach(or => builder.addOutputRecords(or))
+    emd.shuffleRead.foreach(sr => builder.addShuffleRead(sr))
+    emd.shuffleReadRecords.foreach(srr => builder.addShuffleReadRecords(srr))
+    emd.shuffleWrite.foreach(sw => builder.addShuffleWrite(sw))
+    emd.shuffleWriteRecords.foreach(swr => builder.addShuffleWriteRecords(swr))
+    emd.memoryBytesSpilled.foreach(mbs => builder.addMemoryBytesSpilled(mbs))
+    emd.diskBytesSpilled.foreach(dbs => builder.addDiskBytesSpilled(dbs))
+    builder.setPeakMemoryMetrics(serializeExecutorPeakMetricsDistributions(emd.peakMemoryMetrics))
+    builder.build()
+  }
+
+  private def serializeExecutorPeakMetricsDistributions(
+      epmd: ExecutorPeakMetricsDistributions): StoreTypes.ExecutorPeakMetricsDistributions = {
+    val builder = StoreTypes.ExecutorPeakMetricsDistributions.newBuilder()
+    epmd.quantiles.foreach(q => builder.addQuantiles(q))
+    epmd.executorMetrics.foreach(em => builder.addExecutorMetrics(
+      ExecutorMetricsSerializer.serialize(em)))
+    builder.build()
+  }
+
+  override def deserialize(bytes: Array[Byte]): StageDataWrapper = {
+    val binary = StoreTypes.StageDataWrapper.parseFrom(bytes)
+    val info = deserializeStageData(binary.getInfo)
+    new StageDataWrapper(
+      info = info,
+      jobIds = binary.getJobIdsList.asScala.map(_.toInt).toSet,
+      locality = binary.getLocalityMap.asScala.mapValues(_.toLong).toMap
+    )
+  }
+
+  private def deserializeStageData(binary: StoreTypes.StageData): StageData = {
+    val status = StageStatus.valueOf(binary.getStatus.toString)
+    val submissionTime =
+      getOptional(binary.hasSubmissionTime, () => new Date(binary.getSubmissionTime))
+    val firstTaskLaunchedTime =
+      getOptional(binary.hasFirstTaskLaunchedTime, () => new Date(binary.getFirstTaskLaunchedTime))
+    val completionTime =
+      getOptional(binary.hasCompletionTime, () => new Date(binary.getCompletionTime))
+    val failureReason =
+      getOptional(binary.hasFailureReason, () => weakIntern(binary.getFailureReason))
+    val description =
+      getOptional(binary.hasDescription, () => weakIntern(binary.getDescription))
+    val accumulatorUpdates = AccumulableInfoSerializer.deserializeAccumulableInfos(
+      binary.getAccumulatorUpdatesList)
+    val tasks = MapUtils.isEmpty(binary.getTasksMap) match {
+      case true => None
+      case _ => Some(binary.getTasksMap.asScala.map(
+        entry => (entry._1.toLong, deserializeTaskData(entry._2))).toMap)
+    }
+    val executorSummary = MapUtils.isEmpty(binary.getExecutorSummaryMap) match {
+      case true => None
+      case _ => Some(binary.getExecutorSummaryMap.asScala.mapValues(
+          ExecutorStageSummarySerializer.deserializeExecutorStageSummary(_)).toMap

Review Comment:
   can convertible to a method value 



##########
core/src/main/scala/org/apache/spark/status/protobuf/StageDataWrapperSerializer.scala:
##########
@@ -0,0 +1,627 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.Date
+
+import collection.JavaConverters._
+import org.apache.commons.collections4.MapUtils
+
+import org.apache.spark.status.StageDataWrapper
+import org.apache.spark.status.api.v1.{ExecutorMetricsDistributions, ExecutorPeakMetricsDistributions, InputMetricDistributions, InputMetrics, OutputMetricDistributions, OutputMetrics, ShuffleReadMetricDistributions, ShuffleReadMetrics, ShuffleWriteMetricDistributions, ShuffleWriteMetrics, SpeculationStageSummary, StageData, StageStatus, TaskData, TaskMetricDistributions, TaskMetrics}
+import org.apache.spark.status.protobuf.Utils.getOptional
+import org.apache.spark.util.Utils.weakIntern
+
+class StageDataWrapperSerializer extends ProtobufSerDe {
+
+  override val supportClass: Class[_] = classOf[StageDataWrapper]
+
+  override def serialize(input: Any): Array[Byte] =
+    serialize(input.asInstanceOf[StageDataWrapper])
+
+  private def serialize(s: StageDataWrapper): Array[Byte] = {
+    val builder = StoreTypes.StageDataWrapper.newBuilder()
+    builder.setInfo(serializeStageData(s.info))
+    s.jobIds.foreach(id => builder.addJobIds(id.toLong))
+    s.locality.foreach { entry =>
+      builder.putLocality(entry._1, entry._2)
+    }
+    builder.build().toByteArray
+  }
+
+  private def serializeStageData(stageData: StageData): StoreTypes.StageData = {
+    val stageDataBuilder = StoreTypes.StageData.newBuilder()
+    stageDataBuilder
+      .setStatus(serializeStageStatus(stageData.status))
+      .setStageId(stageData.stageId.toLong)
+      .setAttemptId(stageData.attemptId)
+      .setNumTasks(stageData.numTasks)
+      .setNumActiveTasks(stageData.numActiveTasks)
+      .setNumCompleteTasks(stageData.numCompleteTasks)
+      .setNumFailedTasks(stageData.numFailedTasks)
+      .setNumKilledTasks(stageData.numKilledTasks)
+      .setNumCompletedIndices(stageData.numCompletedIndices)
+      .setExecutorDeserializeTime(stageData.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(stageData.executorDeserializeCpuTime)
+      .setExecutorRunTime(stageData.executorRunTime)
+      .setExecutorCpuTime(stageData.executorCpuTime)
+      .setResultSize(stageData.resultSize)
+      .setJvmGcTime(stageData.jvmGcTime)
+      .setResultSerializationTime(stageData.resultSerializationTime)
+      .setMemoryBytesSpilled(stageData.memoryBytesSpilled)
+      .setDiskBytesSpilled(stageData.diskBytesSpilled)
+      .setPeakExecutionMemory(stageData.peakExecutionMemory)
+      .setInputBytes(stageData.inputBytes)
+      .setInputRecords(stageData.inputRecords)
+      .setOutputBytes(stageData.outputBytes)
+      .setOutputRecords(stageData.outputRecords)
+      .setShuffleRemoteBlocksFetched(stageData.shuffleRemoteBlocksFetched)
+      .setShuffleLocalBlocksFetched(stageData.shuffleLocalBlocksFetched)
+      .setShuffleFetchWaitTime(stageData.shuffleFetchWaitTime)
+      .setShuffleRemoteBytesRead(stageData.shuffleRemoteBytesRead)
+      .setShuffleRemoteBytesReadToDisk(stageData.shuffleRemoteBytesReadToDisk)
+      .setShuffleLocalBytesRead(stageData.shuffleLocalBytesRead)
+      .setShuffleReadBytes(stageData.shuffleReadBytes)
+      .setShuffleReadRecords(stageData.shuffleReadRecords)
+      .setShuffleWriteBytes(stageData.shuffleWriteBytes)
+      .setShuffleWriteTime(stageData.shuffleWriteTime)
+      .setShuffleWriteRecords(stageData.shuffleWriteRecords)
+      .setName(stageData.name)
+      .setDetails(stageData.details)
+      .setSchedulingPool(stageData.schedulingPool)
+      .setResourceProfileId(stageData.resourceProfileId)
+    stageData.submissionTime.foreach { d =>
+      stageDataBuilder.setSubmissionTime(d.getTime)
+    }
+    stageData.firstTaskLaunchedTime.foreach { d =>
+      stageDataBuilder.setFirstTaskLaunchedTime(d.getTime)
+    }
+    stageData.completionTime.foreach { d =>
+      stageDataBuilder.setCompletionTime(d.getTime)
+    }
+    stageData.failureReason.foreach { fr =>
+      stageDataBuilder.setFailureReason(fr)
+    }
+    stageData.description.foreach { d =>
+      stageDataBuilder.setDescription(d)
+    }
+    stageData.rddIds.foreach(id => stageDataBuilder.addRddIds(id.toLong))
+    stageData.accumulatorUpdates.foreach { update =>
+      stageDataBuilder.addAccumulatorUpdates(
+        AccumulableInfoSerializer.serializeAccumulableInfo(update))
+    }
+    stageData.tasks.foreach { t =>
+      t.foreach { entry =>
+        stageDataBuilder.putTasks(entry._1, serializeTaskData(entry._2))
+      }
+    }
+    stageData.executorSummary.foreach { es =>
+      es.foreach { entry =>
+        stageDataBuilder.putExecutorSummary(entry._1,
+          ExecutorStageSummarySerializer.serializeExecutorStageSummary(entry._2))
+      }
+    }
+    stageData.speculationSummary.foreach { ss =>
+      stageDataBuilder.setSpeculationSummary(serializeSpeculationStageSummary(ss))
+    }
+    stageData.killedTasksSummary.foreach { entry =>
+      stageDataBuilder.putKilledTasksSummary(entry._1, entry._2)
+    }
+    stageData.peakExecutorMetrics.foreach { pem =>
+      stageDataBuilder.setPeakExecutorMetrics(ExecutorMetricsSerializer.serialize(pem))
+    }
+    stageData.taskMetricsDistributions.foreach { tmd =>
+      stageDataBuilder.setTaskMetricsDistributions(serializeTaskMetricDistributions(tmd))
+    }
+    stageData.executorMetricsDistributions.foreach { emd =>
+      stageDataBuilder.setExecutorMetricsDistributions(serializeExecutorMetricsDistributions(emd))
+    }
+    stageDataBuilder.build()
+  }
+
+  private def serializeStageStatus(s: StageStatus): StoreTypes.StageData.StageStatus = {
+    StoreTypes.StageData.StageStatus.valueOf(s.toString)
+  }
+
+  private def serializeTaskData(t: TaskData): StoreTypes.TaskData = {
+    val taskDataBuilder = StoreTypes.TaskData.newBuilder()
+    taskDataBuilder
+      .setTaskId(t.taskId)
+      .setIndex(t.index)
+      .setAttempt(t.attempt)
+      .setPartitionId(t.partitionId)
+      .setLaunchTime(t.launchTime.getTime)
+      .setExecutorId(t.executorId)
+      .setHost(t.host)
+      .setStatus(t.status)
+      .setTaskLocality(t.taskLocality)
+      .setSpeculative(t.speculative)
+      .setSchedulerDelay(t.schedulerDelay)
+      .setGettingResultTime(t.gettingResultTime)
+    t.resultFetchStart.foreach { rfs =>
+      taskDataBuilder.setResultFetchStart(rfs.getTime)
+    }
+    t.duration.foreach { d =>
+      taskDataBuilder.setDuration(d)
+    }
+    t.accumulatorUpdates.foreach { update =>
+      taskDataBuilder.addAccumulatorUpdates(
+        AccumulableInfoSerializer.serializeAccumulableInfo(update))
+    }
+    t.errorMessage.foreach { em =>
+      taskDataBuilder.setErrorMessage(em)
+    }
+    t.taskMetrics.foreach { tm =>
+      taskDataBuilder.setTaskMetrics(serializeTaskMetrics(tm))
+    }
+    t.executorLogs.foreach { entry =>
+      taskDataBuilder.putExecutorLogs(entry._1, entry._2)
+    }
+    taskDataBuilder.build()
+  }
+
+  private def serializeTaskMetrics(tm: TaskMetrics): StoreTypes.TaskMetrics = {
+    val taskMetricsBuilder = StoreTypes.TaskMetrics.newBuilder()
+    taskMetricsBuilder
+      .setExecutorDeserializeTime(tm.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(tm.executorDeserializeCpuTime)
+      .setExecutorRunTime(tm.executorRunTime)
+      .setExecutorCpuTime(tm.executorCpuTime)
+      .setResultSize(tm.resultSize)
+      .setJvmGcTime(tm.jvmGcTime)
+      .setResultSerializationTime(tm.resultSerializationTime)
+      .setMemoryBytesSpilled(tm.memoryBytesSpilled)
+      .setDiskBytesSpilled(tm.diskBytesSpilled)
+      .setPeakExecutionMemory(tm.peakExecutionMemory)
+      .setInputMetrics(serializeInputMetrics(tm.inputMetrics))
+      .setOutputMetrics(serializeOutputMetrics(tm.outputMetrics))
+      .setShuffleReadMetrics(serializeShuffleReadMetrics(tm.shuffleReadMetrics))
+      .setShuffleWriteMetrics(serializeShuffleWriteMetrics(tm.shuffleWriteMetrics))
+    taskMetricsBuilder.build()
+  }
+
+  private def serializeInputMetrics(im: InputMetrics): StoreTypes.InputMetrics = {
+    StoreTypes.InputMetrics.newBuilder()
+      .setBytesRead(im.bytesRead)
+      .setRecordsRead(im.recordsRead)
+      .build()
+  }
+
+  private def serializeOutputMetrics(om: OutputMetrics): StoreTypes.OutputMetrics = {
+    StoreTypes.OutputMetrics.newBuilder()
+      .setBytesWritten(om.bytesWritten)
+      .setRecordsWritten(om.recordsWritten)
+      .build()
+  }
+
+  private def serializeShuffleReadMetrics(
+      srm: ShuffleReadMetrics): StoreTypes.ShuffleReadMetrics = {
+    StoreTypes.ShuffleReadMetrics.newBuilder()
+      .setRemoteBlocksFetched(srm.remoteBlocksFetched)
+      .setLocalBlocksFetched(srm.localBlocksFetched)
+      .setFetchWaitTime(srm.fetchWaitTime)
+      .setRemoteBytesRead(srm.remoteBytesRead)
+      .setRemoteBytesReadToDisk(srm.remoteBytesReadToDisk)
+      .setLocalBytesRead(srm.localBytesRead)
+      .setRecordsRead(srm.recordsRead)
+      .build()
+  }
+
+  private def serializeShuffleWriteMetrics(
+      swm: ShuffleWriteMetrics): StoreTypes.ShuffleWriteMetrics = {
+    StoreTypes.ShuffleWriteMetrics.newBuilder()
+      .setBytesWritten(swm.bytesWritten)
+      .setWriteTime(swm.writeTime)
+      .setRecordsWritten(swm.recordsWritten)
+      .build()
+  }
+
+  private def serializeSpeculationStageSummary(
+      sss: SpeculationStageSummary): StoreTypes.SpeculationStageSummary = {
+    StoreTypes.SpeculationStageSummary.newBuilder()
+      .setNumTasks(sss.numTasks)
+      .setNumActiveTasks(sss.numActiveTasks)
+      .setNumCompletedTasks(sss.numCompletedTasks)
+      .setNumFailedTasks(sss.numFailedTasks)
+      .setNumKilledTasks(sss.numKilledTasks)
+      .build()
+  }
+
+  private def serializeTaskMetricDistributions(
+      tmd: TaskMetricDistributions): StoreTypes.TaskMetricDistributions = {
+    val builder = StoreTypes.TaskMetricDistributions.newBuilder()
+    tmd.quantiles.foreach(q => builder.addQuantiles(q))
+    tmd.duration.foreach(d => builder.addDuration(d))
+    tmd.executorDeserializeTime.foreach(edt => builder.addExecutorDeserializeTime(edt))
+    tmd.executorDeserializeCpuTime.foreach(edct => builder.addExecutorDeserializeCpuTime(edct))
+    tmd.executorRunTime.foreach(ert => builder.addExecutorRunTime(ert))
+    tmd.executorCpuTime.foreach(ect => builder.addExecutorCpuTime(ect))
+    tmd.resultSize.foreach(rs => builder.addResultSize(rs))
+    tmd.jvmGcTime.foreach(jgt => builder.addJvmGcTime(jgt))
+    tmd.resultSerializationTime.foreach(rst => builder.addResultSerializationTime(rst))
+    tmd.gettingResultTime.foreach(grt => builder.addGettingResultTime(grt))
+    tmd.schedulerDelay.foreach(sd => builder.addSchedulerDelay(sd))
+    tmd.peakExecutionMemory.foreach(pem => builder.addPeakExecutionMemory(pem))
+    tmd.memoryBytesSpilled.foreach(mbs => builder.addMemoryBytesSpilled(mbs))
+    tmd.diskBytesSpilled.foreach(dbs => builder.addDiskBytesSpilled(dbs))
+    builder
+      .setInputMetrics(serializeInputMetricDistributions(tmd.inputMetrics))
+      .setOutputMetrics(serializeOutputMetricDistributions(tmd.outputMetrics))
+      .setShuffleReadMetrics(serializeShuffleReadMetricDistributions(tmd.shuffleReadMetrics))
+      .setShuffleWriteMetrics(serializeShuffleWriteMetricDistributions(tmd.shuffleWriteMetrics))
+      .build()
+  }
+
+  private def serializeInputMetricDistributions(
+      imd: InputMetricDistributions): StoreTypes.InputMetricDistributions = {
+    val builder = StoreTypes.InputMetricDistributions.newBuilder()
+    imd.bytesRead.foreach(br => builder.addBytesRead(br))
+    imd.recordsRead.foreach(rr => builder.addRecordsRead(rr))
+    builder.build()
+  }
+
+  private def serializeOutputMetricDistributions(
+      omd: OutputMetricDistributions): StoreTypes.OutputMetricDistributions = {
+    val builder = StoreTypes.OutputMetricDistributions.newBuilder()
+    omd.bytesWritten.foreach(bw => builder.addBytesWritten(bw))
+    omd.recordsWritten.foreach(rw => builder.addRecordsWritten(rw))
+    builder.build()
+  }
+
+  private def serializeShuffleReadMetricDistributions(
+      srmd: ShuffleReadMetricDistributions): StoreTypes.ShuffleReadMetricDistributions = {
+    val builder = StoreTypes.ShuffleReadMetricDistributions.newBuilder()
+    srmd.readBytes.foreach(rb => builder.addReadBytes(rb))
+    srmd.readRecords.foreach(rr => builder.addReadRecords(rr))
+    srmd.remoteBlocksFetched.foreach(rbf => builder.addRemoteBlocksFetched(rbf))
+    srmd.localBlocksFetched.foreach(lbf => builder.addLocalBlocksFetched(lbf))
+    srmd.fetchWaitTime.foreach(fwt => builder.addFetchWaitTime(fwt))
+    srmd.remoteBytesRead.foreach(rbr => builder.addRemoteBytesRead(rbr))
+    srmd.remoteBytesReadToDisk.foreach(rbrtd => builder.addRemoteBytesReadToDisk(rbrtd))
+    srmd.totalBlocksFetched.foreach(tbf => builder.addTotalBlocksFetched(tbf))
+    builder.build()
+  }
+
+  private def serializeShuffleWriteMetricDistributions(
+      swmd: ShuffleWriteMetricDistributions): StoreTypes.ShuffleWriteMetricDistributions = {
+    val builder = StoreTypes.ShuffleWriteMetricDistributions.newBuilder()
+    swmd.writeBytes.foreach(wb => builder.addWriteBytes(wb))
+    swmd.writeRecords.foreach(wr => builder.addWriteRecords(wr))
+    swmd.writeTime.foreach(wt => builder.addWriteTime(wt))
+    builder.build()
+  }
+
+  private def serializeExecutorMetricsDistributions(
+      emd: ExecutorMetricsDistributions): StoreTypes.ExecutorMetricsDistributions = {
+    val builder = StoreTypes.ExecutorMetricsDistributions.newBuilder()
+    emd.quantiles.foreach(q => builder.addQuantiles(q))
+    emd.taskTime.foreach(tt => builder.addTaskTime(tt))
+    emd.failedTasks.foreach(ft => builder.addFailedTasks(ft))
+    emd.succeededTasks.foreach(st => builder.addSucceededTasks(st))
+    emd.killedTasks.foreach(kt => builder.addKilledTasks(kt))
+    emd.inputBytes.foreach(ib => builder.addInputBytes(ib))
+    emd.inputRecords.foreach(ir => builder.addInputRecords(ir))
+    emd.outputBytes.foreach(ob => builder.addOutputBytes(ob))
+    emd.outputRecords.foreach(or => builder.addOutputRecords(or))
+    emd.shuffleRead.foreach(sr => builder.addShuffleRead(sr))
+    emd.shuffleReadRecords.foreach(srr => builder.addShuffleReadRecords(srr))
+    emd.shuffleWrite.foreach(sw => builder.addShuffleWrite(sw))
+    emd.shuffleWriteRecords.foreach(swr => builder.addShuffleWriteRecords(swr))
+    emd.memoryBytesSpilled.foreach(mbs => builder.addMemoryBytesSpilled(mbs))
+    emd.diskBytesSpilled.foreach(dbs => builder.addDiskBytesSpilled(dbs))
+    builder.setPeakMemoryMetrics(serializeExecutorPeakMetricsDistributions(emd.peakMemoryMetrics))
+    builder.build()
+  }
+
+  private def serializeExecutorPeakMetricsDistributions(
+      epmd: ExecutorPeakMetricsDistributions): StoreTypes.ExecutorPeakMetricsDistributions = {
+    val builder = StoreTypes.ExecutorPeakMetricsDistributions.newBuilder()
+    epmd.quantiles.foreach(q => builder.addQuantiles(q))
+    epmd.executorMetrics.foreach(em => builder.addExecutorMetrics(
+      ExecutorMetricsSerializer.serialize(em)))
+    builder.build()
+  }
+
+  override def deserialize(bytes: Array[Byte]): StageDataWrapper = {
+    val binary = StoreTypes.StageDataWrapper.parseFrom(bytes)
+    val info = deserializeStageData(binary.getInfo)
+    new StageDataWrapper(
+      info = info,
+      jobIds = binary.getJobIdsList.asScala.map(_.toInt).toSet,
+      locality = binary.getLocalityMap.asScala.mapValues(_.toLong).toMap
+    )
+  }
+
+  private def deserializeStageData(binary: StoreTypes.StageData): StageData = {
+    val status = StageStatus.valueOf(binary.getStatus.toString)
+    val submissionTime =
+      getOptional(binary.hasSubmissionTime, () => new Date(binary.getSubmissionTime))
+    val firstTaskLaunchedTime =
+      getOptional(binary.hasFirstTaskLaunchedTime, () => new Date(binary.getFirstTaskLaunchedTime))
+    val completionTime =
+      getOptional(binary.hasCompletionTime, () => new Date(binary.getCompletionTime))
+    val failureReason =
+      getOptional(binary.hasFailureReason, () => weakIntern(binary.getFailureReason))
+    val description =
+      getOptional(binary.hasDescription, () => weakIntern(binary.getDescription))
+    val accumulatorUpdates = AccumulableInfoSerializer.deserializeAccumulableInfos(
+      binary.getAccumulatorUpdatesList)
+    val tasks = MapUtils.isEmpty(binary.getTasksMap) match {
+      case true => None
+      case _ => Some(binary.getTasksMap.asScala.map(
+        entry => (entry._1.toLong, deserializeTaskData(entry._2))).toMap)
+    }
+    val executorSummary = MapUtils.isEmpty(binary.getExecutorSummaryMap) match {
+      case true => None
+      case _ => Some(binary.getExecutorSummaryMap.asScala.mapValues(
+          ExecutorStageSummarySerializer.deserializeExecutorStageSummary(_)).toMap
+      )
+    }
+    val speculationSummary =
+      getOptional(binary.hasSpeculationSummary,
+        () => deserializeSpeculationStageSummary(binary.getSpeculationSummary))
+    val peakExecutorMetrics =
+      getOptional(binary.hasPeakExecutorMetrics,
+        () => ExecutorMetricsSerializer.deserialize(binary.getPeakExecutorMetrics))
+    val taskMetricsDistributions =
+      getOptional(binary.hasTaskMetricsDistributions,
+        () => deserializeTaskMetricDistributions(binary.getTaskMetricsDistributions))
+    val executorMetricsDistributions =
+      getOptional(binary.hasExecutorMetricsDistributions,
+        () => deserializeExecutorMetricsDistributions(binary.getExecutorMetricsDistributions))
+    new StageData(
+      status = status,
+      stageId = binary.getStageId.toInt,
+      attemptId = binary.getAttemptId,
+      numTasks = binary.getNumTasks,
+      numActiveTasks = binary.getNumActiveTasks,
+      numCompleteTasks = binary.getNumCompleteTasks,
+      numFailedTasks = binary.getNumFailedTasks,
+      numKilledTasks = binary.getNumKilledTasks,
+      numCompletedIndices = binary.getNumCompletedIndices,
+      submissionTime = submissionTime,
+      firstTaskLaunchedTime = firstTaskLaunchedTime,
+      completionTime = completionTime,
+      failureReason = failureReason,
+      executorDeserializeTime = binary.getExecutorDeserializeTime,
+      executorDeserializeCpuTime = binary.getExecutorDeserializeCpuTime,
+      executorRunTime = binary.getExecutorRunTime,
+      executorCpuTime = binary.getExecutorCpuTime,
+      resultSize = binary.getResultSize,
+      jvmGcTime = binary.getJvmGcTime,
+      resultSerializationTime = binary.getResultSerializationTime,
+      memoryBytesSpilled = binary.getMemoryBytesSpilled,
+      diskBytesSpilled = binary.getDiskBytesSpilled,
+      peakExecutionMemory = binary.getPeakExecutionMemory,
+      inputBytes = binary.getInputBytes,
+      inputRecords = binary.getInputRecords,
+      outputBytes = binary.getOutputBytes,
+      outputRecords = binary.getOutputRecords,
+      shuffleRemoteBlocksFetched = binary.getShuffleRemoteBlocksFetched,
+      shuffleLocalBlocksFetched = binary.getShuffleLocalBlocksFetched,
+      shuffleFetchWaitTime = binary.getShuffleFetchWaitTime,
+      shuffleRemoteBytesRead = binary.getShuffleRemoteBytesRead,
+      shuffleRemoteBytesReadToDisk = binary.getShuffleRemoteBytesReadToDisk,
+      shuffleLocalBytesRead = binary.getShuffleLocalBytesRead,
+      shuffleReadBytes = binary.getShuffleReadBytes,
+      shuffleReadRecords = binary.getShuffleReadRecords,
+      shuffleWriteBytes = binary.getShuffleWriteBytes,
+      shuffleWriteTime = binary.getShuffleWriteTime,
+      shuffleWriteRecords = binary.getShuffleWriteRecords,
+      name = weakIntern(binary.getName),
+      description = description,
+      details = weakIntern(binary.getDetails),
+      schedulingPool = weakIntern(binary.getSchedulingPool),
+      rddIds = binary.getRddIdsList.asScala.map(_.toInt).toSeq,
+      accumulatorUpdates = accumulatorUpdates.toSeq,
+      tasks = tasks,
+      executorSummary = executorSummary,
+      speculationSummary = speculationSummary,
+      killedTasksSummary = binary.getKilledTasksSummaryMap.asScala.mapValues(_.toInt).toMap,
+      resourceProfileId = binary.getResourceProfileId,
+      peakExecutorMetrics = peakExecutorMetrics,
+      taskMetricsDistributions = taskMetricsDistributions,
+      executorMetricsDistributions = executorMetricsDistributions
+    )
+  }
+
+  private def deserializeSpeculationStageSummary(
+      binary: StoreTypes.SpeculationStageSummary): SpeculationStageSummary = {
+    new SpeculationStageSummary(
+      binary.getNumTasks,
+      binary.getNumActiveTasks,
+      binary.getNumCompletedTasks,
+      binary.getNumFailedTasks,
+      binary.getNumKilledTasks
+    )
+  }
+
+  private def deserializeTaskMetricDistributions(
+      binary: StoreTypes.TaskMetricDistributions): TaskMetricDistributions = {
+    new TaskMetricDistributions(
+      quantiles = binary.getQuantilesList.asScala.map(_.toDouble).toIndexedSeq,
+      duration = binary.getDurationList.asScala.map(_.toDouble).toIndexedSeq,
+      executorDeserializeTime =
+        binary.getExecutorDeserializeTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      executorDeserializeCpuTime =
+        binary.getExecutorDeserializeCpuTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      executorRunTime = binary.getExecutorRunTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      executorCpuTime = binary.getExecutorCpuTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      resultSize = binary.getResultSizeList.asScala.map(_.toDouble).toIndexedSeq,
+      jvmGcTime = binary.getJvmGcTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      resultSerializationTime =
+        binary.getResultSerializationTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      gettingResultTime = binary.getGettingResultTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      schedulerDelay = binary.getSchedulerDelayList.asScala.map(_.toDouble).toIndexedSeq,
+      peakExecutionMemory = binary.getPeakExecutionMemoryList.asScala.map(_.toDouble).toIndexedSeq,
+      memoryBytesSpilled = binary.getMemoryBytesSpilledList.asScala.map(_.toDouble).toIndexedSeq,
+      diskBytesSpilled = binary.getDiskBytesSpilledList.asScala.map(_.toDouble).toIndexedSeq,
+      inputMetrics = deserializeInputMetricDistributions(binary.getInputMetrics),
+      outputMetrics = deserializeOutputMetricDistributions(binary.getOutputMetrics),
+      shuffleReadMetrics = deserializeShuffleReadMetricDistributions(binary.getShuffleReadMetrics),
+      shuffleWriteMetrics =
+        deserializeShuffleWriteMetricDistributions(binary.getShuffleWriteMetrics)
+    )
+  }
+
+  private def deserializeInputMetricDistributions(
+      binary: StoreTypes.InputMetricDistributions): InputMetricDistributions = {
+    new InputMetricDistributions(
+      bytesRead = binary.getBytesReadList.asScala.map(_.toDouble).toIndexedSeq,
+      recordsRead = binary.getRecordsReadList.asScala.map(_.toDouble).toIndexedSeq
+    )
+  }
+
+  private def deserializeOutputMetricDistributions(
+      binary: StoreTypes.OutputMetricDistributions): OutputMetricDistributions = {
+    new OutputMetricDistributions(
+      bytesWritten = binary.getBytesWrittenList.asScala.map(_.toDouble).toIndexedSeq,
+      recordsWritten = binary.getRecordsWrittenList.asScala.map(_.toDouble).toIndexedSeq
+    )
+  }
+
+  private def deserializeShuffleReadMetricDistributions(
+      binary: StoreTypes.ShuffleReadMetricDistributions): ShuffleReadMetricDistributions = {
+    new ShuffleReadMetricDistributions(
+      readBytes = binary.getReadBytesList.asScala.map(_.toDouble).toIndexedSeq,
+      readRecords = binary.getReadRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      remoteBlocksFetched = binary.getRemoteBlocksFetchedList.asScala.map(_.toDouble).toIndexedSeq,
+      localBlocksFetched = binary.getLocalBlocksFetchedList.asScala.map(_.toDouble).toIndexedSeq,
+      fetchWaitTime = binary.getFetchWaitTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      remoteBytesRead = binary.getRemoteBytesReadList.asScala.map(_.toDouble).toIndexedSeq,
+      remoteBytesReadToDisk =
+        binary.getRemoteBytesReadToDiskList.asScala.map(_.toDouble).toIndexedSeq,
+      totalBlocksFetched = binary.getTotalBlocksFetchedList.asScala.map(_.toDouble).toIndexedSeq
+    )
+  }
+
+  private def deserializeShuffleWriteMetricDistributions(
+      binary: StoreTypes.ShuffleWriteMetricDistributions): ShuffleWriteMetricDistributions = {
+    new ShuffleWriteMetricDistributions(
+      writeBytes = binary.getWriteBytesList.asScala.map(_.toDouble).toIndexedSeq,
+      writeRecords = binary.getWriteRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      writeTime = binary.getWriteTimeList.asScala.map(_.toDouble).toIndexedSeq
+    )
+  }
+
+  private def deserializeExecutorMetricsDistributions(
+      binary: StoreTypes.ExecutorMetricsDistributions): ExecutorMetricsDistributions = {
+    new ExecutorMetricsDistributions(
+      quantiles = binary.getQuantilesList.asScala.map(_.toDouble).toIndexedSeq,
+      taskTime = binary.getTaskTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      failedTasks = binary.getFailedTasksList.asScala.map(_.toDouble).toIndexedSeq,
+      succeededTasks = binary.getSucceededTasksList.asScala.map(_.toDouble).toIndexedSeq,
+      killedTasks = binary.getKilledTasksList.asScala.map(_.toDouble).toIndexedSeq,
+      inputBytes = binary.getInputBytesList.asScala.map(_.toDouble).toIndexedSeq,
+      inputRecords = binary.getInputRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      outputBytes = binary.getOutputBytesList.asScala.map(_.toDouble).toIndexedSeq,
+      outputRecords = binary.getOutputRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      shuffleRead = binary.getShuffleReadList.asScala.map(_.toDouble).toIndexedSeq,
+      shuffleReadRecords = binary.getShuffleReadRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      shuffleWrite = binary.getShuffleWriteList.asScala.map(_.toDouble).toIndexedSeq,
+      shuffleWriteRecords = binary.getShuffleWriteRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      memoryBytesSpilled = binary.getMemoryBytesSpilledList.asScala.map(_.toDouble).toIndexedSeq,
+      diskBytesSpilled = binary.getDiskBytesSpilledList.asScala.map(_.toDouble).toIndexedSeq,
+      peakMemoryMetrics = deserializeExecutorPeakMetricsDistributions(binary.getPeakMemoryMetrics)
+    )
+  }
+
+  private def deserializeExecutorPeakMetricsDistributions(
+      binary: StoreTypes.ExecutorPeakMetricsDistributions): ExecutorPeakMetricsDistributions = {
+    new ExecutorPeakMetricsDistributions(
+      quantiles = binary.getQuantilesList.asScala.map(_.toDouble).toIndexedSeq,
+      executorMetrics = binary.getExecutorMetricsList.asScala.map(
+        ExecutorMetricsSerializer.deserialize(_)).toIndexedSeq
+    )
+  }
+
+  private def deserializeTaskData(binary: StoreTypes.TaskData): TaskData = {
+    val resultFetchStart = getOptional(binary.hasResultFetchStart,
+      () => new Date(binary.getResultFetchStart))
+    val duration = getOptional(binary.hasDuration, () => binary.getDuration)
+    val accumulatorUpdates = AccumulableInfoSerializer.deserializeAccumulableInfos(
+      binary.getAccumulatorUpdatesList)
+    val taskMetrics = getOptional(binary.hasTaskMetrics,
+      () => deserializeTaskMetrics(binary.getTaskMetrics))
+    new TaskData(
+      taskId = binary.getTaskId,
+      index = binary.getIndex,
+      attempt = binary.getAttempt,
+      partitionId = binary.getPartitionId,
+      launchTime = new Date(binary.getLaunchTime),
+      resultFetchStart = resultFetchStart,
+      duration = duration,
+      executorId = weakIntern(binary.getExecutorId),

Review Comment:
   When should we use `weakIntern`? Not all serializers use it, will this affect performance?
   
   For example, when `new AccumulableInfo` in `AccumulableInfoSerializer`, we didn't use `weakIntern`
   
   
   
   
   
   



##########
core/src/main/scala/org/apache/spark/status/protobuf/StageDataWrapperSerializer.scala:
##########
@@ -0,0 +1,627 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.Date
+
+import collection.JavaConverters._
+import org.apache.commons.collections4.MapUtils
+
+import org.apache.spark.status.StageDataWrapper
+import org.apache.spark.status.api.v1.{ExecutorMetricsDistributions, ExecutorPeakMetricsDistributions, InputMetricDistributions, InputMetrics, OutputMetricDistributions, OutputMetrics, ShuffleReadMetricDistributions, ShuffleReadMetrics, ShuffleWriteMetricDistributions, ShuffleWriteMetrics, SpeculationStageSummary, StageData, StageStatus, TaskData, TaskMetricDistributions, TaskMetrics}
+import org.apache.spark.status.protobuf.Utils.getOptional
+import org.apache.spark.util.Utils.weakIntern
+
+class StageDataWrapperSerializer extends ProtobufSerDe {
+
+  override val supportClass: Class[_] = classOf[StageDataWrapper]
+
+  override def serialize(input: Any): Array[Byte] =
+    serialize(input.asInstanceOf[StageDataWrapper])
+
+  private def serialize(s: StageDataWrapper): Array[Byte] = {
+    val builder = StoreTypes.StageDataWrapper.newBuilder()
+    builder.setInfo(serializeStageData(s.info))
+    s.jobIds.foreach(id => builder.addJobIds(id.toLong))
+    s.locality.foreach { entry =>
+      builder.putLocality(entry._1, entry._2)
+    }
+    builder.build().toByteArray
+  }
+
+  private def serializeStageData(stageData: StageData): StoreTypes.StageData = {
+    val stageDataBuilder = StoreTypes.StageData.newBuilder()
+    stageDataBuilder
+      .setStatus(serializeStageStatus(stageData.status))
+      .setStageId(stageData.stageId.toLong)
+      .setAttemptId(stageData.attemptId)
+      .setNumTasks(stageData.numTasks)
+      .setNumActiveTasks(stageData.numActiveTasks)
+      .setNumCompleteTasks(stageData.numCompleteTasks)
+      .setNumFailedTasks(stageData.numFailedTasks)
+      .setNumKilledTasks(stageData.numKilledTasks)
+      .setNumCompletedIndices(stageData.numCompletedIndices)
+      .setExecutorDeserializeTime(stageData.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(stageData.executorDeserializeCpuTime)
+      .setExecutorRunTime(stageData.executorRunTime)
+      .setExecutorCpuTime(stageData.executorCpuTime)
+      .setResultSize(stageData.resultSize)
+      .setJvmGcTime(stageData.jvmGcTime)
+      .setResultSerializationTime(stageData.resultSerializationTime)
+      .setMemoryBytesSpilled(stageData.memoryBytesSpilled)
+      .setDiskBytesSpilled(stageData.diskBytesSpilled)
+      .setPeakExecutionMemory(stageData.peakExecutionMemory)
+      .setInputBytes(stageData.inputBytes)
+      .setInputRecords(stageData.inputRecords)
+      .setOutputBytes(stageData.outputBytes)
+      .setOutputRecords(stageData.outputRecords)
+      .setShuffleRemoteBlocksFetched(stageData.shuffleRemoteBlocksFetched)
+      .setShuffleLocalBlocksFetched(stageData.shuffleLocalBlocksFetched)
+      .setShuffleFetchWaitTime(stageData.shuffleFetchWaitTime)
+      .setShuffleRemoteBytesRead(stageData.shuffleRemoteBytesRead)
+      .setShuffleRemoteBytesReadToDisk(stageData.shuffleRemoteBytesReadToDisk)
+      .setShuffleLocalBytesRead(stageData.shuffleLocalBytesRead)
+      .setShuffleReadBytes(stageData.shuffleReadBytes)
+      .setShuffleReadRecords(stageData.shuffleReadRecords)
+      .setShuffleWriteBytes(stageData.shuffleWriteBytes)
+      .setShuffleWriteTime(stageData.shuffleWriteTime)
+      .setShuffleWriteRecords(stageData.shuffleWriteRecords)
+      .setName(stageData.name)
+      .setDetails(stageData.details)
+      .setSchedulingPool(stageData.schedulingPool)
+      .setResourceProfileId(stageData.resourceProfileId)
+    stageData.submissionTime.foreach { d =>
+      stageDataBuilder.setSubmissionTime(d.getTime)
+    }
+    stageData.firstTaskLaunchedTime.foreach { d =>
+      stageDataBuilder.setFirstTaskLaunchedTime(d.getTime)
+    }
+    stageData.completionTime.foreach { d =>
+      stageDataBuilder.setCompletionTime(d.getTime)
+    }
+    stageData.failureReason.foreach { fr =>
+      stageDataBuilder.setFailureReason(fr)
+    }
+    stageData.description.foreach { d =>
+      stageDataBuilder.setDescription(d)
+    }
+    stageData.rddIds.foreach(id => stageDataBuilder.addRddIds(id.toLong))
+    stageData.accumulatorUpdates.foreach { update =>
+      stageDataBuilder.addAccumulatorUpdates(
+        AccumulableInfoSerializer.serializeAccumulableInfo(update))
+    }
+    stageData.tasks.foreach { t =>
+      t.foreach { entry =>
+        stageDataBuilder.putTasks(entry._1, serializeTaskData(entry._2))
+      }
+    }
+    stageData.executorSummary.foreach { es =>
+      es.foreach { entry =>
+        stageDataBuilder.putExecutorSummary(entry._1,
+          ExecutorStageSummarySerializer.serializeExecutorStageSummary(entry._2))
+      }
+    }
+    stageData.speculationSummary.foreach { ss =>
+      stageDataBuilder.setSpeculationSummary(serializeSpeculationStageSummary(ss))
+    }
+    stageData.killedTasksSummary.foreach { entry =>
+      stageDataBuilder.putKilledTasksSummary(entry._1, entry._2)
+    }
+    stageData.peakExecutorMetrics.foreach { pem =>
+      stageDataBuilder.setPeakExecutorMetrics(ExecutorMetricsSerializer.serialize(pem))
+    }
+    stageData.taskMetricsDistributions.foreach { tmd =>
+      stageDataBuilder.setTaskMetricsDistributions(serializeTaskMetricDistributions(tmd))
+    }
+    stageData.executorMetricsDistributions.foreach { emd =>
+      stageDataBuilder.setExecutorMetricsDistributions(serializeExecutorMetricsDistributions(emd))
+    }
+    stageDataBuilder.build()
+  }
+
+  private def serializeStageStatus(s: StageStatus): StoreTypes.StageData.StageStatus = {
+    StoreTypes.StageData.StageStatus.valueOf(s.toString)
+  }
+
+  private def serializeTaskData(t: TaskData): StoreTypes.TaskData = {
+    val taskDataBuilder = StoreTypes.TaskData.newBuilder()
+    taskDataBuilder
+      .setTaskId(t.taskId)
+      .setIndex(t.index)
+      .setAttempt(t.attempt)
+      .setPartitionId(t.partitionId)
+      .setLaunchTime(t.launchTime.getTime)
+      .setExecutorId(t.executorId)
+      .setHost(t.host)
+      .setStatus(t.status)
+      .setTaskLocality(t.taskLocality)
+      .setSpeculative(t.speculative)
+      .setSchedulerDelay(t.schedulerDelay)
+      .setGettingResultTime(t.gettingResultTime)
+    t.resultFetchStart.foreach { rfs =>
+      taskDataBuilder.setResultFetchStart(rfs.getTime)
+    }
+    t.duration.foreach { d =>
+      taskDataBuilder.setDuration(d)
+    }
+    t.accumulatorUpdates.foreach { update =>
+      taskDataBuilder.addAccumulatorUpdates(
+        AccumulableInfoSerializer.serializeAccumulableInfo(update))
+    }
+    t.errorMessage.foreach { em =>
+      taskDataBuilder.setErrorMessage(em)
+    }
+    t.taskMetrics.foreach { tm =>
+      taskDataBuilder.setTaskMetrics(serializeTaskMetrics(tm))
+    }
+    t.executorLogs.foreach { entry =>
+      taskDataBuilder.putExecutorLogs(entry._1, entry._2)
+    }
+    taskDataBuilder.build()
+  }
+
+  private def serializeTaskMetrics(tm: TaskMetrics): StoreTypes.TaskMetrics = {
+    val taskMetricsBuilder = StoreTypes.TaskMetrics.newBuilder()
+    taskMetricsBuilder
+      .setExecutorDeserializeTime(tm.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(tm.executorDeserializeCpuTime)
+      .setExecutorRunTime(tm.executorRunTime)
+      .setExecutorCpuTime(tm.executorCpuTime)
+      .setResultSize(tm.resultSize)
+      .setJvmGcTime(tm.jvmGcTime)
+      .setResultSerializationTime(tm.resultSerializationTime)
+      .setMemoryBytesSpilled(tm.memoryBytesSpilled)
+      .setDiskBytesSpilled(tm.diskBytesSpilled)
+      .setPeakExecutionMemory(tm.peakExecutionMemory)
+      .setInputMetrics(serializeInputMetrics(tm.inputMetrics))
+      .setOutputMetrics(serializeOutputMetrics(tm.outputMetrics))
+      .setShuffleReadMetrics(serializeShuffleReadMetrics(tm.shuffleReadMetrics))
+      .setShuffleWriteMetrics(serializeShuffleWriteMetrics(tm.shuffleWriteMetrics))
+    taskMetricsBuilder.build()
+  }
+
+  private def serializeInputMetrics(im: InputMetrics): StoreTypes.InputMetrics = {
+    StoreTypes.InputMetrics.newBuilder()
+      .setBytesRead(im.bytesRead)
+      .setRecordsRead(im.recordsRead)
+      .build()
+  }
+
+  private def serializeOutputMetrics(om: OutputMetrics): StoreTypes.OutputMetrics = {
+    StoreTypes.OutputMetrics.newBuilder()
+      .setBytesWritten(om.bytesWritten)
+      .setRecordsWritten(om.recordsWritten)
+      .build()
+  }
+
+  private def serializeShuffleReadMetrics(
+      srm: ShuffleReadMetrics): StoreTypes.ShuffleReadMetrics = {
+    StoreTypes.ShuffleReadMetrics.newBuilder()
+      .setRemoteBlocksFetched(srm.remoteBlocksFetched)
+      .setLocalBlocksFetched(srm.localBlocksFetched)
+      .setFetchWaitTime(srm.fetchWaitTime)
+      .setRemoteBytesRead(srm.remoteBytesRead)
+      .setRemoteBytesReadToDisk(srm.remoteBytesReadToDisk)
+      .setLocalBytesRead(srm.localBytesRead)
+      .setRecordsRead(srm.recordsRead)
+      .build()
+  }
+
+  private def serializeShuffleWriteMetrics(
+      swm: ShuffleWriteMetrics): StoreTypes.ShuffleWriteMetrics = {
+    StoreTypes.ShuffleWriteMetrics.newBuilder()
+      .setBytesWritten(swm.bytesWritten)
+      .setWriteTime(swm.writeTime)
+      .setRecordsWritten(swm.recordsWritten)
+      .build()
+  }
+
+  private def serializeSpeculationStageSummary(
+      sss: SpeculationStageSummary): StoreTypes.SpeculationStageSummary = {
+    StoreTypes.SpeculationStageSummary.newBuilder()
+      .setNumTasks(sss.numTasks)
+      .setNumActiveTasks(sss.numActiveTasks)
+      .setNumCompletedTasks(sss.numCompletedTasks)
+      .setNumFailedTasks(sss.numFailedTasks)
+      .setNumKilledTasks(sss.numKilledTasks)
+      .build()
+  }
+
+  private def serializeTaskMetricDistributions(
+      tmd: TaskMetricDistributions): StoreTypes.TaskMetricDistributions = {
+    val builder = StoreTypes.TaskMetricDistributions.newBuilder()
+    tmd.quantiles.foreach(q => builder.addQuantiles(q))
+    tmd.duration.foreach(d => builder.addDuration(d))
+    tmd.executorDeserializeTime.foreach(edt => builder.addExecutorDeserializeTime(edt))
+    tmd.executorDeserializeCpuTime.foreach(edct => builder.addExecutorDeserializeCpuTime(edct))
+    tmd.executorRunTime.foreach(ert => builder.addExecutorRunTime(ert))
+    tmd.executorCpuTime.foreach(ect => builder.addExecutorCpuTime(ect))
+    tmd.resultSize.foreach(rs => builder.addResultSize(rs))
+    tmd.jvmGcTime.foreach(jgt => builder.addJvmGcTime(jgt))
+    tmd.resultSerializationTime.foreach(rst => builder.addResultSerializationTime(rst))
+    tmd.gettingResultTime.foreach(grt => builder.addGettingResultTime(grt))
+    tmd.schedulerDelay.foreach(sd => builder.addSchedulerDelay(sd))
+    tmd.peakExecutionMemory.foreach(pem => builder.addPeakExecutionMemory(pem))
+    tmd.memoryBytesSpilled.foreach(mbs => builder.addMemoryBytesSpilled(mbs))
+    tmd.diskBytesSpilled.foreach(dbs => builder.addDiskBytesSpilled(dbs))
+    builder
+      .setInputMetrics(serializeInputMetricDistributions(tmd.inputMetrics))
+      .setOutputMetrics(serializeOutputMetricDistributions(tmd.outputMetrics))
+      .setShuffleReadMetrics(serializeShuffleReadMetricDistributions(tmd.shuffleReadMetrics))
+      .setShuffleWriteMetrics(serializeShuffleWriteMetricDistributions(tmd.shuffleWriteMetrics))
+      .build()
+  }
+
+  private def serializeInputMetricDistributions(
+      imd: InputMetricDistributions): StoreTypes.InputMetricDistributions = {
+    val builder = StoreTypes.InputMetricDistributions.newBuilder()
+    imd.bytesRead.foreach(br => builder.addBytesRead(br))
+    imd.recordsRead.foreach(rr => builder.addRecordsRead(rr))
+    builder.build()
+  }
+
+  private def serializeOutputMetricDistributions(
+      omd: OutputMetricDistributions): StoreTypes.OutputMetricDistributions = {
+    val builder = StoreTypes.OutputMetricDistributions.newBuilder()
+    omd.bytesWritten.foreach(bw => builder.addBytesWritten(bw))
+    omd.recordsWritten.foreach(rw => builder.addRecordsWritten(rw))
+    builder.build()
+  }
+
+  private def serializeShuffleReadMetricDistributions(
+      srmd: ShuffleReadMetricDistributions): StoreTypes.ShuffleReadMetricDistributions = {
+    val builder = StoreTypes.ShuffleReadMetricDistributions.newBuilder()
+    srmd.readBytes.foreach(rb => builder.addReadBytes(rb))
+    srmd.readRecords.foreach(rr => builder.addReadRecords(rr))
+    srmd.remoteBlocksFetched.foreach(rbf => builder.addRemoteBlocksFetched(rbf))
+    srmd.localBlocksFetched.foreach(lbf => builder.addLocalBlocksFetched(lbf))
+    srmd.fetchWaitTime.foreach(fwt => builder.addFetchWaitTime(fwt))
+    srmd.remoteBytesRead.foreach(rbr => builder.addRemoteBytesRead(rbr))
+    srmd.remoteBytesReadToDisk.foreach(rbrtd => builder.addRemoteBytesReadToDisk(rbrtd))
+    srmd.totalBlocksFetched.foreach(tbf => builder.addTotalBlocksFetched(tbf))
+    builder.build()
+  }
+
+  private def serializeShuffleWriteMetricDistributions(
+      swmd: ShuffleWriteMetricDistributions): StoreTypes.ShuffleWriteMetricDistributions = {
+    val builder = StoreTypes.ShuffleWriteMetricDistributions.newBuilder()
+    swmd.writeBytes.foreach(wb => builder.addWriteBytes(wb))
+    swmd.writeRecords.foreach(wr => builder.addWriteRecords(wr))
+    swmd.writeTime.foreach(wt => builder.addWriteTime(wt))
+    builder.build()
+  }
+
+  private def serializeExecutorMetricsDistributions(
+      emd: ExecutorMetricsDistributions): StoreTypes.ExecutorMetricsDistributions = {
+    val builder = StoreTypes.ExecutorMetricsDistributions.newBuilder()
+    emd.quantiles.foreach(q => builder.addQuantiles(q))
+    emd.taskTime.foreach(tt => builder.addTaskTime(tt))
+    emd.failedTasks.foreach(ft => builder.addFailedTasks(ft))
+    emd.succeededTasks.foreach(st => builder.addSucceededTasks(st))
+    emd.killedTasks.foreach(kt => builder.addKilledTasks(kt))
+    emd.inputBytes.foreach(ib => builder.addInputBytes(ib))
+    emd.inputRecords.foreach(ir => builder.addInputRecords(ir))
+    emd.outputBytes.foreach(ob => builder.addOutputBytes(ob))
+    emd.outputRecords.foreach(or => builder.addOutputRecords(or))
+    emd.shuffleRead.foreach(sr => builder.addShuffleRead(sr))
+    emd.shuffleReadRecords.foreach(srr => builder.addShuffleReadRecords(srr))
+    emd.shuffleWrite.foreach(sw => builder.addShuffleWrite(sw))
+    emd.shuffleWriteRecords.foreach(swr => builder.addShuffleWriteRecords(swr))
+    emd.memoryBytesSpilled.foreach(mbs => builder.addMemoryBytesSpilled(mbs))
+    emd.diskBytesSpilled.foreach(dbs => builder.addDiskBytesSpilled(dbs))
+    builder.setPeakMemoryMetrics(serializeExecutorPeakMetricsDistributions(emd.peakMemoryMetrics))
+    builder.build()
+  }
+
+  private def serializeExecutorPeakMetricsDistributions(
+      epmd: ExecutorPeakMetricsDistributions): StoreTypes.ExecutorPeakMetricsDistributions = {
+    val builder = StoreTypes.ExecutorPeakMetricsDistributions.newBuilder()
+    epmd.quantiles.foreach(q => builder.addQuantiles(q))
+    epmd.executorMetrics.foreach(em => builder.addExecutorMetrics(
+      ExecutorMetricsSerializer.serialize(em)))
+    builder.build()
+  }
+
+  override def deserialize(bytes: Array[Byte]): StageDataWrapper = {
+    val binary = StoreTypes.StageDataWrapper.parseFrom(bytes)
+    val info = deserializeStageData(binary.getInfo)
+    new StageDataWrapper(
+      info = info,
+      jobIds = binary.getJobIdsList.asScala.map(_.toInt).toSet,
+      locality = binary.getLocalityMap.asScala.mapValues(_.toLong).toMap
+    )
+  }
+
+  private def deserializeStageData(binary: StoreTypes.StageData): StageData = {
+    val status = StageStatus.valueOf(binary.getStatus.toString)
+    val submissionTime =
+      getOptional(binary.hasSubmissionTime, () => new Date(binary.getSubmissionTime))
+    val firstTaskLaunchedTime =
+      getOptional(binary.hasFirstTaskLaunchedTime, () => new Date(binary.getFirstTaskLaunchedTime))
+    val completionTime =
+      getOptional(binary.hasCompletionTime, () => new Date(binary.getCompletionTime))
+    val failureReason =
+      getOptional(binary.hasFailureReason, () => weakIntern(binary.getFailureReason))
+    val description =
+      getOptional(binary.hasDescription, () => weakIntern(binary.getDescription))
+    val accumulatorUpdates = AccumulableInfoSerializer.deserializeAccumulableInfos(
+      binary.getAccumulatorUpdatesList)
+    val tasks = MapUtils.isEmpty(binary.getTasksMap) match {
+      case true => None
+      case _ => Some(binary.getTasksMap.asScala.map(
+        entry => (entry._1.toLong, deserializeTaskData(entry._2))).toMap)
+    }
+    val executorSummary = MapUtils.isEmpty(binary.getExecutorSummaryMap) match {
+      case true => None
+      case _ => Some(binary.getExecutorSummaryMap.asScala.mapValues(
+          ExecutorStageSummarySerializer.deserializeExecutorStageSummary(_)).toMap
+      )
+    }
+    val speculationSummary =
+      getOptional(binary.hasSpeculationSummary,
+        () => deserializeSpeculationStageSummary(binary.getSpeculationSummary))
+    val peakExecutorMetrics =
+      getOptional(binary.hasPeakExecutorMetrics,
+        () => ExecutorMetricsSerializer.deserialize(binary.getPeakExecutorMetrics))
+    val taskMetricsDistributions =
+      getOptional(binary.hasTaskMetricsDistributions,
+        () => deserializeTaskMetricDistributions(binary.getTaskMetricsDistributions))
+    val executorMetricsDistributions =
+      getOptional(binary.hasExecutorMetricsDistributions,
+        () => deserializeExecutorMetricsDistributions(binary.getExecutorMetricsDistributions))
+    new StageData(
+      status = status,
+      stageId = binary.getStageId.toInt,
+      attemptId = binary.getAttemptId,
+      numTasks = binary.getNumTasks,
+      numActiveTasks = binary.getNumActiveTasks,
+      numCompleteTasks = binary.getNumCompleteTasks,
+      numFailedTasks = binary.getNumFailedTasks,
+      numKilledTasks = binary.getNumKilledTasks,
+      numCompletedIndices = binary.getNumCompletedIndices,
+      submissionTime = submissionTime,
+      firstTaskLaunchedTime = firstTaskLaunchedTime,
+      completionTime = completionTime,
+      failureReason = failureReason,
+      executorDeserializeTime = binary.getExecutorDeserializeTime,
+      executorDeserializeCpuTime = binary.getExecutorDeserializeCpuTime,
+      executorRunTime = binary.getExecutorRunTime,
+      executorCpuTime = binary.getExecutorCpuTime,
+      resultSize = binary.getResultSize,
+      jvmGcTime = binary.getJvmGcTime,
+      resultSerializationTime = binary.getResultSerializationTime,
+      memoryBytesSpilled = binary.getMemoryBytesSpilled,
+      diskBytesSpilled = binary.getDiskBytesSpilled,
+      peakExecutionMemory = binary.getPeakExecutionMemory,
+      inputBytes = binary.getInputBytes,
+      inputRecords = binary.getInputRecords,
+      outputBytes = binary.getOutputBytes,
+      outputRecords = binary.getOutputRecords,
+      shuffleRemoteBlocksFetched = binary.getShuffleRemoteBlocksFetched,
+      shuffleLocalBlocksFetched = binary.getShuffleLocalBlocksFetched,
+      shuffleFetchWaitTime = binary.getShuffleFetchWaitTime,
+      shuffleRemoteBytesRead = binary.getShuffleRemoteBytesRead,
+      shuffleRemoteBytesReadToDisk = binary.getShuffleRemoteBytesReadToDisk,
+      shuffleLocalBytesRead = binary.getShuffleLocalBytesRead,
+      shuffleReadBytes = binary.getShuffleReadBytes,
+      shuffleReadRecords = binary.getShuffleReadRecords,
+      shuffleWriteBytes = binary.getShuffleWriteBytes,
+      shuffleWriteTime = binary.getShuffleWriteTime,
+      shuffleWriteRecords = binary.getShuffleWriteRecords,
+      name = weakIntern(binary.getName),
+      description = description,
+      details = weakIntern(binary.getDetails),
+      schedulingPool = weakIntern(binary.getSchedulingPool),
+      rddIds = binary.getRddIdsList.asScala.map(_.toInt).toSeq,
+      accumulatorUpdates = accumulatorUpdates.toSeq,
+      tasks = tasks,
+      executorSummary = executorSummary,
+      speculationSummary = speculationSummary,
+      killedTasksSummary = binary.getKilledTasksSummaryMap.asScala.mapValues(_.toInt).toMap,
+      resourceProfileId = binary.getResourceProfileId,
+      peakExecutorMetrics = peakExecutorMetrics,
+      taskMetricsDistributions = taskMetricsDistributions,
+      executorMetricsDistributions = executorMetricsDistributions
+    )
+  }
+
+  private def deserializeSpeculationStageSummary(
+      binary: StoreTypes.SpeculationStageSummary): SpeculationStageSummary = {
+    new SpeculationStageSummary(
+      binary.getNumTasks,
+      binary.getNumActiveTasks,
+      binary.getNumCompletedTasks,
+      binary.getNumFailedTasks,
+      binary.getNumKilledTasks
+    )
+  }
+
+  private def deserializeTaskMetricDistributions(
+      binary: StoreTypes.TaskMetricDistributions): TaskMetricDistributions = {
+    new TaskMetricDistributions(
+      quantiles = binary.getQuantilesList.asScala.map(_.toDouble).toIndexedSeq,
+      duration = binary.getDurationList.asScala.map(_.toDouble).toIndexedSeq,
+      executorDeserializeTime =
+        binary.getExecutorDeserializeTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      executorDeserializeCpuTime =
+        binary.getExecutorDeserializeCpuTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      executorRunTime = binary.getExecutorRunTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      executorCpuTime = binary.getExecutorCpuTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      resultSize = binary.getResultSizeList.asScala.map(_.toDouble).toIndexedSeq,
+      jvmGcTime = binary.getJvmGcTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      resultSerializationTime =
+        binary.getResultSerializationTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      gettingResultTime = binary.getGettingResultTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      schedulerDelay = binary.getSchedulerDelayList.asScala.map(_.toDouble).toIndexedSeq,
+      peakExecutionMemory = binary.getPeakExecutionMemoryList.asScala.map(_.toDouble).toIndexedSeq,
+      memoryBytesSpilled = binary.getMemoryBytesSpilledList.asScala.map(_.toDouble).toIndexedSeq,
+      diskBytesSpilled = binary.getDiskBytesSpilledList.asScala.map(_.toDouble).toIndexedSeq,
+      inputMetrics = deserializeInputMetricDistributions(binary.getInputMetrics),
+      outputMetrics = deserializeOutputMetricDistributions(binary.getOutputMetrics),
+      shuffleReadMetrics = deserializeShuffleReadMetricDistributions(binary.getShuffleReadMetrics),
+      shuffleWriteMetrics =
+        deserializeShuffleWriteMetricDistributions(binary.getShuffleWriteMetrics)
+    )
+  }
+
+  private def deserializeInputMetricDistributions(
+      binary: StoreTypes.InputMetricDistributions): InputMetricDistributions = {
+    new InputMetricDistributions(
+      bytesRead = binary.getBytesReadList.asScala.map(_.toDouble).toIndexedSeq,
+      recordsRead = binary.getRecordsReadList.asScala.map(_.toDouble).toIndexedSeq
+    )
+  }
+
+  private def deserializeOutputMetricDistributions(
+      binary: StoreTypes.OutputMetricDistributions): OutputMetricDistributions = {
+    new OutputMetricDistributions(
+      bytesWritten = binary.getBytesWrittenList.asScala.map(_.toDouble).toIndexedSeq,
+      recordsWritten = binary.getRecordsWrittenList.asScala.map(_.toDouble).toIndexedSeq
+    )
+  }
+
+  private def deserializeShuffleReadMetricDistributions(
+      binary: StoreTypes.ShuffleReadMetricDistributions): ShuffleReadMetricDistributions = {
+    new ShuffleReadMetricDistributions(
+      readBytes = binary.getReadBytesList.asScala.map(_.toDouble).toIndexedSeq,
+      readRecords = binary.getReadRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      remoteBlocksFetched = binary.getRemoteBlocksFetchedList.asScala.map(_.toDouble).toIndexedSeq,
+      localBlocksFetched = binary.getLocalBlocksFetchedList.asScala.map(_.toDouble).toIndexedSeq,
+      fetchWaitTime = binary.getFetchWaitTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      remoteBytesRead = binary.getRemoteBytesReadList.asScala.map(_.toDouble).toIndexedSeq,
+      remoteBytesReadToDisk =
+        binary.getRemoteBytesReadToDiskList.asScala.map(_.toDouble).toIndexedSeq,
+      totalBlocksFetched = binary.getTotalBlocksFetchedList.asScala.map(_.toDouble).toIndexedSeq
+    )
+  }
+
+  private def deserializeShuffleWriteMetricDistributions(
+      binary: StoreTypes.ShuffleWriteMetricDistributions): ShuffleWriteMetricDistributions = {
+    new ShuffleWriteMetricDistributions(
+      writeBytes = binary.getWriteBytesList.asScala.map(_.toDouble).toIndexedSeq,
+      writeRecords = binary.getWriteRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      writeTime = binary.getWriteTimeList.asScala.map(_.toDouble).toIndexedSeq
+    )
+  }
+
+  private def deserializeExecutorMetricsDistributions(
+      binary: StoreTypes.ExecutorMetricsDistributions): ExecutorMetricsDistributions = {
+    new ExecutorMetricsDistributions(
+      quantiles = binary.getQuantilesList.asScala.map(_.toDouble).toIndexedSeq,
+      taskTime = binary.getTaskTimeList.asScala.map(_.toDouble).toIndexedSeq,
+      failedTasks = binary.getFailedTasksList.asScala.map(_.toDouble).toIndexedSeq,
+      succeededTasks = binary.getSucceededTasksList.asScala.map(_.toDouble).toIndexedSeq,
+      killedTasks = binary.getKilledTasksList.asScala.map(_.toDouble).toIndexedSeq,
+      inputBytes = binary.getInputBytesList.asScala.map(_.toDouble).toIndexedSeq,
+      inputRecords = binary.getInputRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      outputBytes = binary.getOutputBytesList.asScala.map(_.toDouble).toIndexedSeq,
+      outputRecords = binary.getOutputRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      shuffleRead = binary.getShuffleReadList.asScala.map(_.toDouble).toIndexedSeq,
+      shuffleReadRecords = binary.getShuffleReadRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      shuffleWrite = binary.getShuffleWriteList.asScala.map(_.toDouble).toIndexedSeq,
+      shuffleWriteRecords = binary.getShuffleWriteRecordsList.asScala.map(_.toDouble).toIndexedSeq,
+      memoryBytesSpilled = binary.getMemoryBytesSpilledList.asScala.map(_.toDouble).toIndexedSeq,
+      diskBytesSpilled = binary.getDiskBytesSpilledList.asScala.map(_.toDouble).toIndexedSeq,
+      peakMemoryMetrics = deserializeExecutorPeakMetricsDistributions(binary.getPeakMemoryMetrics)
+    )
+  }
+
+  private def deserializeExecutorPeakMetricsDistributions(
+      binary: StoreTypes.ExecutorPeakMetricsDistributions): ExecutorPeakMetricsDistributions = {
+    new ExecutorPeakMetricsDistributions(
+      quantiles = binary.getQuantilesList.asScala.map(_.toDouble).toIndexedSeq,
+      executorMetrics = binary.getExecutorMetricsList.asScala.map(
+        ExecutorMetricsSerializer.deserialize(_)).toIndexedSeq

Review Comment:
   can convertible to a method value



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
panbingkun commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1058910077


##########
core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto:
##########
@@ -390,3 +390,214 @@ message SQLExecutionUIData {
   repeated int64 stages = 11;
   map<int64, string> metric_values = 12;
 }
+
+message StageDataWrapper {
+  StageData info = 1;
+  repeated int64 job_ids = 2;
+  map<string, int64> locality = 3;
+}
+
+message TaskData {
+  int64 task_id = 1;
+  int32 index = 2;
+  int32 attempt = 3;
+  int32 partition_id = 4;
+  int64 launch_time = 5;
+  optional int64 result_fetch_start = 6;
+  optional int64 duration = 7;
+  string executor_id = 8;
+  string host = 9;
+  string status = 10;
+  string task_locality = 11;
+  bool speculative = 12;
+  repeated AccumulableInfo accumulator_updates = 13;
+  optional string error_message = 14;
+  optional TaskMetrics task_metrics = 15;
+  map<string, string> executor_logs = 16;
+  int64 scheduler_delay = 17;
+  int64 getting_result_time = 18;
+}
+
+message StageData {
+  enum StageStatus {

Review Comment:
   Ok, Let us follow code style(https://developers.google.com/protocol-buffers/docs/style#enums):
   <img width="860" alt="image" src="https://user-images.githubusercontent.com/15246973/209947610-c586a3af-479b-4453-820b-7b853dff5009.png">
   



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gengliangwang commented on pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on PR #39192:
URL: https://github.com/apache/spark/pull/39192#issuecomment-1368100474

   @panbingkun Thanks for the work, merging to master


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on PR #39192:
URL: https://github.com/apache/spark/pull/39192#issuecomment-1364560350

   Can one of the admins verify this patch?


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gengliangwang commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1057955531


##########
core/src/main/scala/org/apache/spark/status/protobuf/StageDataWrapperSerializer.scala:
##########
@@ -0,0 +1,622 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.Date
+
+import collection.JavaConverters._
+import org.apache.commons.collections4.MapUtils
+
+import org.apache.spark.status.StageDataWrapper
+import org.apache.spark.status.api.v1.{ExecutorMetricsDistributions, ExecutorPeakMetricsDistributions, InputMetricDistributions, InputMetrics, OutputMetricDistributions, OutputMetrics, ShuffleReadMetricDistributions, ShuffleReadMetrics, ShuffleWriteMetricDistributions, ShuffleWriteMetrics, SpeculationStageSummary, StageData, StageStatus, TaskData, TaskMetricDistributions, TaskMetrics}
+import org.apache.spark.status.protobuf.Utils.getOptional
+import org.apache.spark.util.Utils.weakIntern
+
+class StageDataWrapperSerializer extends ProtobufSerDe {
+
+  override val supportClass: Class[_] = classOf[StageDataWrapper]
+
+  override def serialize(input: Any): Array[Byte] =
+    serialize(input.asInstanceOf[StageDataWrapper])
+
+  private def serialize(s: StageDataWrapper): Array[Byte] = {
+    val builder = StoreTypes.StageDataWrapper.newBuilder()
+    builder.setInfo(serializeStageData(s.info))
+    s.jobIds.foreach(id => builder.addJobIds(id.toLong))
+    s.locality.foreach { entry =>
+      builder.putLocality(entry._1, entry._2)
+    }
+    builder.build().toByteArray
+  }
+
+  private def serializeStageData(stageData: StageData): StoreTypes.StageData = {
+    val stageDataBuilder = StoreTypes.StageData.newBuilder()
+    stageDataBuilder
+      .setStatus(serializeStageStatus(stageData.status))
+      .setStageId(stageData.stageId.toLong)
+      .setAttemptId(stageData.attemptId)
+      .setNumTasks(stageData.numTasks)
+      .setNumActiveTasks(stageData.numActiveTasks)
+      .setNumCompleteTasks(stageData.numCompleteTasks)
+      .setNumFailedTasks(stageData.numFailedTasks)
+      .setNumKilledTasks(stageData.numKilledTasks)
+      .setNumCompletedIndices(stageData.numCompletedIndices)
+      .setExecutorDeserializeTime(stageData.executorDeserializeTime)
+      .setExecutorDeserializeCpuTime(stageData.executorDeserializeCpuTime)
+      .setExecutorRunTime(stageData.executorRunTime)
+      .setExecutorCpuTime(stageData.executorCpuTime)
+      .setResultSize(stageData.resultSize)
+      .setJvmGcTime(stageData.jvmGcTime)
+      .setResultSerializationTime(stageData.resultSerializationTime)
+      .setMemoryBytesSpilled(stageData.memoryBytesSpilled)
+      .setDiskBytesSpilled(stageData.diskBytesSpilled)
+      .setPeakExecutionMemory(stageData.peakExecutionMemory)
+      .setInputBytes(stageData.inputBytes)
+      .setInputRecords(stageData.inputRecords)
+      .setOutputBytes(stageData.outputBytes)
+      .setOutputRecords(stageData.outputRecords)
+      .setShuffleRemoteBlocksFetched(stageData.shuffleRemoteBlocksFetched)
+      .setShuffleLocalBlocksFetched(stageData.shuffleLocalBlocksFetched)
+      .setShuffleFetchWaitTime(stageData.shuffleFetchWaitTime)
+      .setShuffleRemoteBytesRead(stageData.shuffleRemoteBytesRead)
+      .setShuffleRemoteBytesReadToDisk(stageData.shuffleRemoteBytesReadToDisk)
+      .setShuffleLocalBytesRead(stageData.shuffleLocalBytesRead)
+      .setShuffleReadBytes(stageData.shuffleReadBytes)
+      .setShuffleReadRecords(stageData.shuffleReadRecords)
+      .setShuffleWriteBytes(stageData.shuffleWriteBytes)
+      .setShuffleWriteTime(stageData.shuffleWriteTime)
+      .setShuffleWriteRecords(stageData.shuffleWriteRecords)
+      .setName(stageData.name)
+      .setDetails(stageData.details)
+      .setSchedulingPool(stageData.schedulingPool)
+      .setResourceProfileId(stageData.resourceProfileId)
+    stageData.submissionTime.foreach { d =>
+      stageDataBuilder.setSubmissionTime(d.getTime)
+    }
+    stageData.firstTaskLaunchedTime.foreach { d =>
+      stageDataBuilder.setFirstTaskLaunchedTime(d.getTime)
+    }
+    stageData.completionTime.foreach { d =>
+      stageDataBuilder.setCompletionTime(d.getTime)
+    }
+    stageData.failureReason.foreach { fr =>
+      stageDataBuilder.setFailureReason(fr)
+    }
+    stageData.description.foreach { d =>
+      stageDataBuilder.setDescription(d)
+    }
+    stageData.rddIds.foreach(id => stageDataBuilder.addRddIds(id.toLong))
+    stageData.accumulatorUpdates.foreach { update =>
+      stageDataBuilder.addAccumulatorUpdates(Utils.serializeAccumulableInfo(update))
+    }

Review Comment:
   +1 for AccumulableInfoSerializer



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gengliangwang commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1059197136


##########
core/src/main/scala/org/apache/spark/status/protobuf/AccumulableInfoSerializer.scala:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.{List => JList}
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.status.api.v1.AccumulableInfo
+import org.apache.spark.status.protobuf.Utils.getOptional
+
+object AccumulableInfoSerializer {

Review Comment:
   So that we don't need to have `private[protobuf]` before each method



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
panbingkun commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1058801116


##########
core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto:
##########
@@ -390,3 +390,214 @@ message SQLExecutionUIData {
   repeated int64 stages = 11;
   map<int64, string> metric_values = 12;
 }
+
+message StageDataWrapper {
+  StageData info = 1;
+  repeated int64 job_ids = 2;
+  map<string, int64> locality = 3;
+}
+
+message TaskData {
+  int64 task_id = 1;
+  int32 index = 2;
+  int32 attempt = 3;
+  int32 partition_id = 4;
+  int64 launch_time = 5;
+  optional int64 result_fetch_start = 6;
+  optional int64 duration = 7;
+  string executor_id = 8;
+  string host = 9;
+  string status = 10;
+  string task_locality = 11;
+  bool speculative = 12;
+  repeated AccumulableInfo accumulator_updates = 13;
+  optional string error_message = 14;
+  optional TaskMetrics task_metrics = 15;
+  map<string, string> executor_logs = 16;
+  int64 scheduler_delay = 17;
+  int64 getting_result_time = 18;
+}
+
+message StageData {
+  enum StageStatus {

Review Comment:
   A.If the enum definition of `JobExecutionStatus` in JobData
   <img width="224" alt="image" src="https://user-images.githubusercontent.com/15246973/209922039-617b3993-b6c2-48dc-a24f-fdb343585c68.png">
   
   SQLExecutionUIData can use it as follows:
   <img width="414" alt="image" src="https://user-images.githubusercontent.com/15246973/209922101-a2504145-87a3-46f7-b918-6d4363773d17.png">
   
   B.Other choices for the definition of `JobExecutionStatus` as follows:
   <img width="233" alt="image" src="https://user-images.githubusercontent.com/15246973/209922683-296a5c81-daa8-48ad-8bd2-8ea4682953e5.png">
   <img width="699" alt="image" src="https://user-images.githubusercontent.com/15246973/209922716-51945000-6654-4283-affb-37e2dbd5978e.png">
   <img width="390" alt="image" src="https://user-images.githubusercontent.com/15246973/209922853-d0d1affa-6c94-454a-883a-b87551c7ddb0.png">
   
   Otherwise, I'm afraid there are other enumeration names that will conflict(eg: StageStatus.FAILED VS JobExecutionStatus.FAILED). @gengliangwang @LuciferYang 



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
panbingkun commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1058307735


##########
core/src/main/scala/org/apache/spark/status/protobuf/StageDataWrapperSerializer.scala:
##########
@@ -0,0 +1,627 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.Date
+
+import collection.JavaConverters._
+import org.apache.commons.collections4.MapUtils
+
+import org.apache.spark.status.StageDataWrapper
+import org.apache.spark.status.api.v1.{ExecutorMetricsDistributions, ExecutorPeakMetricsDistributions, InputMetricDistributions, InputMetrics, OutputMetricDistributions, OutputMetrics, ShuffleReadMetricDistributions, ShuffleReadMetrics, ShuffleWriteMetricDistributions, ShuffleWriteMetrics, SpeculationStageSummary, StageData, StageStatus, TaskData, TaskMetricDistributions, TaskMetrics}
+import org.apache.spark.status.protobuf.Utils.getOptional
+import org.apache.spark.util.Utils.weakIntern
+
+class StageDataWrapperSerializer extends ProtobufSerDe {
+
+  override val supportClass: Class[_] = classOf[StageDataWrapper]
+
+  override def serialize(input: Any): Array[Byte] =

Review Comment:
   Done



##########
core/src/main/scala/org/apache/spark/status/protobuf/AccumulableInfoSerializer.scala:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.{List => JList}
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.status.api.v1.AccumulableInfo
+import org.apache.spark.status.protobuf.Utils.getOptional
+
+object AccumulableInfoSerializer {
+
+  private[protobuf] def serializeAccumulableInfo(
+      input: AccumulableInfo): StoreTypes.AccumulableInfo = {
+    val builder = StoreTypes.AccumulableInfo.newBuilder()
+      .setId(input.id)
+      .setName(input.name)
+      .setValue(input.value)
+    input.update.foreach(builder.setUpdate)
+    builder.build()
+  }
+
+  private[protobuf] def deserializeAccumulableInfos(
+      updates: JList[StoreTypes.AccumulableInfo]): ArrayBuffer[AccumulableInfo] = {
+    val accumulatorUpdates = new ArrayBuffer[AccumulableInfo]()

Review Comment:
   Done



##########
core/src/main/scala/org/apache/spark/status/protobuf/AccumulableInfoSerializer.scala:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.{List => JList}
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.status.api.v1.AccumulableInfo
+import org.apache.spark.status.protobuf.Utils.getOptional
+
+object AccumulableInfoSerializer {
+
+  private[protobuf] def serializeAccumulableInfo(
+      input: AccumulableInfo): StoreTypes.AccumulableInfo = {
+    val builder = StoreTypes.AccumulableInfo.newBuilder()
+      .setId(input.id)
+      .setName(input.name)
+      .setValue(input.value)
+    input.update.foreach(builder.setUpdate)
+    builder.build()
+  }
+
+  private[protobuf] def deserializeAccumulableInfos(

Review Comment:
   Done



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #39192: [SPARK-41423][CORE] Protobuf serializer for StageDataWrapper

Posted by GitBox <gi...@apache.org>.
panbingkun commented on code in PR #39192:
URL: https://github.com/apache/spark/pull/39192#discussion_r1058307873


##########
core/src/main/scala/org/apache/spark/status/protobuf/AccumulableInfoSerializer.scala:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.spark.status.protobuf
+
+import java.util.{List => JList}
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.status.api.v1.AccumulableInfo
+import org.apache.spark.status.protobuf.Utils.getOptional
+
+object AccumulableInfoSerializer {
+
+  private[protobuf] def serializeAccumulableInfo(

Review Comment:
   Done



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org