aboutsummaryrefslogtreecommitdiff
path: root/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala
diff options
context:
space:
mode:
Diffstat (limited to 'core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala')
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala34
1 files changed, 27 insertions, 7 deletions
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala
index f255f5be63..c37a3604d2 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala
@@ -37,10 +37,11 @@ import org.apache.spark.util.Utils
class BlockManagerId private (
private var executorId_ : String,
private var host_ : String,
- private var port_ : Int)
+ private var port_ : Int,
+ private var topologyInfo_ : Option[String])
extends Externalizable {
- private def this() = this(null, null, 0) // For deserialization only
+ private def this() = this(null, null, 0, None) // For deserialization only
def executorId: String = executorId_
@@ -60,6 +61,8 @@ class BlockManagerId private (
def port: Int = port_
+ def topologyInfo: Option[String] = topologyInfo_
+
def isDriver: Boolean = {
executorId == SparkContext.DRIVER_IDENTIFIER ||
executorId == SparkContext.LEGACY_DRIVER_IDENTIFIER
@@ -69,24 +72,33 @@ class BlockManagerId private (
out.writeUTF(executorId_)
out.writeUTF(host_)
out.writeInt(port_)
+ out.writeBoolean(topologyInfo_.isDefined)
+ // we only write topologyInfo if we have it
+ topologyInfo.foreach(out.writeUTF(_))
}
override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException {
executorId_ = in.readUTF()
host_ = in.readUTF()
port_ = in.readInt()
+ val isTopologyInfoAvailable = in.readBoolean()
+ topologyInfo_ = if (isTopologyInfoAvailable) Option(in.readUTF()) else None
}
@throws(classOf[IOException])
private def readResolve(): Object = BlockManagerId.getCachedBlockManagerId(this)
- override def toString: String = s"BlockManagerId($executorId, $host, $port)"
+ override def toString: String = s"BlockManagerId($executorId, $host, $port, $topologyInfo)"
- override def hashCode: Int = (executorId.hashCode * 41 + host.hashCode) * 41 + port
+ override def hashCode: Int =
+ ((executorId.hashCode * 41 + host.hashCode) * 41 + port) * 41 + topologyInfo.hashCode
override def equals(that: Any): Boolean = that match {
case id: BlockManagerId =>
- executorId == id.executorId && port == id.port && host == id.host
+ executorId == id.executorId &&
+ port == id.port &&
+ host == id.host &&
+ topologyInfo == id.topologyInfo
case _ =>
false
}
@@ -101,10 +113,18 @@ private[spark] object BlockManagerId {
* @param execId ID of the executor.
* @param host Host name of the block manager.
* @param port Port of the block manager.
+ * @param topologyInfo topology information for the blockmanager, if available
+ * This can be network topology information for use while choosing peers
+ * while replicating data blocks. More information available here:
+ * [[org.apache.spark.storage.TopologyMapper]]
* @return A new [[org.apache.spark.storage.BlockManagerId]].
*/
- def apply(execId: String, host: String, port: Int): BlockManagerId =
- getCachedBlockManagerId(new BlockManagerId(execId, host, port))
+ def apply(
+ execId: String,
+ host: String,
+ port: Int,
+ topologyInfo: Option[String] = None): BlockManagerId =
+ getCachedBlockManagerId(new BlockManagerId(execId, host, port, topologyInfo))
def apply(in: ObjectInput): BlockManagerId = {
val obj = new BlockManagerId()