Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/3345#issuecomment-68014353
  
    @markhamstra I noticed that there's a mailing list post (which never made 
it to the list, I think, because the author had not completed the mailing list 
registration and posted through Nabble): 
http://apache-spark-user-list.1001560.n3.nabble.com/Confirming-race-condition-in-DagScheduler-NoSuchElementException-td20691.html
    
    According to that post:
    
    > [...] looking at the DagScheduler code (even in master), it seems like 
the problem was due to getCacheLocs are called from two places: 
getPreferredLocation and getMissingParentStages. One of them is sync', while 
the other is not.
    
    I can confirm that this is the case.  It looks like `getPreferredLocs` was 
marked as `synchronized` because it's called from SparkContext:
    
    ```scala
      /**
       * Synchronized method that might be called from other threads.
       * @param rdd whose partitions are to be looked at
       * @param partition to lookup locality information for
       * @return list of machines that are preferred by the partition
       */
      private[spark]
      def getPreferredLocs(rdd: RDD[_], partition: Int): Seq[TaskLocation] = 
synchronized {
        getPreferredLocsInternal(rdd, partition, new HashSet)
      }
    ```
    
    However, it's not obvious why this is synchronized.  There are no other 
synchronized blocks or methods in `DAGScheduler`, so I think this 
synchronization is only serving to prevent multiple threads from being in 
`getPreferredLocs` calls at the same time, not to guard access to DAGScheduler 
state.
    
    So, I think that there could be a legitimate synchronization issue here, 
but I don't think that this PR is the right fix.  I think that a clean fix 
would be to replace this `synchronized` with a message send / reply to the 
DAGScheduler actor, which ensures that all accesses of the DAGScheduler state 
are processed through the actor.


---
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 [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to