Hadoop Map/Reduce
  1. Hadoop Map/Reduce
  2. MAPREDUCE-1144

JT should not hold lock while writing user history logs to DFS

    Details

    • Type: Bug Bug
    • Status: Resolved
    • Priority: Major Major
    • Resolution: Won't Fix
    • Affects Version/s: 0.20.1
    • Fix Version/s: None
    • Component/s: jobtracker
    • Labels:
      None

      Description

      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.

        Issue Links

          Activity

          Transition Time In Source Status Execution Times Last Executer Last Execution Date
          Open Open Resolved Resolved
          1d 4h 28m 1 Arun C Murthy 24/Oct/09 19:40
          Resolved Resolved Reopened Reopened
          4m 25s 1 Todd Lipcon 24/Oct/09 19:44
          Reopened Reopened Resolved Resolved
          1739d 22m 1 Allen Wittenauer 29/Jul/14 20:07
          Zheng Shao made changes -
          Assignee Zheng Shao [ zshao ]
          Zheng Shao made changes -
          Assignee Zheng Shao [ zshao ]
          Allen Wittenauer made changes -
          Status Reopened [ 4 ] Resolved [ 5 ]
          Resolution Won't Fix [ 2 ]
          zhaoyunjiong made changes -
          Link This issue relates to MAPREDUCE-5711 [ MAPREDUCE-5711 ]
          zhaoyunjiong made changes -
          Attachment MAPREDUCE-1144-branch-1.2.patch [ 12622316 ]
          Hide
          zhaoyunjiong added a comment -

          We have same problem too in our cluster.
          This patch put JobHistory log to a queue first. So JobTracker won't hung due to DataNode.
          The original TestJobHistory is enough, so I didn't add any new test case.
          JobHistory is already rewrite for trunk, no patch for trunk.

          Show
          zhaoyunjiong added a comment - We have same problem too in our cluster. This patch put JobHistory log to a queue first. So JobTracker won't hung due to DataNode. The original TestJobHistory is enough, so I didn't add any new test case. JobHistory is already rewrite for trunk, no patch for trunk.
          Hide
          Sadayuki Furuhashi added a comment -

          I think I got the same (or similar) problem. A thread locks JobTracker instance in a synchronized block in JobTracker.submitJob method.
          Here are stacktraces at the moment: https://gist.github.com/frsyuki/b26904890f889c2bc0c0#file-jt-stacktrace-4-L299 (hadoop-2.0.0-mr1-cdh4.2.0)
          Other threads including jetty's HTTP handler threads and all org.apache.hadoop.ipc.Server$Handler threads are blocked.

          Show
          Sadayuki Furuhashi added a comment - I think I got the same (or similar) problem. A thread locks JobTracker instance in a synchronized block in JobTracker.submitJob method. Here are stacktraces at the moment: https://gist.github.com/frsyuki/b26904890f889c2bc0c0#file-jt-stacktrace-4-L299 (hadoop-2.0.0-mr1-cdh4.2.0) Other threads including jetty's HTTP handler threads and all org.apache.hadoop.ipc.Server$Handler threads are blocked.
          Hide
          Karthik Kambatla (Inactive) added a comment -

          In this particular case of moving history related files, the simplest approach seems to be to modify the behavior in job-history management. For instance, #moveToDone() can be executed asynchronously using a thread-pool; in fact, the method notifies JT of the completion when it is done moving files.

          Should more such situations arise, we can always use a thread-pool in JT for calls to external entities - DFS in this case, unless it needs to be blocking for correctness. We can cross that bridge when we get there.

          Show
          Karthik Kambatla (Inactive) added a comment - In this particular case of moving history related files, the simplest approach seems to be to modify the behavior in job-history management. For instance, #moveToDone() can be executed asynchronously using a thread-pool; in fact, the method notifies JT of the completion when it is done moving files. Should more such situations arise, we can always use a thread-pool in JT for calls to external entities - DFS in this case, unless it needs to be blocking for correctness. We can cross that bridge when we get there.
          Hide
          Todd Lipcon added a comment -

          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.

          Show
          Todd Lipcon added a comment - 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.
          Hide
          Sharad Agarwal added a comment -

          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

          Show
          Sharad Agarwal added a comment - 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
          Todd Lipcon made changes -
          Summary JT should not hold lock while writing history to DFS JT should not hold lock while writing user history logs to DFS
          Todd Lipcon made changes -
          Resolution Duplicate [ 3 ]
          Status Resolved [ 5 ] Reopened [ 4 ]
          Hide
          Todd Lipcon added a comment -

          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)

          Show
          Todd Lipcon added a comment - 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)
          Arun C Murthy made changes -
          Field Original Value New Value
          Status Open [ 1 ] Resolved [ 5 ]
          Resolution Duplicate [ 3 ]
          Hide
          Arun C Murthy added a comment -

          Already fixed in MAPREDUCE-816.

          Show
          Arun C Murthy added a comment - Already fixed in MAPREDUCE-816 .
          Hide
          Todd Lipcon added a comment -

          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.

          Show
          Todd Lipcon added a comment - 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.
          Hide
          Todd Lipcon added a comment -

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

          Show
          Todd Lipcon added a comment - Is there any chance this could be voted into 0.20.2? It's causing pretty bad stability issues on one cluster.
          Hide
          Sharad Agarwal added a comment -

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

          Show
          Sharad Agarwal added a comment - 0.21 has a feature ( MAPREDUCE-814 ) where completed job history logs from JT local disk are moved to HDFS in a separate thread.
          Hide
          Todd Lipcon added a comment -

          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

          Show
          Todd Lipcon added a comment - 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
          Todd Lipcon created issue -

            People

            • Assignee:
              Unassigned
              Reporter:
              Todd Lipcon
            • Votes:
              0 Vote for this issue
              Watchers:
              14 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development