mridulm commented on code in PR #40690:
URL: https://github.com/apache/spark/pull/40690#discussion_r1160407817


##########
core/src/main/scala/org/apache/spark/MapOutputTracker.scala:
##########
@@ -157,22 +164,29 @@ private class ShuffleStatus(
       invalidateSerializedMapOutputStatusCache()
     }
     mapStatuses(mapIndex) = status
+    mapIdToMapIndex(status.mapId) = mapIndex
   }
 
   /**
    * 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)
+      // OpenHashMap would return 0 if the key doesn't exist.
+      val mapIndex = if (mapIdToMapIndex.contains(mapId)) {
+        Some(mapIdToMapIndex(mapId))
+      } else {
+        None
+      }

Review Comment:
   Thoughts on changing this to either:
   * Add a `get(key): Option[V]` to `OpenHashMap` or
   * Simply do `Option(mapIdToMapIndex(mapId))` ?
   
   The second option relies on implementation detail that we return `null` when 
`apply` does not find a `key` - while the first option is more principled 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to