@@ -322,36 +322,22 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging
322322 // For testing
323323 def getMapSizesByExecutorId (shuffleId : Int , reduceId : Int )
324324 : Iterator [(BlockManagerId , Seq [(BlockId , Long , Int )])] = {
325- getMapSizesByExecutorId(shuffleId, reduceId, reduceId + 1 )
325+ getMapSizesByExecutorId(shuffleId, 0 , Int . MaxValue , reduceId, reduceId + 1 )
326326 }
327327
328328 /**
329329 * Called from executors to get the server URIs and output sizes for each shuffle block that
330330 * needs to be read from a given range of map output partitions (startPartition is included but
331- * endPartition is excluded from the range).
331+ * endPartition is excluded from the range) within a range of mappers (startMapIndex is included
332+ * but endMapIndex is excluded). If endMapIndex=Int.MaxValue, the actual endMapIndex will be
333+ * changed to the length of total map outputs.
332334 *
333335 * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId,
334336 * and the second item is a sequence of (shuffle block id, shuffle block size, map index)
335337 * tuples describing the shuffle blocks that are stored at that block manager.
338+ * Note that zero-sized blocks are excluded in the result.
336339 */
337340 def getMapSizesByExecutorId (
338- shuffleId : Int ,
339- startPartition : Int ,
340- endPartition : Int )
341- : Iterator [(BlockManagerId , Seq [(BlockId , Long , Int )])]
342-
343- /**
344- * Called from executors to get the server URIs and output sizes for each shuffle block that
345- * needs to be read from a given range of map output partitions (startPartition is included but
346- * endPartition is excluded from the range) and is produced by
347- * a range of mappers (startMapIndex, endMapIndex, startMapIndex is included and
348- * the endMapIndex is excluded).
349- *
350- * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId,
351- * and the second item is a sequence of (shuffle block id, shuffle block size, map index)
352- * tuples describing the shuffle blocks that are stored at that block manager.
353- */
354- def getMapSizesByRange (
355341 shuffleId : Int ,
356342 startMapIndex : Int ,
357343 endMapIndex : Int ,
@@ -734,38 +720,22 @@ private[spark] class MapOutputTrackerMaster(
734720 }
735721 }
736722
737- // Get blocks sizes by executor Id. Note that zero-sized blocks are excluded in the result.
738723 // This method is only called in local-mode.
739724 def getMapSizesByExecutorId (
740- shuffleId : Int ,
741- startPartition : Int ,
742- endPartition : Int )
743- : Iterator [(BlockManagerId , Seq [(BlockId , Long , Int )])] = {
744- logDebug(s " Fetching outputs for shuffle $shuffleId, partitions $startPartition- $endPartition" )
745- shuffleStatuses.get(shuffleId) match {
746- case Some (shuffleStatus) =>
747- shuffleStatus.withMapStatuses { statuses =>
748- MapOutputTracker .convertMapStatuses(
749- shuffleId, startPartition, endPartition, statuses, 0 , shuffleStatus.mapStatuses.length)
750- }
751- case None =>
752- Iterator .empty
753- }
754- }
755-
756- override def getMapSizesByRange (
757725 shuffleId : Int ,
758726 startMapIndex : Int ,
759727 endMapIndex : Int ,
760728 startPartition : Int ,
761729 endPartition : Int ): Iterator [(BlockManagerId , Seq [(BlockId , Long , Int )])] = {
762- logDebug(s " Fetching outputs for shuffle $shuffleId, mappers $startMapIndex- $endMapIndex" +
763- s " partitions $startPartition- $endPartition" )
730+ logDebug(s " Fetching outputs for shuffle $shuffleId" )
764731 shuffleStatuses.get(shuffleId) match {
765732 case Some (shuffleStatus) =>
766733 shuffleStatus.withMapStatuses { statuses =>
734+ val actualEndMapIndex = if (endMapIndex == Int .MaxValue ) statuses.length else endMapIndex
735+ logDebug(s " Convert map statuses for shuffle $shuffleId, " +
736+ s " mappers $startMapIndex- $actualEndMapIndex, partitions $startPartition- $endPartition" )
767737 MapOutputTracker .convertMapStatuses(
768- shuffleId, startPartition, endPartition, statuses, startMapIndex, endMapIndex )
738+ shuffleId, startPartition, endPartition, statuses, startMapIndex, actualEndMapIndex )
769739 }
770740 case None =>
771741 Iterator .empty
@@ -798,37 +768,20 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr
798768 */
799769 private val fetchingLock = new KeyLock [Int ]
800770
801- // Get blocks sizes by executor Id. Note that zero-sized blocks are excluded in the result.
802771 override def getMapSizesByExecutorId (
803- shuffleId : Int ,
804- startPartition : Int ,
805- endPartition : Int )
806- : Iterator [(BlockManagerId , Seq [(BlockId , Long , Int )])] = {
807- logDebug(s " Fetching outputs for shuffle $shuffleId, partitions $startPartition- $endPartition" )
808- val statuses = getStatuses(shuffleId, conf)
809- try {
810- MapOutputTracker .convertMapStatuses(
811- shuffleId, startPartition, endPartition, statuses, 0 , statuses.length)
812- } catch {
813- case e : MetadataFetchFailedException =>
814- // We experienced a fetch failure so our mapStatuses cache is outdated; clear it:
815- mapStatuses.clear()
816- throw e
817- }
818- }
819-
820- override def getMapSizesByRange (
821772 shuffleId : Int ,
822773 startMapIndex : Int ,
823774 endMapIndex : Int ,
824775 startPartition : Int ,
825776 endPartition : Int ): Iterator [(BlockManagerId , Seq [(BlockId , Long , Int )])] = {
826- logDebug(s " Fetching outputs for shuffle $shuffleId, mappers $startMapIndex- $endMapIndex" +
827- s " partitions $startPartition- $endPartition" )
777+ logDebug(s " Fetching outputs for shuffle $shuffleId" )
828778 val statuses = getStatuses(shuffleId, conf)
829779 try {
780+ val actualEndMapIndex = if (endMapIndex == Int .MaxValue ) statuses.length else endMapIndex
781+ logDebug(s " Convert map statuses for shuffle $shuffleId, " +
782+ s " mappers $startMapIndex- $actualEndMapIndex, partitions $startPartition- $endPartition" )
830783 MapOutputTracker .convertMapStatuses(
831- shuffleId, startPartition, endPartition, statuses, startMapIndex, endMapIndex )
784+ shuffleId, startPartition, endPartition, statuses, startMapIndex, actualEndMapIndex )
832785 } catch {
833786 case e : MetadataFetchFailedException =>
834787 // We experienced a fetch failure so our mapStatuses cache is outdated; clear it:
0 commit comments