[GitHub] [spark] thejdeep commented on a diff in pull request #36165: [SPARK-36620][SHUFFLE] Add Push Based Shuffle client side read metrics

2022-11-29 Thread GitBox


thejdeep commented on code in PR #36165:
URL: https://github.com/apache/spark/pull/36165#discussion_r1035141286


##
core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala:
##
@@ -866,6 +930,7 @@ final class ShuffleBlockFetcherIterator(
 }
 
 if (blockId.isShuffleChunk) {
+  shuffleMetrics.incCorruptMergedBlockChunks(1)

Review Comment:
   Thanks for calling this out, added this change.



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] thejdeep commented on a diff in pull request #36165: [SPARK-36620][SHUFFLE] Add Push Based Shuffle client side read metrics

2022-11-29 Thread GitBox


thejdeep commented on code in PR #36165:
URL: https://github.com/apache/spark/pull/36165#discussion_r1035142645


##
core/src/main/scala/org/apache/spark/InternalAccumulator.scala:
##
@@ -54,6 +55,16 @@ private[spark] object InternalAccumulator {
 val LOCAL_BYTES_READ = SHUFFLE_READ_METRICS_PREFIX + "localBytesRead"
 val FETCH_WAIT_TIME = SHUFFLE_READ_METRICS_PREFIX + "fetchWaitTime"
 val RECORDS_READ = SHUFFLE_READ_METRICS_PREFIX + "recordsRead"
+val REMOTE_REQS_DURATION = SHUFFLE_READ_METRICS_PREFIX + 
"remoteReqsDuration"
+val CORRUPT_MERGED_BLOCK_CHUNKS = SHUFFLE_PUSH_READ_METRICS_PREFIX + 
"corruptMergedBlockChunks"
+val FALLBACK_COUNT = SHUFFLE_PUSH_READ_METRICS_PREFIX + "fallbackCount"

Review Comment:
   renamed in all sources to `mergedFetchFallbackCount `



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] thejdeep commented on a diff in pull request #36165: [SPARK-36620][SHUFFLE] Add Push Based Shuffle client side read metrics

2022-12-05 Thread GitBox


thejdeep commented on code in PR #36165:
URL: https://github.com/apache/spark/pull/36165#discussion_r1039691827


##
core/src/main/scala/org/apache/spark/executor/Executor.scala:
##
@@ -791,6 +770,53 @@ private[spark] class Executor(
   }
 }
 
+private def incrementShuffleMetrics(
+  executorSource: ExecutorSource,
+  metrics: TaskMetrics
+): Unit = {
+  executorSource.METRIC_SHUFFLE_FETCH_WAIT_TIME
+.inc(metrics.shuffleReadMetrics.fetchWaitTime)
+  
executorSource.METRIC_SHUFFLE_WRITE_TIME.inc(metrics.shuffleWriteMetrics.writeTime)
+  executorSource.METRIC_SHUFFLE_TOTAL_BYTES_READ
+.inc(metrics.shuffleReadMetrics.totalBytesRead)
+  executorSource.METRIC_SHUFFLE_REMOTE_BYTES_READ
+.inc(metrics.shuffleReadMetrics.remoteBytesRead)
+  executorSource.METRIC_SHUFFLE_REMOTE_BYTES_READ_TO_DISK
+.inc(metrics.shuffleReadMetrics.remoteBytesReadToDisk)
+  executorSource.METRIC_SHUFFLE_LOCAL_BYTES_READ
+.inc(metrics.shuffleReadMetrics.localBytesRead)
+  executorSource.METRIC_SHUFFLE_RECORDS_READ
+.inc(metrics.shuffleReadMetrics.recordsRead)
+  executorSource.METRIC_SHUFFLE_REMOTE_BLOCKS_FETCHED
+.inc(metrics.shuffleReadMetrics.remoteBlocksFetched)
+  executorSource.METRIC_SHUFFLE_LOCAL_BLOCKS_FETCHED
+.inc(metrics.shuffleReadMetrics.localBlocksFetched)
+  executorSource.METRIC_SHUFFLE_REMOTE_REQS_DURATION
+.inc(metrics.shuffleReadMetrics.remoteReqsDuration)
+  executorSource.METRIC_SHUFFLE_BYTES_WRITTEN
+.inc(metrics.shuffleWriteMetrics.bytesWritten)
+  executorSource.METRIC_SHUFFLE_RECORDS_WRITTEN
+.inc(metrics.shuffleWriteMetrics.recordsWritten)
+  executorSource.METRIC_PUSH_BASED_SHUFFLE_CORRUPT_MERGED_BLOCK_CHUNKS
+.inc(metrics.shuffleReadMetrics.corruptMergedBlockChunks)
+  executorSource.METRIC_PUSH_BASED_SHUFFLE_MERGED_FETCH_FALLBACK_COUNT
+.inc(metrics.shuffleReadMetrics.fallbackCount)

Review Comment:
   Thanks, updated to reflect this.



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] thejdeep commented on a diff in pull request #36165: [SPARK-36620][SHUFFLE] Add Push Based Shuffle client side read metrics

