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

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

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.


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

Posted by "Todd Lipcon (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-1144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12771080#action_12771080 ] 

Todd Lipcon commented on MAPREDUCE-1144:
----------------------------------------

Sharad: would you support a patch for branch-20 that puts user log writing in a separate thread? It's a shame that a DFS stall can lock up all of mapreduce.

> JT should not hold lock while writing user history logs 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.


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

Posted by "Todd Lipcon (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-1144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12769694#action_12769694 ] 

Todd Lipcon commented on MAPREDUCE-1144:
----------------------------------------

Sharad: it actually looks like the code base this cluster is running already does include MAPREDUCE-814. I guess this means we potentially have a configuration issue where it's writing to HDFS to begin with rather than a local file before moving it. I'll circle back with more info when I have it.

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


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

Posted by "Todd Lipcon (JIRA)" <ji...@apache.org>.
     [ 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.


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

Posted by "Todd Lipcon (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-1144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12769229#action_12769229 ] 

Todd Lipcon commented on MAPREDUCE-1144:
----------------------------------------

Last things in the logs before it hung:

2009-10-23 01:56:55,216 INFO org.apache.hadoop.mapred.JobInProgress: Job job_200910191402_1443 has completed successfully.
2009-10-23 01:57:41,853 WARN org.apache.hadoop.hdfs.DFSClient: DFSOutputStream ResponseProcessor exception  for block blk_-2046762813438535112_105966370java.net.SocketTimeoutException: 69
000 millis timeout while waiting for channel to be ready for read. ch : java.nio.channels.SocketChannel[connected local=/10.100.50.253:40763 remote=/10.100.50.13:50010]
        at org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:164)
        at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:155)
        at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:128)
        at java.io.DataInputStream.readFully(DataInputStream.java:178)
        at java.io.DataInputStream.readLong(DataInputStream.java:399)
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:2398)

2009-10-23 01:57:41,854 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_-2046762813438535112_105966370 bad datanode[0] 10.100.50.13:50010
2009-10-23 01:57:41,854 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_-2046762813438535112_105966370 in pipeline 10.100.50.13:50010, 10.100.50.11:50010, 10.100.50.69
:50010: bad datanode 10.100.50.13:50010
2009-10-23 02:08:52,658 WARN org.apache.hadoop.hdfs.DFSClient: DFSOutputStream ResponseProcessor exception  for block blk_7310325538633196445_105957594java.net.SocketTimeoutException: 690
00 millis timeout while waiting for channel to be ready for read. ch : java.nio.channels.SocketChannel[connected local=/10.100.50.253:54118 remote=/10.100.50.71:50010]
        at org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:164)
        at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:155)
        at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:128)
        at java.io.DataInputStream.readFully(DataInputStream.java:178)
        at java.io.DataInputStream.readLong(DataInputStream.java:399)
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:2398)

2009-10-23 02:08:52,659 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_7310325538633196445_105957594 bad datanode[0] 10.100.50.71:50010
2009-10-23 02:08:52,659 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_7310325538633196445_105957594 in pipeline 10.100.50.71:50010, 10.100.50.61:50010, 10.100.50.28:
50010: bad datanode 10.100.50.71:50010


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


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

Posted by "Todd Lipcon (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-1144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12769686#action_12769686 ] 

Todd Lipcon commented on MAPREDUCE-1144:
----------------------------------------

Is there any chance this could be voted into 0.20.2? It's causing pretty bad stability issues on one cluster.

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


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

Posted by "Todd Lipcon (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/MAPREDUCE-1144?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Todd Lipcon updated MAPREDUCE-1144:
-----------------------------------

    Summary: JT should not hold lock while writing user history logs to DFS  (was: JT should not hold lock while writing history to DFS)

> JT should not hold lock while writing user history logs 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.


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

Posted by "Sharad Agarwal (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-1144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12769611#action_12769611 ] 

Sharad Agarwal commented on MAPREDUCE-1144:
-------------------------------------------

0.21 has a feature (MAPREDUCE-814) where completed job history logs from JT local disk are moved to HDFS in a separate thread. 

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


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

Posted by "Sharad Agarwal (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-1144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12770383#action_12770383 ] 

Sharad Agarwal commented on MAPREDUCE-1144:
-------------------------------------------

Since MAPREDUCE-814 adds the capability to have job logs in HDFS, there is not much utility in enabling the user logs. Users can directly access those from HDFS done folder location. Infact in 0.21, user log has been removed as part of job history format/API refactoring - MAPREDUCE-157

> JT should not hold lock while writing user history logs 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.


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

Posted by "Arun C Murthy (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/MAPREDUCE-1144?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Arun C Murthy resolved MAPREDUCE-1144.
--------------------------------------

    Resolution: Duplicate

Already fixed in MAPREDUCE-816.

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