You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hbase.apache.org by "stack (JIRA)" <ji...@apache.org> on 2008/11/20 22:04:44 UTC
[jira] Commented: (HBASE-667) Hung regionserver; hung on hdfs:
writeChunk, DFSClient.java:2126, DataStreamer socketWrite
[ https://issues.apache.org/jira/browse/HBASE-667?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12649486#action_12649486 ]
stack commented on HBASE-667:
-----------------------------
Saw this again on streamy cluster yesterday; hadoop 0.18.x and an hbase that is close to TRUNK. Two regionservers I think, one at least, stuck in same place. Wouldn't go down.
{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}
Here is the seemingly accompanying datastreamer for the probable log file:
{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}
Looks like above is stuck trying to set up proxy:
{code}
public static VersionedProtocol waitForProxy(Class protocol,
long clientVersion,
InetSocketAddress addr,
Configuration conf
) throws IOException {
while (true) {
try {
return getProxy(protocol, clientVersion, addr, conf);
} catch(ConnectException se) { // namenode has not been started
LOG.info("Server at " + addr + " not available yet, Zzzzz...");
} catch(SocketTimeoutException te) { // namenode is busy
LOG.info("Problem connecting to server: " + addr);
}
try {
Thread.sleep(1000);
} catch (InterruptedException ie) {
// IGNORE
}
}
}
{code}
Here is where first stack trace is hungup:
{code}
2466 // @see FSOutputSummer#writeChunk()
2467 @Override
2468 protected synchronized void writeChunk(byte[] b, int offset, int len, byte[] checksum)
2469 throws IOException {
2470 checkOpen();
2471 isClosed();
2472
2473 int cklen = checksum.length;
2474 int bytesPerChecksum = this.checksum.getBytesPerChecksum();
2475 if (len > bytesPerChecksum) {
2476 throw new IOException("writeChunk() buffer size is " + len +
2477 " is larger than supported bytesPerChecksum " +
2478 bytesPerChecksum);
2479 }
2480 if (checksum.length != this.checksum.getChecksumSize()) {
2481 throw new IOException("writeChunk() checksum size is supposed to be " +
2482 this.checksum.getChecksumSize() +
2483 " but found to be " + checksum.length);
2484 }
2485
2486 synchronized (dataQueue) {
{code}
...
trying to synchronize on dataQueue.
DataQueue is held by DataStreamer#run.
Its down in processDatanodeError trying to recover a problem with a block.
One thing to try would be to enable hadoop ipc logging next time we see this. I think its stuck trying to connect to a datanode that had just died. Logging would help here.
> Hung regionserver; hung on hdfs: writeChunk, DFSClient.java:2126, DataStreamer socketWrite
> ------------------------------------------------------------------------------------------
>
> Key: HBASE-667
> URL: https://issues.apache.org/jira/browse/HBASE-667
> Project: Hadoop HBase
> Issue Type: Bug
> Affects Versions: 0.1.2
> Reporter: stack
>
> Internally, came across a hung regionserver. Here is relevant excerpt from thread dump:
> {code}
> "ResponseProcessor for block blk_-6991279486194843565" daemon prio=1 tid=0x00002aab3ac13c50 nid=0x7ad7 runnable [0x0000000043080000..0x0000000043080d00]
> at java.net.SocketInputStream.socketRead0(Native Method)
> at java.net.SocketInputStream.read(Unknown Source)
> at java.io.DataInputStream.readFully(Unknown Source)
> at java.io.DataInputStream.readLong(Unknown Source)
> at org.apache.hadoop.dfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:1734)
> ...
> "IPC Server handler 3 on 60020" daemon prio=1 tid=0x00002aab3c05abf0 nid=0x6b61 waiting for monitor entry [0x0000000042878000..0x0000000042878d00]
> at org.apache.hadoop.hbase.HLog.append(HLog.java:371)
> - waiting to lock <0x00002aaab69d1180> (a java.lang.Integer)
> at org.apache.hadoop.hbase.HRegion.update(HRegion.java:1629)
> at org.apache.hadoop.hbase.HRegion.batchUpdate(HRegion.java:1432)
> at org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1552)
> at sun.reflect.GeneratedMethodAccessor9.invoke(Unknown Source)
> at sun.reflect.DelegatingMethodAccessorImpl.invoke(Unknown Source)
> at java.lang.reflect.Method.invoke(Unknown Source)
> at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:901)
> .....
> "IPC Server handler 1 on 60020" daemon prio=1 tid=0x00002aab3c3220a0 nid=0x6b5f waiting for monitor entry [0x0000000042676000..0x0000000042676c00]
> at org.apache.hadoop.dfs.DFSClient$DFSOutputStream.writeChunk(DFSClient.java:2126)
> - waiting to lock <0x00002aaab69d1a28> (a java.util.LinkedList)
> at org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunk(FSOutputSummer.java:141)
> at org.apache.hadoop.fs.FSOutputSummer.write1(FSOutputSummer.java:100)
> at org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:86)
> - locked <0x00002aaab69d15b0> (a org.apache.hadoop.dfs.DFSClient$DFSOutputStream)
> at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:41)
> at java.io.DataOutputStream.write(Unknown Source)
> - locked <0x00002aaab69d1228> (a org.apache.hadoop.fs.FSDataOutputStream)
> at org.apache.hadoop.io.SequenceFile$Writer.append(SequenceFile.java:990)
> - locked <0x00002aaab69d1050> (a org.apache.hadoop.io.SequenceFile$Writer)
> at org.apache.hadoop.hbase.HLog.append(HLog.java:387)
> - locked <0x00002aaab69d1180> (a java.lang.Integer)
> at org.apache.hadoop.hbase.HRegion.update(HRegion.java:1629)
> at org.apache.hadoop.hbase.HRegion.batchUpdate(HRegion.java:1432)
> at org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1552)
> at sun.reflect.GeneratedMethodAccessor9.invoke(Unknown Source)
> at sun.reflect.DelegatingMethodAccessorImpl.invoke(Unknown Source)
> at java.lang.reflect.Method.invoke(Unknown Source)
> at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:901)
> .....
> "DataStreamer for file /hbase/aa0-005-2.u.powerset.com/log_208.76.45.223_1212443824255_60020/hlog.dat.000 block blk_-6991279486194843565" daemon prio=1 tid=0x00002aab3c1b2e70 nid=0x6b50 runnable [0x0000000041969000..0x0000000041969c80]
> 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 <0x00002aaab7652b70> (a java.io.BufferedOutputStream)
> at java.io.DataOutputStream.write(Unknown Source)
> - locked <0x00002aaab7652288> (a java.io.DataOutputStream)
> at org.apache.hadoop.dfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:1631)
> - locked <0x00002aaab69d1a28> (a java.util.LinkedList)
> {code}
> I've seen this before. I saw this this morning where a pure hadoop client was hung in same way. This is hadoop 0.16.4. Seems like a pure hadoop prob.
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.