2022-12-05 Thread GitBox


thejdeep commented on code in PR #36165:
URL: https://github.com/apache/spark/pull/36165#discussion_r1040478699


##
core/src/test/resources/HistoryServerExpectations/excludeOnFailure_node_for_stage_expectation.json:
##
@@ -81,7 +93,19 @@
   "remoteBytesRead" : 0,
   "remoteBytesReadToDisk" : 0,
   "localBytesRead" : 0,
-  "recordsRead" : 0
+  "recordsRead" : 0,
+  "pushBased": {

Review Comment:
   This is the API response that serialized based on the [case classes in 
api.scala 
here](https://github.com/apache/spark/blob/7259a8c0147d9da70848ef2e7499a1712a4c4741/core/src/main/scala/org/apache/spark/status/api/v1/api.scala#L384).
 We only write "Push Based Shuffle" when logging to the event file



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] thejdeep commented on a diff in pull request #36165: [SPARK-36620][SHUFFLE] Add Push Based Shuffle client side read metrics

2022-12-06 Thread GitBox


thejdeep commented on code in PR #36165:
URL: https://github.com/apache/spark/pull/36165#discussion_r1040884126


##
core/src/test/resources/HistoryServerExpectations/excludeOnFailure_node_for_stage_expectation.json:
##
@@ -81,7 +93,19 @@
   "remoteBytesRead" : 0,
   "remoteBytesReadToDisk" : 0,
   "localBytesRead" : 0,
-  "recordsRead" : 0
+  "recordsRead" : 0,
+  "pushBased": {

Review Comment:
   Do you recommend changing this to "pushBasedShuffle" ? to note:  it is 
inside the context of shuffle read metrics)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] thejdeep commented on a diff in pull request #36165: [SPARK-36620][SHUFFLE] Add Push Based Shuffle client side read metrics

2022-12-07 Thread GitBox


thejdeep commented on code in PR #36165:
URL: https://github.com/apache/spark/pull/36165#discussion_r1042043726


##
core/src/test/resources/HistoryServerExpectations/excludeOnFailure_node_for_stage_expectation.json:
##
@@ -81,7 +93,19 @@
   "remoteBytesRead" : 0,
   "remoteBytesReadToDisk" : 0,
   "localBytesRead" : 0,
-  "recordsRead" : 0
+  "recordsRead" : 0,
+  "pushBased": {

Review Comment:
   Changed it to "pushBasedShuffle", thanks for the suggestion.



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] thejdeep commented on a diff in pull request #36165: [SPARK-36620][SHUFFLE] Add Push Based Shuffle client side read metrics

2022-12-12 Thread GitBox


thejdeep commented on code in PR #36165:
URL: https://github.com/apache/spark/pull/36165#discussion_r1046138205


