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



##########
File path: core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala
##########
@@ -28,16 +28,23 @@ import org.apache.spark.internal.config
 import org.apache.spark.storage.BlockManagerId
 import org.apache.spark.util.Utils
 
+trait Location extends Externalizable {
+  def host: String
+  def port: Int
+  def hostPort: String
+  def executorId: String = "unknown"

Review comment:
       I think it (exposing `BlockManagerId`) depends on the use cases. We 
could probably expose it if `BlockManagerId` is generally satisfied all the 
common use cases(e.g., BlockManager, hdfs, file server). That being said, I 
think the concept of `BlockManagerId` is not suitable for every case. For 
example, each executor would be corresponding to a `BlockManagerId`.  But, in 
the case of hdfs, all the executors would only have one corresponding location 
indeed. 
   
   Adding `Location` would be more flexible for users if they have specific 
requirements that `BlockManagerId` can't satisfy. Besides, I think it's safer 
for Spark as `BlockManagerId` is not only used for shuffle phase but also 
widely used in RDD cache.




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