hadoop-mapreduce-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Nathan Roberts (JIRA)" <j...@apache.org>
Subject [jira] [Updated] (MAPREDUCE-4728) Interaction between oob heartbeats and damper can cause TT to heartbeat with zero delay
Date Wed, 17 Oct 2012 16:38:03 GMT

     [ https://issues.apache.org/jira/browse/MAPREDUCE-4728?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]

Nathan Roberts updated MAPREDUCE-4728:
--------------------------------------

    Attachment: MAPREDUCE-4728.patch

quick patch illustrating possible approach
                
> Interaction between oob heartbeats and damper can cause TT to heartbeat with zero delay
> ---------------------------------------------------------------------------------------
>
>                 Key: MAPREDUCE-4728
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-4728
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>    Affects Versions: 1.0.3
>            Reporter: Nathan Roberts
>         Attachments: MAPREDUCE-4728.patch
>
>
> When mapreduce.tasktracker.outofband.heartbeat is true and mapreduce.tasktracker.outofband.heartbeat.damper
is something largish (like the default of 1000000), the TT doesn't wait for tasks to finish
before heartbeating back to the JT. This causes excessive load on the JT which in-turn reduces
overall cluster performance.
> I believe the problem is that in the following block of code, when getHeartbeatInterval()
returns 0, we heartbeat back immediately BUT finishedCount does not get reset. It looks like
nothing ever gets us out of this situation so we basically heartbeat without ever sleeping.
>  
> {code}
>         // accelerate to account for multiple finished tasks up-front
>         long remaining =
>           (lastHeartbeat + getHeartbeatInterval(finishedCount.get())) - now;
>         while (remaining > 0) {
>           // sleeps for the wait time or
>           // until there are *enough* empty slots to schedule tasks
>           synchronized (finishedCount) {
>             finishedCount.wait(remaining);
>             // Recompute
>             now = System.currentTimeMillis();
>             remaining =
>               (lastHeartbeat + getHeartbeatInterval(finishedCount.get())) - now;
>             if (remaining <= 0) {
>               // Reset count
>               finishedCount.set(0);
>               break;
>             }
>           }
>         }
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Mime
View raw message