##
core/src/main/scala/org/apache/spark/status/api/v1/api.scala:
##
@@ -302,7 +312,9 @@ class StageData private[spark](
 @JsonDeserialize(using = classOf[ExecutorMetricsJsonDeserializer])
 val peakExecutorMetrics: Option[ExecutorMetrics],
 val taskMetricsDistributions: Option[TaskMetricDistributions],
-val executorMetricsDistributions: Option[ExecutorMetricsDistributions])
+val executorMetricsDistributions: Option[ExecutorMetricsDistributions],
+val isPushBasedShuffleEnabled: Boolean,

Review Comment:
   Removed `Based` from user facing metrics.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] thejdeep commented on a diff in pull request #36165: [SPARK-36620][SHUFFLE] Add Push Based Shuffle client side read metrics

2022-12-12 Thread GitBox


thejdeep commented on code in PR #36165:
URL: https://github.com/apache/spark/pull/36165#discussion_r1046139711


##
core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala:
##
@@ -726,6 +736,61 @@ final class ShuffleBlockFetcherIterator(
 }
   }
 
+  // Number of map blocks in a ShuffleBlockChunk
+  private def getShuffleChunkCardinality(blockId: ShuffleBlockChunkId): Int = {
+
pushBasedFetchHelper.getRoaringBitMap(blockId).map(_.getCardinality).getOrElse(0)
+  }
+
+  // Check if the merged block is local to the host or not
+  private def isLocalMergedBlockAddress(address: BlockManagerId): Boolean = {
+address.executorId.isEmpty && address.host == 
blockManager.blockManagerId.host

Review Comment:
   Reused method from `PushBasedFetchHelper`. There was a class instance 
available for use, so I used that instead of making the function static.



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] thejdeep commented on a diff in pull request #36165: [SPARK-36620][SHUFFLE] Add Push Based Shuffle client side read metrics

2022-12-12 Thread GitBox


thejdeep commented on code in PR #36165:
URL: https://github.com/apache/spark/pull/36165#discussion_r1046140712


##
core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json:
##
@@ -27,13 +27,23 @@
   "outputBytes" : 0,
   "outputRecords" : 0,
   "shuffleRemoteBlocksFetched" : 0,
-  "shuffleLocalBlocksFetched" : 64,
-  "shuffleFetchWaitTime" : 1,
+  "shuffleLocalBlocksFetched" : 0,
+  "shuffleFetchWaitTime" : 0,

Review Comment:
   There was a logical bug in merging the read metrics, thanks for catching 
this! Updated the expectation JSON.



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] thejdeep commented on a diff in pull request #36165: [SPARK-36620][SHUFFLE] Add Push Based Shuffle client side read metrics

2022-12-21 Thread GitBox


thejdeep commented on code in PR #36165:
URL: https://github.com/apache/spark/pull/36165#discussion_r1054375191


