Skip to content

Commit c995e07

Browse files
rxingatorsmile
authored andcommitted
[SPARK-26140] followup: rename ShuffleMetricsReporter
## What changes were proposed in this pull request? In apache#23105, due to working on two parallel PRs at once, I made the mistake of committing the copy of the PR that used the name ShuffleMetricsReporter for the interface, rather than the appropriate one ShuffleReadMetricsReporter. This patch fixes that. ## How was this patch tested? This should be fine as long as compilation passes. Closes apache#23147 from rxin/ShuffleReadMetricsReporter. Authored-by: Reynold Xin <[email protected]> Signed-off-by: gatorsmile <[email protected]>
1 parent 9deaa72 commit c995e07

File tree

6 files changed

+7
-40
lines changed

6 files changed

+7
-40
lines changed

core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,7 @@
1818
package org.apache.spark.executor
1919

2020
import org.apache.spark.annotation.DeveloperApi
21-
import org.apache.spark.shuffle.ShuffleMetricsReporter
21+
import org.apache.spark.shuffle.ShuffleReadMetricsReporter
2222
import org.apache.spark.util.LongAccumulator
2323

2424

@@ -130,7 +130,7 @@ class ShuffleReadMetrics private[spark] () extends Serializable {
130130
* shuffle dependency, and all temporary metrics will be merged into the [[ShuffleReadMetrics]] at
131131
* last.
132132
*/
133-
private[spark] class TempShuffleReadMetrics extends ShuffleMetricsReporter {
133+
private[spark] class TempShuffleReadMetrics extends ShuffleReadMetricsReporter {
134134
private[this] var _remoteBlocksFetched = 0L
135135
private[this] var _localBlocksFetched = 0L
136136
private[this] var _remoteBytesRead = 0L

core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,7 @@ private[spark] class BlockStoreShuffleReader[K, C](
3333
startPartition: Int,
3434
endPartition: Int,
3535
context: TaskContext,
36-
readMetrics: ShuffleMetricsReporter,
36+
readMetrics: ShuffleReadMetricsReporter,
3737
serializerManager: SerializerManager = SparkEnv.get.serializerManager,
3838
blockManager: BlockManager = SparkEnv.get.blockManager,
3939
mapOutputTracker: MapOutputTracker = SparkEnv.get.mapOutputTracker)

core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -49,7 +49,7 @@ private[spark] trait ShuffleManager {
4949
startPartition: Int,
5050
endPartition: Int,
5151
context: TaskContext,
52-
metrics: ShuffleMetricsReporter): ShuffleReader[K, C]
52+
metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C]
5353

5454
/**
5555
* Remove a shuffle's metadata from the ShuffleManager.

core/src/main/scala/org/apache/spark/shuffle/ShuffleMetricsReporter.scala

Lines changed: 0 additions & 33 deletions
This file was deleted.

core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -115,7 +115,7 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager
115115
startPartition: Int,
116116
endPartition: Int,
117117
context: TaskContext,
118-
metrics: ShuffleMetricsReporter): ShuffleReader[K, C] = {
118+
metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = {
119119
new BlockStoreShuffleReader(
120120
handle.asInstanceOf[BaseShuffleHandle[K, _, C]],
121121
startPartition, endPartition, context, metrics)

core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,7 @@ import org.apache.spark.internal.Logging
3030
import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer}
3131
import org.apache.spark.network.shuffle._
3232
import org.apache.spark.network.util.TransportConf
33-
import org.apache.spark.shuffle.{FetchFailedException, ShuffleMetricsReporter}
33+
import org.apache.spark.shuffle.{FetchFailedException, ShuffleReadMetricsReporter}
3434
import org.apache.spark.util.Utils
3535
import org.apache.spark.util.io.ChunkedByteBufferOutputStream
3636

@@ -73,7 +73,7 @@ final class ShuffleBlockFetcherIterator(
7373
maxBlocksInFlightPerAddress: Int,
7474
maxReqSizeShuffleToMem: Long,
7575
detectCorrupt: Boolean,
76-
shuffleMetrics: ShuffleMetricsReporter)
76+
shuffleMetrics: ShuffleReadMetricsReporter)
7777
extends Iterator[(BlockId, InputStream)] with DownloadFileManager with Logging {
7878

7979
import ShuffleBlockFetcherIterator._

0 commit comments

Comments
 (0)