You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-dev@hadoop.apache.org by "stack (JIRA)" <ji...@apache.org> on 2009/05/23 22:17:45 UTC

[jira] Created: (HADOOP-5904) Hung on hdfs: writeChunk, DFSClient.java:2126, DataStreamer socketWrite

Hung on hdfs: writeChunk, DFSClient.java:2126, DataStreamer socketWrite
-----------------------------------------------------------------------

                 Key: HADOOP-5904
                 URL: https://issues.apache.org/jira/browse/HADOOP-5904
             Project: Hadoop Core
          Issue Type: Bug
          Components: dfs
    Affects Versions: 0.19.1, 0.19.0, 0.18.3
            Reporter: stack


We've seen this hang rare enough but when it happens it locks up the application.  We've seen it at least in 0.18.x and 0.19.x (we don't have much experience with 0.20.x hdfs yet).

Here we're doing a sequencefile#append

{code}
"IPC Server handler 9 on 60020" daemon prio=10 tid=0x00007fef1c3f0400 nid=0x7470 waiting for monitor entry [0x0000000042d18000..0x0000000042d189f0]
   java.lang.Thread.State: BLOCKED (on object monitor)
	at org.apache.hadoop.dfs.DFSClient$DFSOutputStream.writeChunk(DFSClient.java:2486)
	- waiting to lock <0x00007fef38ecc138> (a java.util.LinkedList)
	- locked <0x00007fef38ecbdb8> (a org.apache.hadoop.dfs.DFSClient$DFSOutputStream)
	at org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunk(FSOutputSummer.java:155)
	at org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:132)
	- locked <0x00007fef38ecbdb8> (a org.apache.hadoop.dfs.DFSClient$DFSOutputStream)
	at org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:121)
	- locked <0x00007fef38ecbdb8> (a org.apache.hadoop.dfs.DFSClient$DFSOutputStream)
	at org.apache.hadoop.fs.FSOutputSummer.write1(FSOutputSummer.java:112)
	at org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:86)
	- locked <0x00007fef38ecbdb8> (a org.apache.hadoop.dfs.DFSClient$DFSOutputStream)
	at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:47)
	at java.io.DataOutputStream.write(DataOutputStream.java:107)
	- locked <0x00007fef38e09fc0> (a org.apache.hadoop.fs.FSDataOutputStream)
	at org.apache.hadoop.io.SequenceFile$Writer.append(SequenceFile.java:1016)
	- locked <0x00007fef38e09f30> (a org.apache.hadoop.io.SequenceFile$Writer)
	at org.apache.hadoop.io.SequenceFile$Writer.append(SequenceFile.java:980)
	- locked <0x00007fef38e09f30> (a org.apache.hadoop.io.SequenceFile$Writer)
	at org.apache.hadoop.hbase.regionserver.HLog.doWrite(HLog.java:461)
	at org.apache.hadoop.hbase.regionserver.HLog.append(HLog.java:421)
	- locked <0x00007fef29ad9588> (a java.lang.Integer)
	at org.apache.hadoop.hbase.regionserver.HRegion.update(HRegion.java:1676)
	at org.apache.hadoop.hbase.regionserver.HRegion.batchUpdate(HRegion.java:1439)
	at org.apache.hadoop.hbase.regionserver.HRegion.batchUpdate(HRegion.java:1378)
	at org.apache.hadoop.hbase.regionserver.HRegionServer.batchUpdates(HRegionServer.java:1184)
	at sun.reflect.GeneratedMethodAccessor27.invoke(Unknown Source)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:616)
	at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:622)
	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:888)
{code}

The DataStreamer that is supposed to servicing the above writeChunk is stuck here:

{code}
"DataStreamer for file /hbase/log_72.34.249.212_1225407466779_60020/hlog.dat.1227075571390 block blk_-7436808403424765554_553837" daemon prio=10 tid=0x0000000001c84c00 nid=0x7125 in Object.wait() [0x00000000409b3000..0x00000000409b3d70]
   java.lang.Thread.State: WAITING (on object monitor)
	at java.lang.Object.wait(Native Method)
	at java.lang.Object.wait(Object.java:502)
	at org.apache.hadoop.ipc.Client.call(Client.java:709)
	- locked <0x00007fef39520bb8> (a org.apache.hadoop.ipc.Client$Call)
	at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:216)
	at org.apache.hadoop.dfs.$Proxy4.getProtocolVersion(Unknown Source)
	at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:319)
	at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:306)
	at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:343)
	at org.apache.hadoop.ipc.RPC.waitForProxy(RPC.java:288)
	at org.apache.hadoop.dfs.DFSClient.createClientDatanodeProtocolProxy(DFSClient.java:139)
	at org.apache.hadoop.dfs.DFSClient$DFSOutputStream.processDatanodeError(DFSClient.java:2185)
	at org.apache.hadoop.dfs.DFSClient$DFSOutputStream.access$1400(DFSClient.java:1735)
	at org.apache.hadoop.dfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:1889)
	- locked <0x00007fef38ecc138> (a java.util.LinkedList)
{code}

The writeChunk is trying to synchronize on dataQueue.

DataQueue is held by DataStreamer#run which is down in processDatanodeError trying to recover a problem with a block.

Another example of the hang and some more detail can be found over in HBASE-667.

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


[jira] Commented: (HADOOP-5904) Hung on hdfs: writeChunk, DFSClient.java:2126, DataStreamer socketWrite

Posted by "Kan Zhang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-5904?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12712590#action_12712590 ] 

Kan Zhang commented on HADOOP-5904:
-----------------------------------

> this issue will be fixed in the trunk as part of HADOOP-5859.
I meant only the locking part (processDatanodeError will not hold the dataQueue lock). However, if DataStreamer#run is 
stuck in processDatanodeError forever, eventually dataQueue will be full and writeChunk will hang.

