Github user viper-kun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11886#discussion_r57102821
  
    --- Diff: core/src/main/scala/org/apache/spark/MapOutputTracker.scala ---
    @@ -443,13 +443,12 @@ private[spark] class MapOutputTrackerMaster(conf: 
SparkConf)
               statuses = mapStatuses.getOrElse(shuffleId, Array[MapStatus]())
               epochGotten = epoch
           }
    -    }
    -    // If we got here, we failed to find the serialized locations in the 
cache, so we pulled
    -    // out a snapshot of the locations as "statuses"; let's serialize and 
return that
    -    val bytes = MapOutputTracker.serializeMapStatuses(statuses)
    -    logInfo("Size of output statuses for shuffle %d is %d 
bytes".format(shuffleId, bytes.length))
    -    // Add them into the table only if the epoch hasn't changed while we 
were working
    -    epochLock.synchronized {
    +      
    +      // If we got here, we failed to find the serialized locations in the 
cache, so we pulled
    --- End diff --
    
    The problem description:
    Execute the query listed in 
jira(https://issues.apache.org/jira/browse/SPARK-14065), all tasks running on 
some executor are slow.
    
    Slow executor logs show that RPC(GetMapOutputStatuses) get  
RpcTimeoutException
    ```
    Error sending message [message = GetMapOutputStatuses(1)] in 1 attempts | 
org.apache.spark.Logging$class.logWarning(Logging.scala:92)
    org.apache.spark.rpc.RpcTimeoutException: Futures timed out after [120 
seconds]. This timeout is controlled by spark.network.timeout
    ```
    
    Driver logs shows that serialize mapstatus is slow
    ```
    16/03/22 11:47:07 INFO [dispatcher-event-loop-36] MapOutputTrackerMaster: 
Size of output statuses for shuffle 1 is 10298063 bytes
    16/03/22 11:47:14 INFO [dispatcher-event-loop-30] MapOutputTrackerMaster: 
Size of output statuses for shuffle 1 is 10298063 bytes
    16/03/22 11:47:21 INFO [dispatcher-event-loop-3] MapOutputTrackerMaster: 
Size of output statuses for shuffle 1 is 10298063 bytes
    16/03/22 11:47:27 INFO [dispatcher-event-loop-32] MapOutputTrackerMaster: 
Size of output statuses for shuffle 1 is 10298063 bytes
    16/03/22 11:47:34 INFO [dispatcher-event-loop-31] MapOutputTrackerMaster: 
Size of output statuses for shuffle 1 is 10298063 bytes
    16/03/22 11:47:41 INFO [dispatcher-event-loop-38] MapOutputTrackerMaster: 
Size of output statuses for shuffle 1 is 10298063 bytes
    16/03/22 11:47:47 INFO [dispatcher-event-loop-4] MapOutputTrackerMaster: 
Size of output statuses for shuffle 1 is 10298063 bytes
    16/03/22 11:47:54 INFO [dispatcher-event-loop-37] MapOutputTrackerMaster: 
Size of output statuses for shuffle 1 is 10298063 bytes
    16/03/22 11:48:00 INFO [dispatcher-event-loop-28] MapOutputTrackerMaster: 
Size of output statuses for shuffle 1 is 10298063 bytes
    ```
    
    When reduce task start, all executors will get mapstatus from 
driver.Because "MapOutputTracker.serializeMapStatuses(statuses)" is out of 
epochLock.synchronized {}
    ,all thread will do the operation - 
MapOutputTracker.serializeMapStatuses(statuses). In function 
serializeMapStatuses,  it has sync on statuses. So Serialize is one by one. 
    Every serialize cost 7 seconds. We have 80 executors, it total cost 560 
seconds. The result is  some executor get mapstatus timeout.
    This patch put "MapOutputTracker.serializeMapStatuses(statuses)" into 
epochLock.synchronized {}, it will increase probability of using cached 
serialized status.
    
    I have test  listed sql in 30T tpcds. The result shows it  faster than old. 
    
![image](https://cloud.githubusercontent.com/assets/6460155/13974069/9f448cc4-f0e3-11e5-9a5d-bc3cd8300db1.png)



---
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