Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19763#discussion_r151801570
  
    --- Diff: core/src/main/scala/org/apache/spark/MapOutputTracker.scala ---
    @@ -473,16 +477,41 @@ private[spark] class MapOutputTrackerMaster(
       }
     
       /**
    +   * Try to equally divide Range(0, num) to divisor slices
    +   */
    +  def equallyDivide(num: Int, divisor: Int): Iterator[Seq[Int]] = {
    +    assert(divisor > 0, "Divisor should be positive")
    +    val (each, remain) = (num / divisor, num % divisor)
    +    val (smaller, bigger) = (0 until num).splitAt((divisor-remain) * each)
    +    if (each != 0) {
    +      smaller.grouped(each) ++ bigger.grouped(each + 1)
    +    } else {
    +      bigger.grouped(each + 1)
    +    }
    +  }
    +
    +  /**
        * Return statistics about all of the outputs for a given shuffle.
        */
       def getStatistics(dep: ShuffleDependency[_, _, _]): MapOutputStatistics 
= {
         shuffleStatuses(dep.shuffleId).withMapStatuses { statuses =>
           val totalSizes = new Array[Long](dep.partitioner.numPartitions)
    -      for (s <- statuses) {
    -        for (i <- 0 until totalSizes.length) {
    -          totalSizes(i) += s.getSizeForBlock(i)
    -        }
    +      val mapStatusSubmitTasks = ArrayBuffer[Future[_]]()
    +      var taskSlices = parallelism
    +
    +      equallyDivide(totalSizes.length, taskSlices).foreach {
    +        reduceIds =>
    +          mapStatusSubmitTasks += threadPoolMapStats.submit(
    +            new Runnable {
    +              override def run(): Unit = {
    +                for (s <- statuses; i <- reduceIds) {
    +                  totalSizes(i) += s.getSizeForBlock(i)
    +                }
    +              }
    +            }
    +          )
           }
    +      mapStatusSubmitTasks.foreach(_.get())
    --- End diff --
    
    Don't use `scala.concurrent.ExecutionContext.Implicits.global`. You need to 
create a thread pool.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to