hadoop-common-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From André Martin (JIRA) <j...@apache.org>
Subject [jira] Commented: (HADOOP-3197) Deadlock in DFCClient
Date Sun, 06 Apr 2008 09:51:25 GMT

    [ https://issues.apache.org/jira/browse/HADOOP-3197?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12586115#action_12586115
] 

André Martin commented on HADOOP-3197:
--------------------------------------

Raghu,
right, there is no real circular dependency. However, all threads are waiting for the DataStreamer
that blocks all other threads since the java.net.SocketOutputStream.socketWrite0(Native Method)
seems to got stuck for no real reason. Other DFS Clients are able to write to the DFS cluster
without any delays etc. and all datanodes are active & alive.
I finally killed the application (after taking this ThreadDump) since there was no progress
at all for more than 12 hours. No Timeout or any other exception has been thrown.
AFAIK socketWrite is a blocking call. However, even when the reader on the other end is slow,
there should be at least some progress?!?

> Deadlock in DFCClient
> ---------------------
>
>                 Key: HADOOP-3197
>                 URL: https://issues.apache.org/jira/browse/HADOOP-3197
>             Project: Hadoop Core
>          Issue Type: Bug
>    Affects Versions: 0.16.1
>            Reporter: André Martin
>
> The DFS Client hangs - attached the thread dump - looks like a dead lock to me...
> {noformat}
> "ResponseProcessor for block blk_-7822837545361798562" prio=10 tid=0x00002aab993dcc00
nid=0x5241 waiting for monitor entry [0x000000004365e000..0x000000004365ecc0]
>    java.lang.Thread.State: BLOCKED (on object monitor)
> 	at org.apache.hadoop.dfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:1771)
> 	- waiting to lock <0x00002aaaecf2dd08> (a java.util.LinkedList)
> "DataStreamer for file /seDNS/mapred-out/18A59C65A91D44E5BA24785DF103D1781BB0137E.cache.new
block blk_-7822837545361798562" prio=10 tid=0x00002aab96a46000 nid=0x523f runnable [0x000000004345c000..0x000000004345cc40]
>    java.lang.Thread.State: RUNNABLE
> 	at java.net.SocketOutputStream.socketWrite0(Native Method)
> 	at java.net.SocketOutputStream.socketWrite(Unknown Source)
> 	at java.net.SocketOutputStream.write(Unknown Source)
> 	at java.io.BufferedOutputStream.write(Unknown Source)
> 	- locked <0x00002aaaecf2ec50> (a java.io.BufferedOutputStream)
> 	at java.io.DataOutputStream.write(Unknown Source)
> 	- locked <0x00002aaaecf2ec20> (a java.io.DataOutputStream)
> 	at org.apache.hadoop.dfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:1623)
> 	- locked <0x00002aaaecf2dd08> (a java.util.LinkedList)
> "BackupJobQueuesThread" prio=10 tid=0x00002aab94b94000 nid=0x7cb2 waiting for monitor
entry [0x000000004244c000..0x000000004244cd40]
>    java.lang.Thread.State: BLOCKED (on object monitor)
> 	at org.apache.hadoop.dfs.DFSClient$DFSOutputStream.writeChunk(DFSClient.java:2117)
> 	- waiting to lock <0x00002aaaecf2dd08> (a java.util.LinkedList)
> 	at org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunk(FSOutputSummer.java:141)
> 	at org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:124)
> 	- locked <0x00002aaaecf2e670> (a org.apache.hadoop.dfs.DFSClient$DFSOutputStream)
> 	at org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:58)
> 	- locked <0x00002aaaecf2e670> (a org.apache.hadoop.dfs.DFSClient$DFSOutputStream)
> 	at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:36)
> 	at java.io.DataOutputStream.writeBytes(Unknown Source)
> 	at sedns.serializer.file.FileSerializerServer.serializeJobQueuesAndCache(FileSerializerServer.java:723)
> 	- locked <0x00002aaab430fec8> (a java.util.Collections$SynchronizedSet)
> 	at sedns.pastry.application.ServerApp$BackupJobListThread.run(ServerApp.java:476)
> "org.apache.hadoop.dfs.DFSClient$LeaseChecker@3acafb56" daemon prio=10 tid=0x00002aab94bc7c00
nid=0x7ca7 waiting on condition [0x0000000041941000..0x0000000041941bc0]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)
> 	at java.lang.Thread.sleep(Native Method)
> 	at org.apache.hadoop.dfs.DFSClient$LeaseChecker.run(DFSClient.java:597)
> 	at java.lang.Thread.run(Unknown Source)
> {noformat}
>  

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


Mime
View raw message