@@ -355,6 +355,21 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging
355355 startPartition : Int ,
356356 endPartition : Int ): Iterator [(BlockManagerId , Seq [(BlockId , Long , Int )])]
357357
358+ /**
359+ * Called from executors to get the server URIs and output sizes for each shuffle block that
360+ * needs to be read from a specific map output partitions (partitionIndex) and is
361+ * produced by a range mapper (startMapId, endMapId)
362+ *
363+ * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId,
364+ * and the second item is a sequence of (shuffle block id, shuffle block size, map index)
365+ * tuples describing the shuffle blocks that are stored at that block manager.
366+ */
367+ def getMapSizesByRangeMapIndex (
368+ shuffleId : Int ,
369+ partitionIndex : Int ,
370+ startMapId : Int ,
371+ endMapId : Int ): Iterator [(BlockManagerId , Seq [(BlockId , Long , Int )])]
372+
358373 /**
359374 * Deletes map output status information for the specified shuffle stage.
360375 */
@@ -688,21 +703,28 @@ private[spark] class MapOutputTrackerMaster(
688703 }
689704
690705 /**
691- * Return the location where the Mapper ran. The locations each includes both a host and an
706+ * Return the locations where the Mappers ran. The locations each includes both a host and an
692707 * executor id on that host.
693708 *
694709 * @param dep shuffle dependency object
695- * @param mapId the map id
710+ * @param startMapId the start map id
711+ * @param endMapId the end map id
696712 * @return a sequence of locations where task runs.
697713 */
698- def getMapLocation (dep : ShuffleDependency [_, _, _], mapId : Int ): Seq [String ] =
714+ def getMapLocation (
715+ dep : ShuffleDependency [_, _, _],
716+ startMapId : Int ,
717+ endMapId : Int ): Seq [String ] =
699718 {
700719 val shuffleStatus = shuffleStatuses.get(dep.shuffleId).orNull
701720 if (shuffleStatus != null ) {
702721 shuffleStatus.withMapStatuses { statuses =>
703- if (mapId >= 0 && mapId < statuses.length) {
704- Seq ( ExecutorCacheTaskLocation (statuses(mapId).location.host,
705- statuses(mapId).location.executorId).toString)
722+ if (startMapId < endMapId && (startMapId >= 0 && endMapId < statuses.length)) {
723+ val statusesPicked = statuses.slice(startMapId, endMapId).filter(_ != null )
724+ statusesPicked.map { status =>
725+ ExecutorCacheTaskLocation (status.location.host,
726+ status.location.executorId).toString
727+ }.toSeq
706728 } else {
707729 Nil
708730 }
@@ -767,6 +789,22 @@ private[spark] class MapOutputTrackerMaster(
767789 }
768790 }
769791
792+ override def getMapSizesByRangeMapIndex (
793+ shuffleId : Int ,
794+ partitionIndex : Int ,
795+ startMapId : Int ,
796+ endMapId : Int ): Iterator [(BlockManagerId , Seq [(BlockId , Long , Int )])] = {
797+ shuffleStatuses.get(shuffleId) match {
798+ case Some (shuffleStatus) =>
799+ shuffleStatus.withMapStatuses { statuses =>
800+ MapOutputTracker .convertMapStatuses(
801+ shuffleId, partitionIndex, statuses, startMapId, endMapId)
802+ }
803+ case None =>
804+ Iterator .empty
805+ }
806+ }
807+
770808 override def stop (): Unit = {
771809 mapOutputRequests.offer(PoisonPill )
772810 threadpool.shutdown()
@@ -831,6 +869,22 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr
831869 }
832870 }
833871
872+ override def getMapSizesByRangeMapIndex (
873+ shuffleId : Int ,
874+ partitionIndex : Int ,
875+ startMapId : Int ,
876+ endMapId : Int ): Iterator [(BlockManagerId , Seq [(BlockId , Long , Int )])] = {
877+ val statuses = getStatuses(shuffleId, conf)
878+ try {
879+ MapOutputTracker .convertMapStatuses(shuffleId, partitionIndex, statuses, startMapId, endMapId)
880+ } catch {
881+ case e : MetadataFetchFailedException =>
882+ // We experienced a fetch failure so our mapStatuses cache is outdated; clear it:
883+ mapStatuses.clear()
884+ throw e
885+ }
886+ }
887+
834888 /**
835889 * Get or fetch the array of MapStatuses for a given shuffle ID. NOTE: clients MUST synchronize
836890 * on this array when reading it, because on the driver, we may be changing it in place.
@@ -1013,4 +1067,50 @@ private[spark] object MapOutputTracker extends Logging {
10131067
10141068 splitsByAddress.iterator
10151069 }
1070+
1071+ /**
1072+ * Given an array of map statuses, a specific map output partitions and a range
1073+ * mappers (startMapId, endMapId),returns a sequence that, for each block manager ID,
1074+ * lists the shuffle block IDs and corresponding shuffle
1075+ * block sizes stored at that block manager.
1076+ * Note that empty blocks are filtered in the result.
1077+ *
1078+ * If any of the statuses is null (indicating a missing location due to a failed mapper),
1079+ * throws a FetchFailedException.
1080+ *
1081+ * @param shuffleId Identifier for the shuffle
1082+ * @param partitionIndex Specific of map output partition ID
1083+ * @param statuses List of map statuses, indexed by map partition index.
1084+ * @param startMapId Start Map ID
1085+ * @param endMapId End map ID
1086+ * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId,
1087+ * and the second item is a sequence of (shuffle block id, shuffle block size, map index)
1088+ * tuples describing the shuffle blocks that are stored at that block manager.
1089+ */
1090+ def convertMapStatuses (
1091+ shuffleId : Int ,
1092+ partitionIndex : Int ,
1093+ statuses : Array [MapStatus ],
1094+ startMapId : Int ,
1095+ endMapId : Int ): Iterator [(BlockManagerId , Seq [(BlockId , Long , Int )])] = {
1096+ assert (statuses != null )
1097+ val splitsByAddress = new HashMap [BlockManagerId , ListBuffer [(BlockId , Long , Int )]]
1098+ val iter = statuses.iterator.zipWithIndex
1099+ for ((status, mapIndex) <- iter.slice(startMapId, endMapId)) {
1100+ if (status == null ) {
1101+ val errorMessage = s " Missing an output location for shuffle $shuffleId"
1102+ logError(errorMessage)
1103+ throw new MetadataFetchFailedException (shuffleId, partitionIndex, errorMessage)
1104+ } else {
1105+ val size = status.getSizeForBlock(partitionIndex)
1106+ if (size != 0 ) {
1107+ splitsByAddress.getOrElseUpdate(status.location, ListBuffer ()) +=
1108+ ((ShuffleBlockId (shuffleId, status.mapId, partitionIndex), size, mapIndex))
1109+ }
1110+ }
1111+ }
1112+
1113+ splitsByAddress.iterator
1114+ }
1115+
10161116}
0 commit comments