mridulm commented on a change in pull request #31876:
URL: https://github.com/apache/spark/pull/31876#discussion_r608448359



##########
File path: core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala
##########
@@ -52,8 +53,20 @@ private[spark] sealed trait MapStatus {
    * partitionId of the task or taskContext.taskAttemptId is used.
    */
   def mapId: Long
-}
 
+  protected def loadLocation(in: ObjectInput): Location = {
+    val conf = SparkEnv.get.conf
+    conf.get(config.SHUFFLE_LOCATION_PLUGIN_CLASS).map { locClass =>
+      val loc = Utils.loadExtensions(
+        classOf[Location],
+        Seq(locClass),
+        conf
+      ).head
+      loc.readExternal(in)
+      loc

Review comment:
       Do we want to explore a `LocationFactory` in addition to `Location` - 
which is used here ?
   `loadExtensions` is not cheap, and doing this for every instance creation is 
going to be very expensive.
   
   A factory will also allow for implementations to cache `Location` - else we 
will have GC issues at driver.




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

Reply via email to