> Hung on hdfs: writeChunk, DFSClient.java:2126, DataStreamer socketWrite
> -----------------------------------------------------------------------
>
>                 Key: HADOOP-5904
>                 URL: https://issues.apache.org/jira/browse/HADOOP-5904
>             Project: Hadoop Core
>          Issue Type: Bug
>          Components: dfs
>    Affects Versions: 0.18.3, 0.19.0, 0.19.1
>            Reporter: stack
>
> We've seen this hang rare enough but when it happens it locks up the application.  We've seen it at least in 0.18.x and 0.19.x (we don't have much experience with 0.20.x hdfs yet).
> Here we're doing a sequencefile#append
> {code}
> "IPC Server handler 9 on 60020" daemon prio=10 tid=0x00007fef1c3f0400 nid=0x7470 waiting for monitor entry [0x0000000042d18000..0x0000000042d189f0]
>    java.lang.Thread.State: BLOCKED (on object monitor)
> 	at org.apache.hadoop.dfs.DFSClient$DFSOutputStream.writeChunk(DFSClient.java:2486)
> 	- waiting to lock <0x00007fef38ecc138> (a java.util.LinkedList)
> 	- locked <0x00007fef38ecbdb8> (a org.apache.hadoop.dfs.DFSClient$DFSOutputStream)
> 	at org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunk(FSOutputSummer.java:155)
> 	at org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:132)
> 	- locked <0x00007fef38ecbdb8> (a org.apache.hadoop.dfs.DFSClient$DFSOutputStream)
> 	at org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:121)
> 	- locked <0x00007fef38ecbdb8> (a org.apache.hadoop.dfs.DFSClient$DFSOutputStream)
> 	at org.apache.hadoop.fs.FSOutputSummer.write1(FSOutputSummer.java:112)
> 	at org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:86)
> 	- locked <0x00007fef38ecbdb8> (a org.apache.hadoop.dfs.DFSClient$DFSOutputStream)
> 	at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:47)
> 	at java.io.DataOutputStream.write(DataOutputStream.java:107)
> 	- locked <0x00007fef38e09fc0> (a org.apache.hadoop.fs.FSDataOutputStream)
> 	at org.apache.hadoop.io.SequenceFile$Writer.append(SequenceFile.java:1016)
> 	- locked <0x00007fef38e09f30> (a org.apache.hadoop.io.SequenceFile$Writer)
> 	at org.apache.hadoop.io.SequenceFile$Writer.append(SequenceFile.java:980)
> 	- locked <0x00007fef38e09f30> (a org.apache.hadoop.io.SequenceFile$Writer)
> 	at org.apache.hadoop.hbase.regionserver.HLog.doWrite(HLog.java:461)
> 	at org.apache.hadoop.hbase.regionserver.HLog.append(HLog.java:421)
> 	- locked <0x00007fef29ad9588> (a java.lang.Integer)
> 	at org.apache.hadoop.hbase.regionserver.HRegion.update(HRegion.java:1676)
> 	at org.apache.hadoop.hbase.regionserver.HRegion.batchUpdate(HRegion.java:1439)
> 	at org.apache.hadoop.hbase.regionserver.HRegion.batchUpdate(HRegion.java:1378)
> 	at org.apache.hadoop.hbase.regionserver.HRegionServer.batchUpdates(HRegionServer.java:1184)
> 	at sun.reflect.GeneratedMethodAccessor27.invoke(Unknown Source)
> 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> 	at java.lang.reflect.Method.invoke(Method.java:616)
> 	at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:622)
> 	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:888)
> {code}
> The DataStreamer that is supposed to servicing the above writeChunk is stuck here:
> {code}
> "DataStreamer for file /hbase/log_72.34.249.212_1225407466779_60020/hlog.dat.1227075571390 block blk_-7436808403424765554_553837" daemon prio=10 tid=0x0000000001c84c00 nid=0x7125 in Object.wait() [0x00000000409b3000..0x00000000409b3d70]
>    java.lang.Thread.State: WAITING (on object monitor)
> 	at java.lang.Object.wait(Native Method)
> 	at java.lang.Object.wait(Object.java:502)
> 	at org.apache.hadoop.ipc.Client.call(Client.java:709)
> 	- locked <0x00007fef39520bb8> (a org.apache.hadoop.ipc.Client$Call)
> 	at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:216)
> 	at org.apache.hadoop.dfs.$Proxy4.getProtocolVersion(Unknown Source)
> 	at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:319)
> 	at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:306)
> 	at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:343)
> 	at org.apache.hadoop.ipc.RPC.waitForProxy(RPC.java:288)
> 	at org.apache.hadoop.dfs.DFSClient.createClientDatanodeProtocolProxy(DFSClient.java:139)
> 	at org.apache.hadoop.dfs.DFSClient$DFSOutputStream.processDatanodeError(DFSClient.java:2185)
> 	at org.apache.hadoop.dfs.DFSClient$DFSOutputStream.access$1400(DFSClient.java:1735)
> 	at org.apache.hadoop.dfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:1889)
> 	- locked <0x00007fef38ecc138> (a java.util.LinkedList)
> {code}
> The writeChunk is trying to synchronize on dataQueue.
> DataQueue is held by DataStreamer#run which is down in processDatanodeError trying to recover a problem with a block.
> Another example of the hang and some more detail can be found over in HBASE-667.

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


[jira] Commented: (HADOOP-5904) Hung on hdfs: writeChunk, DFSClient.java:2126, DataStreamer socketWrite

Posted by "Kan Zhang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-5904?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12712462#action_12712462 ] 

Kan Zhang commented on HADOOP-5904:
-----------------------------------

this issue will be fixed in the trunk as part of HADOOP-5859.

