You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "mridulm (via GitHub)" <gi...@apache.org> on 2023/05/06 08:56:45 UTC
[GitHub] [spark] mridulm commented on a diff in pull request #40690: [SPARK-43043][CORE] Improve the performance of MapOutputTracker.updateMapOutput
mridulm commented on code in PR #40690:
URL: https://github.com/apache/spark/pull/40690#discussion_r1186666232
##########
core/src/main/scala/org/apache/spark/MapOutputTracker.scala:
##########
@@ -157,22 +164,32 @@ private class ShuffleStatus(
invalidateSerializedMapOutputStatusCache()
}
mapStatuses(mapIndex) = status
+ mapIdToMapIndex(status.mapId) = mapIndex
+ }
+
+ /**
+ * Get the map output that corresponding to a given mapId.
+ */
+ def getMapStatus(mapId: Long): Option[MapStatus] = withReadLock {
+ mapIdToMapIndex.get(mapId).map(mapStatuses(_))
}
/**
* Update the map output location (e.g. during migration).
*/
def updateMapOutput(mapId: Long, bmAddress: BlockManagerId): Unit = withWriteLock {
try {
- val mapStatusOpt = mapStatuses.find(x => x != null && x.mapId == mapId)
+ val mapIndex = mapIdToMapIndex.get(mapId)
+ val mapStatusOpt = Option(mapIndex.map(mapStatuses(_)).getOrElse(null))
Review Comment:
```suggestion
val mapStatusOpt = mapIndex.map(mapStatuses(_)
```
##########
core/src/main/scala/org/apache/spark/MapOutputTracker.scala:
##########
@@ -157,22 +164,32 @@ private class ShuffleStatus(
invalidateSerializedMapOutputStatusCache()
}
mapStatuses(mapIndex) = status
+ mapIdToMapIndex(status.mapId) = mapIndex
+ }
+
+ /**
+ * Get the map output that corresponding to a given mapId.
+ */
+ def getMapStatus(mapId: Long): Option[MapStatus] = withReadLock {
+ mapIdToMapIndex.get(mapId).map(mapStatuses(_))
}
/**
* Update the map output location (e.g. during migration).
*/
def updateMapOutput(mapId: Long, bmAddress: BlockManagerId): Unit = withWriteLock {
try {
- val mapStatusOpt = mapStatuses.find(x => x != null && x.mapId == mapId)
+ val mapIndex = mapIdToMapIndex.get(mapId)
+ val mapStatusOpt = Option(mapIndex.map(mapStatuses(_)).getOrElse(null))
mapStatusOpt match {
case Some(mapStatus) =>
logInfo(s"Updating map output for ${mapId} to ${bmAddress}")
mapStatus.updateLocation(bmAddress)
invalidateSerializedMapOutputStatusCache()
case None =>
- val index = mapStatusesDeleted.indexWhere(x => x != null && x.mapId == mapId)
- if (index >= 0 && mapStatuses(index) == null) {
+ if (mapIndex.nonEmpty && mapStatuses(mapIndex.get) == null &&
+ mapStatusesDeleted(mapIndex.get).mapId == mapId) {
Review Comment:
We dont need to check for `mapStatuses(mapIndex.get) == null` - it should already be `null` since we are in the `None` case.
```suggestion
if (mapIndex.map(mapStatusesDeleted).exists(_.mapId == mapId) {
```
--
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