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

    https://github.com/apache/spark/pull/6607#discussion_r31800833
  
    --- Diff: 
streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala
 ---
    @@ -308,12 +305,41 @@ class ReceiverTracker(ssc: StreamingContext, 
skipReceiverLaunch: Boolean = false
             supervisor.start()
             supervisor.awaitTermination()
           }
    +
           // Run the dummy Spark job to ensure that all slaves have registered.
           // This avoids all the receivers to be scheduled on the same node.
           if (!ssc.sparkContext.isLocal) {
             ssc.sparkContext.makeRDD(1 to 50, 50).map(x => (x, 
1)).reduceByKey(_ + _, 20).collect()
           }
     
    +      // Right now, we only honor preferences if all receivers have them
    +      val hasLocationPreferences = 
receivers.map(_.preferredLocation.isDefined).reduce(_ && _)
    +
    +      // If no location preferences are specified, set host location for 
each receiver
    +      // so as to distribute them evenly over executors in a round-robin 
fashion
    +      var roundRobin = false;
    +      if (!hasLocationPreferences && !ssc.sparkContext.isLocal) {
    +        val executors = getExecutors(ssc)
    +        if (!executors.isEmpty) {
    +          var i = 0;
    +          for (i <- 0 to (receivers.length - 1)) {
    +            receivers(i).host = Some(executors(i % executors.length))
    +          }
    +          roundRobin = true
    +        }
    +      }
    +
    +      val tempRDD =
    +        if (hasLocationPreferences) {
    --- End diff --
    
    Thanks for the review. PreferredLocation set for all receivers or none 
sounds like a good assumption (either user decides how to schedule or we do). 
Here is a slightly modified version of the algorithm: 
    
    0. Leave receiver class unmodified and use an external array for storing 
locations
    1. If location preferences are set (which should be for all receivers or 
none), goto 4
    2. If num_receivers > num_executors, distribute receivers among executors 
(as in current PR)
    3. If num_executors > num_receivers, distribute executors among receivers 
    4. makeRDD


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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

Reply via email to