MapOutputTracker

MapOutputTracker is available using SparkEnv (on the driver and executors).

SparkEnv.get.mapOutputTracker

MapOutputTrackers

MapOutputTracker Description

MapOutputTrackerMaster

Runs on the driver

MapOutputTrackerWorker

Runs on executors

Creating Instance

MapOutputTracker takes a single SparkConf to be created.

MapOutputTracker RPC Endpoint

trackerEndpoint is a RpcEndpointRef of the MapOutputTracker RPC endpoint.

trackerEndpoint is initialized (registered or looked up) when SparkEnv is created for the driver and executors.

trackerEndpoint is used to communicate (synchronously).

trackerEndpoint is cleared (null) when MapOutputTrackerMaster is requested to stop.

serializeMapStatuses Utility

serializeMapStatuses(
  statuses: Array[MapStatus],
  broadcastManager: BroadcastManager,
  isLocal: Boolean,
  minBroadcastSize: Int): (Array[Byte], Broadcast[Array[Byte]])

serializeMapStatuses serializes the given array of map output locations into an efficient byte format (to send to reduce tasks). serializeMapStatuses compresses the serialized bytes using GZIP. They are supposed to be pretty compressible because many map outputs will be on the same hostname.

Internally, serializeMapStatuses creates a Java ByteArrayOutputStream.

serializeMapStatuses writes out 0 (direct) first.

serializeMapStatuses creates a Java GZIPOutputStream (with the ByteArrayOutputStream created) and writes out the given statuses array.

serializeMapStatuses decides whether to return the output array (of the output stream) or use a broadcast variable based on the size of the byte array.

If the size of the result byte array is the given minBroadcastSize threshold or bigger, serializeMapStatuses requests the input BroadcastManager to create a broadcast variable.

serializeMapStatuses resets the ByteArrayOutputStream and starts over.

serializeMapStatuses writes out 1 (broadcast) first.

serializeMapStatuses creates a new Java GZIPOutputStream (with the ByteArrayOutputStream created) and writes out the broadcast variable.

serializeMapStatuses prints out the following INFO message to the logs:

Broadcast mapstatuses size = [length], actual size = [length]

serializeMapStatuses is used when ShuffleStatus is requested to serialize shuffle map output statuses.

deserializeMapStatuses Utility

deserializeMapStatuses(
  bytes: Array[Byte]): Array[MapStatus]

deserializeMapStatuses…​FIXME

deserializeMapStatuses is used when…​FIXME

sendTracker Utility

sendTracker(
  message: Any): Unit

sendTracker…​FIXME

sendTracker is used when…​FIXME

Finding Locations with Blocks and Sizes

getMapSizesByExecutorId(
  shuffleId: Int,
  startPartition: Int,
  endPartition: Int): Seq[(BlockManagerId, Seq[(BlockId, Long)])]

getMapSizesByExecutorId returns a collection of BlockManagerIds with their blocks and sizes.

When executed, you should see the following DEBUG message in the logs:

Fetching outputs for shuffle [id], partitions [startPartition]-[endPartition]

getMapSizesByExecutorId finds map outputs for the input shuffleId.

getMapSizesByExecutorId gets the map outputs for all the partitions (despite the method’s signature).

In the end, getMapSizesByExecutorId converts shuffle map outputs (as MapStatuses) into the collection of BlockManagerIds with their blocks and sizes.

getMapSizesByExecutorId is used when BlockStoreShuffleReader is requested to read combined records for a reduce task.

Deregistering Map Output Status Information of Shuffle Stage

unregisterShuffle(
  shuffleId: Int): Unit

Deregisters map output status information for the given shuffle stage

Used when:

Stopping MapOutputTracker

stop(): Unit

stop does nothing at all.

stop is used when SparkEnv is requested to stop (and stops all the services, incl. MapOutputTracker).

Converting MapStatuses To BlockManagerIds with ShuffleBlockIds and Their Sizes

convertMapStatuses(
  shuffleId: Int,
  startPartition: Int,
  endPartition: Int,
  statuses: Array[MapStatus]): Seq[(BlockManagerId, Seq[(BlockId, Long)])]

convertMapStatuses iterates over the input statuses array (of MapStatus entries indexed by map id) and creates a collection of BlockManagerId (for each MapStatus entry) with a ShuffleBlockId (with the input shuffleId, a mapId, and partition ranging from the input startPartition and endPartition) and estimated size for the reduce block for every status and partitions.

For any empty MapStatus, you should see the following ERROR message in the logs:

Missing an output location for shuffle [id]

And convertMapStatuses throws a MetadataFetchFailedException (with shuffleId, startPartition, and the above error message).

Sending Blocking Messages To trackerEndpoint RpcEndpointRef

askTracker[T](message: Any): T

askTracker sends the message to trackerEndpoint RpcEndpointRef and waits for a result.

When an exception happens, you should see the following ERROR message in the logs and askTracker throws a SparkException.

Error communicating with MapOutputTracker

askTracker is used when MapOutputTracker fetches map outputs for ShuffleDependency remotely and sends a one-way message.

Epoch

Starts from 0 when MapOutputTracker is created.

Can be updated (on MapOutputTrackerWorkers) or incremented (on the driver’s MapOutputTrackerMaster).