You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-dev@hadoop.apache.org by "Todd Lipcon (JIRA)" <ji...@apache.org> on 2009/10/24 20:44:59 UTC

[jira] Reopened: (MAPREDUCE-1144) JT should not hold lock while writing history to DFS

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

Todd Lipcon reopened MAPREDUCE-1144:
------------------------------------


Upon further investigation, it's actually the user log history file (default hadoop.job.user.history.location value) that's blocked the JT. This does not appear to be fixed by MAPREDUCE-814, which was already applied on the cluster in question.

It seems to me that the actions on the user log directory should also be done in another thread to avoid the issue above. One cranky datanode can lock the jobtracker for hours at a time (the lack of timeouts seems to be a DFS bug)

> JT should not hold lock while writing history to DFS
> ----------------------------------------------------
>
>                 Key: MAPREDUCE-1144
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-1144
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>          Components: jobtracker
>    Affects Versions: 0.20.1
>            Reporter: Todd Lipcon
>
> I've seen behavior a few times now where the DFS is being slow for one reason or another, and the JT essentially locks up waiting on it while one thread tries for a long time to write history files out. The stack trace blocking everything is:
> Thread 210 (IPC Server handler 10 on 7277):
>   State: WAITING
>   Blocked count: 171424
>   Waited count: 1209604
>   Waiting on java.util.LinkedList@407dd154
>   Stack:
>     java.lang.Object.wait(Native Method)
>     java.lang.Object.wait(Object.java:485)
>     org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.flushInternal(DFSClient.java:3122)
>     org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.closeInternal(DFSClient.java:3202)
>     org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.close(DFSClient.java:3151)
>     org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:67)
>     org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106)
>     sun.nio.cs.StreamEncoder.implClose(StreamEncoder.java:301)
>     sun.nio.cs.StreamEncoder.close(StreamEncoder.java:130)
>     java.io.OutputStreamWriter.close(OutputStreamWriter.java:216)
>     java.io.BufferedWriter.close(BufferedWriter.java:248)
>     java.io.PrintWriter.close(PrintWriter.java:295)
>     org.apache.hadoop.mapred.JobHistory$JobInfo.logFinished(JobHistory.java:1349)
>     org.apache.hadoop.mapred.JobInProgress.jobComplete(JobInProgress.java:2167)
>     org.apache.hadoop.mapred.JobInProgress.completedTask(JobInProgress.java:2111)
>     org.apache.hadoop.mapred.JobInProgress.updateTaskStatus(JobInProgress.java:873)
>     org.apache.hadoop.mapred.JobTracker.updateTaskStatuses(JobTracker.java:3598)
>     org.apache.hadoop.mapred.JobTracker.processHeartbeat(JobTracker.java:2792)
>     org.apache.hadoop.mapred.JobTracker.heartbeat(JobTracker.java:2581)
>     sun.reflect.GeneratedMethodAccessor14.invoke(Unknown Source)
> We should try not to do external IO while holding the JT lock, and instead write the data to an in-memory buffer, drop the lock, and then write.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.