##
core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala:
##
@@ -1653,7 +1663,19 @@ private[spark] object JsonProtocolSuite extends 
Assertions {
   |  "Remote Bytes Read": 1000,
   |  "Remote Bytes Read To Disk": 400,
   |  "Local Bytes Read": 1100,
-  |  "Total Records Read": 10
+  |  "Total Records Read": 10,
+  |  "Remote Requests Duration": 900,
+  |  "Push Based Shuffle": {
+  | "Corrupt Merged Block Chunks" : 100,
+  | "Merged Fetch Fallback Count" : 100,
+  | "Merged Remote Blocks Fetched" : 0,
+  | "Merged Local Blocks Fetched" : 0,
+  | "Merged Remote Chunks Fetched" : 0,
+  | "Merged Local Chunks Fetched" : 0,
+  | "Merged Remote Bytes Read" : 0,
+  | "Merged Local Bytes Read" : 0,
+  | "Merged Remote Requests Duration": 900

Review Comment:
   This is [currently being 
set](https://github.com/apache/spark/blob/e46031651617393bb182e44ae04920d6c89592f6/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala#L1315)
 to `executorDeserializeTime` + `jvmGCTime` while making the test task metrics, 
do you think that assumption is incorrect ? 



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] thejdeep commented on a diff in pull request #36165: [SPARK-36620][SHUFFLE] Add Push Based Shuffle client side read metrics

2022-12-27 Thread GitBox


thejdeep commented on code in PR #36165:
URL: https://github.com/apache/spark/pull/36165#discussion_r1058019046


##
core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto:
##
@@ -100,11 +100,21 @@ message TaskDataWrapper {
   int64 shuffle_remote_bytes_read_to_disk = 34;
   int64 shuffle_local_bytes_read = 35;
   int64 shuffle_records_read = 36;
-  int64 shuffle_bytes_written = 37;
-  int64 shuffle_write_time = 38;
-  int64 shuffle_records_written = 39;
-  int64 stage_id = 40;
-  int32 stage_attempt_id = 41;
+  int64 shuffle_corrupt_merged_block_chunks = 37;
+  int64 shuffle_merged_fetch_fallback_count = 38;
+  int64 shuffle_merged_remote_blocks_fetched = 39;
+  int64 shuffle_merged_local_blocks_fetched = 40;
+  int64 shuffle_merged_remote_chunks_fetched = 41;
+  int64 shuffle_merged_local_chunks_fetched = 42;
+  int64 shuffle_merged_remote_bytes_read = 43;
+  int64 shuffle_merged_local_bytes_read = 44;
+  int64 shuffle_remote_reqs_duration = 45;
+  int64 shuffle_merged_remote_req_duration = 46;
+  int64 shuffle_bytes_written = 47;
+  int64 shuffle_write_time = 48;
+  int64 shuffle_records_written = 49;
+  int64 stage_id = 50;
+  int32 stage_attempt_id = 51;

Review Comment:
   @mridulm Do you suggest we keep the attribute `id`s same as before ? This 
would mean having to change the function definitions across most of the diff in 
this PR



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] thejdeep commented on a diff in pull request #36165: [SPARK-36620][SHUFFLE] Add Push Based Shuffle client side read metrics

2022-12-27 Thread GitBox


thejdeep commented on code in PR #36165:
URL: https://github.com/apache/spark/pull/36165#discussion_r1058019046


##
core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto:
##
@@ -100,11 +100,21 @@ message TaskDataWrapper {
   int64 shuffle_remote_bytes_read_to_disk = 34;
   int64 shuffle_local_bytes_read = 35;
   int64 shuffle_records_read = 36;
-  int64 shuffle_bytes_written = 37;
-  int64 shuffle_write_time = 38;
-  int64 shuffle_records_written = 39;
-  int64 stage_id = 40;
-  int32 stage_attempt_id = 41;
+  int64 shuffle_corrupt_merged_block_chunks = 37;
+  int64 shuffle_merged_fetch_fallback_count = 38;
+  int64 shuffle_merged_remote_blocks_fetched = 39;
+  int64 shuffle_merged_local_blocks_fetched = 40;
+  int64 shuffle_merged_remote_chunks_fetched = 41;
+  int64 shuffle_merged_local_chunks_fetched = 42;
+  int64 shuffle_merged_remote_bytes_read = 43;
+  int64 shuffle_merged_local_bytes_read = 44;
+  int64 shuffle_remote_reqs_duration = 45;
+  int64 shuffle_merged_remote_req_duration = 46;
+  int64 shuffle_bytes_written = 47;
+  int64 shuffle_write_time = 48;
+  int64 shuffle_records_written = 49;
+  int64 stage_id = 50;
+  int32 stage_attempt_id = 51;

Review Comment:
   @mridulm Do you suggest we keep the attribute `id`s same as before ? This 
would mean having to change the function definitions and usages across most of 
the diff in this PR



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] thejdeep commented on a diff in pull request #36165: [SPARK-36620][SHUFFLE] Add Push Based Shuffle client side read metrics

2023-01-03 Thread GitBox


thejdeep commented on code in PR #36165:
URL: https://github.com/apache/spark/pull/36165#discussion_r1060720199


##
core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto:
##
@@ -100,11 +100,21 @@ message TaskDataWrapper {
   int64 shuffle_remote_bytes_read_to_disk = 34;
   int64 shuffle_local_bytes_read = 35;
   int64 shuffle_records_read = 36;
-  int64 shuffle_bytes_written = 37;
-  int64 shuffle_write_time = 38;
-  int64 shuffle_records_written = 39;
-  int64 stage_id = 40;
-  int32 stage_attempt_id = 41;
+  int64 shuffle_corrupt_merged_block_chunks = 37;
+  int64 shuffle_merged_fetch_fallback_count = 38;
+  int64 shuffle_merged_remote_blocks_fetched = 39;
+  int64 shuffle_merged_local_blocks_fetched = 40;
+  int64 shuffle_merged_remote_chunks_fetched = 41;
+  int64 shuffle_merged_local_chunks_fetched = 42;
+  int64 shuffle_merged_remote_bytes_read = 43;
+  int64 shuffle_merged_local_bytes_read = 44;
+  int64 shuffle_remote_reqs_duration = 45;
+  int64 shuffle_merged_remote_req_duration = 46;
+  int64 shuffle_bytes_written = 47;
+  int64 shuffle_write_time = 48;
+  int64 shuffle_records_written = 49;
+  int64 stage_id = 50;
+  int32 stage_attempt_id = 51;

Review Comment:
   @mridulm Thanks for reviewing, changed it to keep the existing protobuf ids



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] thejdeep commented on a diff in pull request #36165: [SPARK-36620][SHUFFLE] Add Push Based Shuffle client side read metrics

2023-01-04 Thread GitBox


thejdeep commented on code in PR #36165:
URL: https://github.com/apache/spark/pull/36165#discussion_r1061723365


##
core/src/main/scala/org/apache/spark/status/storeTypes.scala:
##
@@ -233,6 +243,38 @@ private[spark] class TaskDataWrapper(
 val shuffleLocalBytesRead: Long,
 @KVIndexParam(value = TaskIndexNames.SHUFFLE_READ_RECORDS, parent = 
TaskIndexNames.STAGE)
 val shuffleRecordsRead: Long,
+@KVIndexParam(

Review Comment:
   Although we do not sort these in the UI, adding indices for these are a 
requirement for computing quantiles and while fetching `TaskDataWrapper`s.



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] thejdeep commented on a diff in pull request #36165: [SPARK-36620][SHUFFLE] Add Push Based Shuffle client side read metrics

2023-01-05 Thread GitBox


thejdeep commented on code in PR #36165:
URL: https://github.com/apache/spark/pull/36165#discussion_r1063026190


##
core/src/main/scala/org/apache/spark/status/protobuf/StageDataWrapperSerializer.scala:
##
@@ -496,7 +555,52 @@ class StageDataWrapperSerializer extends ProtobufSerDe {
   remoteBytesRead = 
binary.getRemoteBytesReadList.asScala.map(_.toDouble).toIndexedSeq,
   remoteBytesReadToDisk =
 
binary.getRemoteBytesReadToDiskList.asScala.map(_.toDouble).toIndexedSeq,
-  totalBlocksFetched = 
binary.getTotalBlocksFetchedList.asScala.map(_.toDouble).toIndexedSeq
+  totalBlocksFetched = 
binary.getTotalBlocksFetchedList.asScala.map(_.toDouble).toIndexedSeq,
+  remoteReqsDuration = 
binary.getRemoteReqsDurationList.asScala.map(_.toDouble).toIndexedSeq,
+  shufflePushReadMetricsDist =
+
deserializeShufflePushReadMetricsDistributions(binary.getShufflePushReadMetricsDist)
+)
+  }
+
+  private def deserializeShufflePushReadMetricsDistributions(
+  binary: StoreTypes.ShuffleReadMetricDistributions): 
ShuffleReadMetricDistributions = {

Review Comment:
   Removed the duplicate method, thanks.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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


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