hbase-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "stack (JIRA)" <j...@apache.org>
Subject [jira] Created: (HBASE-634) hbase hungup on hdfs: appending to HLog. Won't shut down.
Date Wed, 21 May 2008 21:45:55 GMT
hbase hungup on hdfs: appending to HLog.  Won't shut down.
----------------------------------------------------------

                 Key: HBASE-634
                 URL: https://issues.apache.org/jira/browse/HBASE-634
             Project: Hadoop HBase
          Issue Type: Bug
            Reporter: stack
             Fix For: 0.2.0, 0.1.2


On our internal cluster, noticed hung regionserver.  Hang manifest itself in log as thousands
of lines of:

{code}
Call queue overflow discarding oldest call batchUpdate
{code}

Thread dumping, a bunch of threads are waiting to append to HLog:

{code}
     41 "IPC Server handler 8 on 60020" daemon prio=1 tid=0x00002aab40226770 nid=0x3890 waiting
for monitor entry [0x0000000042d7d000..0x0000000042d7db00]
     42         at org.apache.hadoop.hbase.HLog.append(HLog.java:370)
     43         - waiting to lock <0x00002aaab7815d38> (a java.lang.Integer)
     44         at org.apache.hadoop.hbase.HRegion.update(HRegion.java:1624)
     45         at org.apache.hadoop.hbase.HRegion.batchUpdate(HRegion.java:1427)
     46         at org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1554)
     47         at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source)
     48         at sun.reflect.DelegatingMethodAccessorImpl.invoke(Unknown Source)
     49         at java.lang.reflect.Method.invoke(Unknown Source)
     50         at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
     51         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:901)
{code}

... but they can't because another thread is stuck trying to write the HLog:

{code}
     16 "IPC Server handler 9 on 60020" daemon prio=1 tid=0x00002aab402278d0 nid=0x3891 in
Object.wait() [0x0000000042e7e000..0x0000000042e7eb80]
     17         at java.lang.Object.wait(Native Method)
     18         at java.lang.Object.wait(Unknown Source)
     19         at org.apache.hadoop.dfs.DFSClient$DFSOutputStream.writeChunk(DFSClient.java:2131)
     20         - locked <0x00002aaab7ee5038> (a java.util.LinkedList)
     21         at org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunk(FSOutputSummer.java:141)
     22         at org.apache.hadoop.fs.FSOutputSummer.write1(FSOutputSummer.java:100)
     23         at org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:86)
     24         - locked <0x00002aaab7ee4cb0> (a org.apache.hadoop.dfs.DFSClient$DFSOutputStream)
     25         at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:41)
     26         at java.io.DataOutputStream.write(Unknown Source)
     27         - locked <0x00002aaab7e73ea8> (a org.apache.hadoop.fs.FSDataOutputStream)
     28         at org.apache.hadoop.io.SequenceFile$Writer.append(SequenceFile.java:990)
     29         - locked <0x00002aaab7ee5200> (a org.apache.hadoop.io.SequenceFile$Writer)
     30         at org.apache.hadoop.hbase.HLog.append(HLog.java:387)
     31         - locked <0x00002aaab7815d38> (a java.lang.Integer)
     32         at org.apache.hadoop.hbase.HRegion.update(HRegion.java:1624)
     33         at org.apache.hadoop.hbase.HRegion.batchUpdate(HRegion.java:1427)
     34         at org.apache.hadoop.hbase.HRegionServer.batchUpdate(HRegionServer.java:1554)
     35         at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source)
     36         at sun.reflect.DelegatingMethodAccessorImpl.invoke(Unknown Source)
     37         at java.lang.reflect.Method.invoke(Unknown Source)
     38         at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:413)
     39         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:901)
{code}

Looking in code, the above sleep will be woken when we get response from datanode -- a response
that never comes in this case.  The Responder thread itself is stuck trying to read a long
from the datanode:

{code}
      3 "ResponseProcessor for block blk_3392187502501092232" daemon prio=1 tid=0x00002aab38cd8ba0
nid=0x7700 runnable [0x0000000043080000..0x0000000043080c80]
      4         at java.net.SocketInputStream.socketRead0(Native Method)
      5         at java.net.SocketInputStream.read(Unknown Source)
      6         at java.io.DataInputStream.readFully(Unknown Source)
      7         at java.io.DataInputStream.readLong(Unknown Source)
      8         at org.apache.hadoop.dfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:1734)
{code}

Related is the DFSClient DataStreamer, itself is in a sleep

{code}
     10 "DataStreamer for file /hbase/aa0-005-2.u.powerset.com/log_208.76.44.96_1211224091595_60020/hlog.dat.004"
daemon prio=1 tid=0x00002aab38a34920 nid=0x6e1b in Object.wait() [0x0000000043484000..0x0000000043484b00]
     11         at java.lang.Object.wait(Native Method)
     12         at java.lang.Object.wait(Unknown Source)
     13         at org.apache.hadoop.dfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:1656)
     14         - locked <0x00002aaab7ee5060> (a java.util.LinkedList)
{code}

The hang doesn't change after 5 or 6 thread dumps nor does it change though I shutdown the
regionserver.

Would need to figure why the datanode stopped responding, why we haven't timedout our read
at least.

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