dongjoon-hyun commented on a change in pull request #32730:
URL: https://github.com/apache/spark/pull/32730#discussion_r645698008
##########
File path: core/src/main/scala/org/apache/spark/MapOutputTracker.scala
##########
@@ -154,14 +160,25 @@ private class ShuffleStatus(
*/
def updateMapOutput(mapId: Long, bmAddress: BlockManagerId): Unit =
withWriteLock {
try {
- val mapStatusOpt = mapStatuses.find(_.mapId == mapId)
+ val mapStatusOpt = mapStatuses.find(x => x != null && x.mapId == mapId)
mapStatusOpt match {
case Some(mapStatus) =>
logInfo(s"Updating map output for ${mapId} to ${bmAddress}")
mapStatus.updateLocation(bmAddress)
invalidateSerializedMapOutputStatusCache()
case None =>
- logWarning(s"Asked to update map output ${mapId} for untracked map
status.")
+ val index = mapStatusesDeleted.indexWhere(x => x != null && x.mapId
== mapId)
+ if (index >= 0) {
+ val mapStatus = mapStatusesDeleted(index)
+ mapStatus.updateLocation(bmAddress)
+ assert(mapStatuses(index) == null)
+ mapStatuses(index) = mapStatus
+ _numAvailableMapOutputs += 1
+ invalidateSerializedMapOutputStatusCache()
+ logDebug(s"Recover ${mapStatus.mapId} ${mapStatus.location}")
Review comment:
Right, it will be matched at that time. I'll update the 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.
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]