> Hung on hdfs: writeChunk, DFSClient.java:2126, DataStreamer socketWrite
> -----------------------------------------------------------------------
>
>                 Key: HADOOP-5904
>                 URL: https://issues.apache.org/jira/browse/HADOOP-5904
>             Project: Hadoop Core
>          Issue Type: Bug
>          Components: dfs
>    Affects Versions: 0.18.3, 0.19.0, 0.19.1
>            Reporter: stack
>
> We've seen this hang rare enough but when it happens it locks up the application.  We've seen it at least in 0.18.x and 0.19.x (we don't have much experience with 0.20.x hdfs yet).
> Here we're doing a sequencefile#append
> {code}
> "IPC Server handler 9 on 60020" daemon prio=10 tid=0x00007fef1c3f0400 nid=0x7470 waiting for monitor entry [0x0000000042d18000..0x0000000042d189f0]
>    java.lang.Thread.State: BLOCKED (on object monitor)
> 	at org.apache.hadoop.dfs.DFSClient$DFSOutputStream.writeChunk(DFSClient.java:2486)
> 	- waiting to lock <0x00007fef38ecc138> (a java.util.LinkedList)
> 	- locked <0x00007fef38ecbdb8> (a org.apache.hadoop.dfs.DFSClient$DFSOutputStream)
> 	at org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunk(FSOutputSummer.java:155)
> 	at org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:132)
> 	- locked <0x00007fef38ecbdb8> (a org.apache.hadoop.dfs.DFSClient$DFSOutputStream)
> 	at org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:121)
> 	- locked <0x00007fef38ecbdb8> (a org.apache.hadoop.dfs.DFSClient$DFSOutputStream)
> 	at org.apache.hadoop.fs.FSOutputSummer.write1(FSOutputSummer.java:112)
> 	at org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:86)
> 	- locked <0x00007fef38ecbdb8> (a org.apache.hadoop.dfs.DFSClient$DFSOutputStream)
> 	at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:47)
> 	at java.io.DataOutputStream.write(DataOutputStream.java:107)
> 	- locked <0x00007fef38e09fc0> (a org.apache.hadoop.fs.FSDataOutputStream)
> 	at org.apache.hadoop.io.SequenceFile$Writer.append(SequenceFile.java:1016)
> 	- locked <0x00007fef38e09f30> (a org.apache.hadoop.io.SequenceFile$Writer)
> 	at org.apache.hadoop.io.SequenceFile$Writer.append(SequenceFile.java:980)
> 	- locked <0x00007fef38e09f30> (a org.apache.hadoop.io.SequenceFile$Writer)
> 	at org.apache.hadoop.hbase.regionserver.HLog.doWrite(HLog.java:461)
> 	at org.apache.hadoop.hbase.regionserver.HLog.append(HLog.java:421)
> 	- locked <0x00007fef29ad9588> (a java.lang.Integer)
> 	at org.apache.hadoop.hbase.regionserver.HRegion.update(HRegion.java:1676)
> 	at org.apache.hadoop.hbase.regionserver.HRegion.batchUpdate(HRegion.java:1439)
> 	at org.apache.hadoop.hbase.regionserver.HRegion.batchUpdate(HRegion.java:1378)
> 	at org.apache.hadoop.hbase.regionserver.HRegionServer.batchUpdates(HRegionServer.java:1184)
> 	at sun.reflect.GeneratedMethodAccessor27.invoke(Unknown Source)
> 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> 	at java.lang.reflect.Method.invoke(Method.java:616)
> 	at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:622)
> 	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:888)
> {code}
> The DataStreamer that is supposed to servicing the above writeChunk is stuck here:
> {code}
> "DataStreamer for file /hbase/log_72.34.249.212_1225407466779_60020/hlog.dat.1227075571390 block blk_-7436808403424765554_553837" daemon prio=10 tid=0x0000000001c84c00 nid=0x7125 in Object.wait() [0x00000000409b3000..0x00000000409b3d70]
>    java.lang.Thread.State: WAITING (on object monitor)
> 	at java.lang.Object.wait(Native Method)
> 	at java.lang.Object.wait(Object.java:502)
> 	at org.apache.hadoop.ipc.Client.call(Client.java:709)
> 	- locked <0x00007fef39520bb8> (a org.apache.hadoop.ipc.Client$Call)
> 	at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:216)
> 	at org.apache.hadoop.dfs.$Proxy4.getProtocolVersion(Unknown Source)
> 	at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:319)
> 	at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:306)
> 	at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:343)
> 	at org.apache.hadoop.ipc.RPC.waitForProxy(RPC.java:288)
> 	at org.apache.hadoop.dfs.DFSClient.createClientDatanodeProtocolProxy(DFSClient.java:139)
> 	at org.apache.hadoop.dfs.DFSClient$DFSOutputStream.processDatanodeError(DFSClient.java:2185)
> 	at org.apache.hadoop.dfs.DFSClient$DFSOutputStream.access$1400(DFSClient.java:1735)
> 	at org.apache.hadoop.dfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:1889)
> 	- locked <0x00007fef38ecc138> (a java.util.LinkedList)
> {code}
> The writeChunk is trying to synchronize on dataQueue.
> DataQueue is held by DataStreamer#run which is down in processDatanodeError trying to recover a problem with a block.
> Another example of the hang and some more detail can be found over in HBASE-667.

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