LuciferYang commented on code in PR #52386:
URL: https://github.com/apache/spark/pull/52386#discussion_r2380944418


##########
core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala:
##########
@@ -58,19 +58,19 @@ private[spark] class IndexShuffleBlockResolver(
     conf: SparkConf,
     // var for testing
     var _blockManager: BlockManager,
-    val taskIdMapsForShuffle: JMap[Int, OpenHashSet[Long]])
+    val taskIdMapsForShuffle: ConcurrentHashMap[Int, OpenHashSet[Long]])
   extends ShuffleBlockResolver
   with Logging with MigratableResolver {
 
   def this(conf: SparkConf) = {
-    this(conf, null, Collections.emptyMap())
+    this(conf, null, new ConcurrentHashMap[Int, OpenHashSet[Long]]())

Review Comment:
   I take back my previous comment. 
   
   Passing in `Collections.emptyMap()` in the original code seems problematic. 
When `taskIdMapsForShuffle` is an instance of `EMPTY_MAP`, calling 
`taskIdMapsForShuffle.computeIfAbsent` will throw an exception, even though it 
seemingly didn't accidentally enter this path before. 
   
   So, it's indeed inappropriate to pass in an `EmptyConcurrentMap` similar to 
`Collections.emptyMap()` here.



-- 
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