[ 
https://issues.apache.org/jira/browse/HBASE-634?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12617199#action_12617199
 ] 

stack commented on HBASE-634:
-----------------------------

Here is hregionserver hung on close flushing -- this is 0.2.0 hbase and hadoop 
0.17 so a little different:

{code}
Thread 8 (regionserver/0:0:0:0:0:0:0:0:60020.cacheFlusher):
  State: WAITING
  Blocked count: 15537
  Waited count: 588
  Waiting on [EMAIL PROTECTED]
  Stack:
    java.lang.Object.wait(Native Method)
    java.lang.Object.wait(Object.java:485)
    
org.apache.hadoop.dfs.DFSClient$DFSOutputStream.flushInternal(DFSClient.java:2538)
    
org.apache.hadoop.dfs.DFSClient$DFSOutputStream.closeInternal(DFSClient.java:2631)
    org.apache.hadoop.dfs.DFSClient$DFSOutputStream.close(DFSClient.java:2576)
    
org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:59)
    org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:79)
    org.apache.hadoop.io.SequenceFile$Writer.close(SequenceFile.java:969)
    org.apache.hadoop.io.MapFile$Writer.close(MapFile.java:182)
    
org.apache.hadoop.hbase.regionserver.HStoreFile$BloomFilterMapFile$Writer.close(HStoreFile.java:848)
    
org.apache.hadoop.hbase.regionserver.HStore.internalFlushCache(HStore.java:626)
    org.apache.hadoop.hbase.regionserver.HStore.flushCache(HStore.java:567)
    
org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:1054)
    org.apache.hadoop.hbase.regionserver.HRegion.flushcache(HRegion.java:965)
    org.apache.hadoop.hbase.regionserver.Flusher.flushRegion(Flusher.java:173)
    org.apache.hadoop.hbase.regionserver.Flusher.run(Flusher.java:91)

{code}

> 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.1.2, 0.2.0
>
>
> 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.

Reply via email to