You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hbase.apache.org by "Jean-Adrien (JIRA)" <ji...@apache.org> on 2009/02/24 13:22:02 UTC

[jira] Created: (HBASE-1214) HRegionServer fails to stop if hdfs client it tries to recover a block

HRegionServer fails to stop if hdfs client it tries to recover a block 
-----------------------------------------------------------------------

                 Key: HBASE-1214
                 URL: https://issues.apache.org/jira/browse/HBASE-1214
             Project: Hadoop HBase
          Issue Type: Bug
          Components: master, regionserver
    Affects Versions: 0.19.0
         Environment: 4 Node cluster with poor hardware: (DN+RS / DN+RS / RS / MA+NN)
1 Gb memory each 
Ubuntu linux
Java 6
            Reporter: Jean-Adrien


One of my region server falls in a long GC time that get it unresponsive during about 10 minutes.
As I can see in the log, it seems that its DFSClient component was sending a file to hdfs, the sending times out when it recovers from GC:

h5. Region server log after recovering from GC
{noformat}
2009-02-21 01:22:26,454 WARN org.apache.hadoop.hdfs.DFSClient: DFSOutputStream ResponseProcessor exception  for block blk_7545556036225037274_1820952java.io.IOException: Bad response 1 for block blk_7545556036225037274_1820952 from datanode 192.168.1.10:50010
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:2342)
{noformat}

h5. corresponding error in receiving datanode
{noformat}
2009-02-21 01:23:56,608 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Exception in receiveBlock for block blk_7545556036225037274_1820952 java.io.EOFException: while trying to read 65557 bytes
[...]
2009-02-21 01:23:59,076 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 0 for block blk_7545556036225037274_1820952 Interrupted.
2009-02-21 01:24:01,484 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 0 for block blk_7545556036225037274_1820952 terminating
2009-02-21 01:24:01,485 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: writeBlock blk_7545556036225037274_1820952 received exception java.io.EOFException: while trying to read 65557 bytes
{noformat}

Since region server misses its lease to report to the master, it has to close all its regions before recover, and reopens them when the master asks for.
>From this time, it tries to _recover_ this block, as seen in the regionserver log:

h5. Region server log in an endless loop to recover
{noformat}
009-02-21 01:22:29,327 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_7545556036225037274_1820952 bad datanode[1] 192.168.1.10:50010
2009-02-21 01:22:29,327 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_7545556036225037274_1820952 in pipeline 192.168.1.13:50010, 192.
168.1.10:50010: bad datanode 192.168.1.10:50010
2009-02-21 01:22:29,689 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_7545556036225037274_1820952 failed  because recovery from primary datanode 192.168.1.13:50010 failed 2 times. Will retry...
{noformat}

To this _recover_ request, all datanodes fail with this Exception

{noformat}
2009-02-21 01:24:18,650 INFO org.apache.hadoop.ipc.Server: IPC Server handler 1 on 50020, call recoverBlock(blk_7545556036225037274_1820952, false, [Lorg.apache.hadoop.hdfs.protocol.DatanodeInfo;@311c4f) from 192.168.1.13:56968: error: org.apache.hadoop.ipc.RemoteException: java.io.IOException: blk_7545556036225037274_1820952 is already commited, storedBlock == null.
        at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.nextGenerationStampForBlock(FSNamesystem.java:4536)
        at org.apache.hadoop.hdfs.server.namenode.NameNode.nextGenerationStamp(NameNode.java:402)
        at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)

org.apache.hadoop.ipc.RemoteException: java.io.IOException: blk_7545556036225037274_1820952 is already commited, storedBlock == null.
        at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.nextGenerationStampForBlock(FSNamesystem.java:4536)
        at org.apache.hadoop.hdfs.server.namenode.NameNode.nextGenerationStamp(NameNode.java:402)
        at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)

        at org.apache.hadoop.ipc.Client.call(Client.java:696)
        at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:216)
        at $Proxy4.nextGenerationStamp(Unknown Source)
        at org.apache.hadoop.hdfs.server.datanode.DataNode.syncBlock(DataNode.java:1466)
        at org.apache.hadoop.hdfs.server.datanode.DataNode.recoverBlock(DataNode.java:1440)
        at org.apache.hadoop.hdfs.server.datanode.DataNode.recoverBlock(DataNode.java:1506)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
{noformat}

The problem is that, in this case, the RegionServer and therefore the master fails to stop when I launch the stop-hbase script.
A kill (-TERM) on the region server launches the SIGTERM handler thread, but the loop which try to reach the block does not stop.
here is a thread dump of the region server at this time:

h5. Thread dump of the region server
{noformat}
Full thread dump Java HotSpot(TM) Server VM (10.0-b23 mixed mode):

"Thread-16" prio=10 tid=0x081c3c00 nid=0x6e81 in Object.wait() [0x8f74f000..0x8f74fec0]
   java.lang.Thread.State: WAITING (on object monitor)                                
        at java.lang.Object.wait(Native Method)                                        
        at java.lang.Thread.join(Thread.java:1143)                                    
        - locked <0x93d0d060> (a java.lang.Thread)                                    
        at java.lang.Thread.join(Thread.java:1196)                                    
        at org.apache.hadoop.hbase.util.Threads.shutdown(Threads.java:78)              
        at org.apache.hadoop.hbase.util.Threads.shutdown(Threads.java:66)              
        at org.apache.hadoop.hbase.regionserver.HRegionServer$ShutdownThread.run(HRegionServer.java:814)

"SIGTERM handler" daemon prio=10 tid=0x08dbcc00 nid=0x6e80 in Object.wait() [0x8edaf000..0x8edaff40]
   java.lang.Thread.State: WAITING (on object monitor)                                              
        at java.lang.Object.wait(Native Method)                                                    
        at java.lang.Thread.join(Thread.java:1143)                                                  
        - locked <0x93d0f3c0> (a org.apache.hadoop.hbase.regionserver.HRegionServer$ShutdownThread)
        at java.lang.Thread.join(Thread.java:1196)                                                  
        at java.lang.ApplicationShutdownHooks.run(ApplicationShutdownHooks.java:79)                
        at java.lang.Shutdown.runHooks(Shutdown.java:89)                                            
        at java.lang.Shutdown.sequence(Shutdown.java:133)                                          
        at java.lang.Shutdown.exit(Shutdown.java:178)                                              
        - locked <0xb0d3ed60> (a java.lang.Class for java.lang.Shutdown)                            
        at java.lang.Terminator$1.handle(Terminator.java:35)                                        
        at sun.misc.Signal$1.run(Signal.java:195)                                                  
        at java.lang.Thread.run(Thread.java:619)                                                    

"Attach Listener" daemon prio=10 tid=0x081e8800 nid=0x6e50 waiting on condition [0x00000000..0x00000000]
   java.lang.Thread.State: RUNNABLE                                                                    

"IPC Client (47) connection to /192.168.1.13:50020 from an unknown user" daemon prio=10 tid=0x8e97a400 nid=0x39d1 in Object.wait() [0x8ec81000..0x8ec81fc0]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                              
        at java.lang.Object.wait(Native Method)                                                                                                            
        at org.apache.hadoop.ipc.Client$Connection.waitForWork(Client.java:395)                                                                            
        - locked <0xa44a36a0> (a org.apache.hadoop.ipc.Client$Connection)                                                                                  
        at org.apache.hadoop.ipc.Client$Connection.run(Client.java:437)                                                                                    

"IPC Client (47) connection to /192.168.1.10:50020 from an unknown user" daemon prio=10 tid=0x8e5f1c00 nid=0x523f in Object.wait() [0x8ecd3000..0x8ecd3140]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                              
        at java.lang.Object.wait(Native Method)                                                                                                            
        at org.apache.hadoop.ipc.Client$Connection.waitForWork(Client.java:395)                                                                            
        - locked <0xa448ca50> (a org.apache.hadoop.ipc.Client$Connection)                                                                                  
        at org.apache.hadoop.ipc.Client$Connection.run(Client.java:437)                                                                                    

"BlockFSInputStreamReferenceQueueChecker" daemon prio=10 tid=0x085a9400 nid=0x8c8 waiting on condition [0x8ee6c000..0x8ee6cdc0]
   java.lang.Thread.State: WAITING (parking)                                                                                  
        at sun.misc.Unsafe.park(Native Method)                                                                                
        - parking to wait for  <0x93e76068> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)          
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                                  
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)  
        at java.util.concurrent.DelayQueue.take(DelayQueue.java:160)                                                          
        at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:582)        
        at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:575)        
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:946)                                        
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:906)                                    
        at java.lang.Thread.run(Thread.java:619)                                                                              

"IPC Server handler 9 on 60020" daemon prio=10 tid=0x0882e000 nid=0x889 waiting on condition [0x8eebd000..0x8eebdfc0]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 8 on 60020" daemon prio=10 tid=0x0882cc00 nid=0x888 waiting on condition [0x8ef0e000..0x8ef0f040]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 7 on 60020" daemon prio=10 tid=0x0882b800 nid=0x887 waiting on condition [0x8ef5f000..0x8ef5fec0]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 6 on 60020" daemon prio=10 tid=0x0882a400 nid=0x886 waiting on condition [0x8efb0000..0x8efb0f40]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 5 on 60020" daemon prio=10 tid=0x08829000 nid=0x885 waiting on condition [0x8f001000..0x8f001dc0]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 4 on 60020" daemon prio=10 tid=0x083bcc00 nid=0x884 waiting on condition [0x8f052000..0x8f052e40]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 3 on 60020" daemon prio=10 tid=0x083bb800 nid=0x883 waiting on condition [0x8f0a3000..0x8f0a40c0]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 2 on 60020" daemon prio=10 tid=0x083ba400 nid=0x882 waiting on condition [0x8f0f4000..0x8f0f5140]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 1 on 60020" daemon prio=10 tid=0x083b9400 nid=0x881 waiting on condition [0x8f145000..0x8f145fc0]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 0 on 60020" daemon prio=10 tid=0x083b8400 nid=0x880 waiting on condition [0x8f196000..0x8f197040]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server listener on 60020" daemon prio=10 tid=0x083bfc00 nid=0x87f runnable [0x8f1e7000..0x8f1e7ec0]
   java.lang.Thread.State: RUNNABLE                                                                    
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)                                        
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)                                
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)                            
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)                                
        - locked <0x93d0c090> (a sun.nio.ch.Util$1)                                                    
        - locked <0x93d0c120> (a java.util.Collections$UnmodifiableSet)                                
        - locked <0x93d0c0b0> (a sun.nio.ch.EPollSelectorImpl)                                          
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)                                        
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:84)                                        
        at org.apache.hadoop.hbase.ipc.HBaseServer$Listener.run(HBaseServer.java:299)                  

"IPC Server Responder" daemon prio=10 tid=0x083bec00 nid=0x87e runnable [0x8f238000..0x8f238f40]
   java.lang.Thread.State: RUNNABLE                                                            
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)                                
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)                        
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)                    
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)                        
        - locked <0x93d0c0f0> (a sun.nio.ch.Util$1)                                            
        - locked <0x93d0c100> (a java.util.Collections$UnmodifiableSet)                        
        - locked <0x93d0be30> (a sun.nio.ch.EPollSelectorImpl)                                  
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)                                
        at org.apache.hadoop.hbase.ipc.HBaseServer$Responder.run(HBaseServer.java:458)          

"SocketListener0-1" prio=10 tid=0x088d5000 nid=0x87c in Object.wait() [0x8f2da000..0x8f2dae40]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)                                  
        at java.lang.Object.wait(Native Method)                                              
        at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:522)                    
        - locked <0x93e39bc8> (a org.mortbay.util.ThreadPool$PoolThread)                      

"SocketListener0-0" prio=10 tid=0x08ab5000 nid=0x87b in Object.wait() [0x8f32b000..0x8f32c0c0]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)                                  
        at java.lang.Object.wait(Native Method)                                              
        at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:522)                    
        - locked <0x93e39920> (a org.mortbay.util.ThreadPool$PoolThread)                      

"Acceptor ServerSocket[addr=0.0.0.0/0.0.0.0,port=0,localport=60030]" prio=10 tid=0x084f6800 nid=0x87a runnable [0x8f37c000..0x8f37d140]
   java.lang.Thread.State: RUNNABLE                                                                                                    
        at java.net.PlainSocketImpl.socketAccept(Native Method)                                                                        
        at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:384)                                                                  
        - locked <0x93e382a8> (a java.net.SocksSocketImpl)                                                                            
        at java.net.ServerSocket.implAccept(ServerSocket.java:453)                                                                    
        at java.net.ServerSocket.accept(ServerSocket.java:421)                                                                        
        at org.mortbay.util.ThreadedServer.acceptSocket(ThreadedServer.java:432)                                                      
        at org.mortbay.util.ThreadedServer$Acceptor.run(ThreadedServer.java:631)                                                      

"SessionScavenger" daemon prio=10 tid=0x088f1400 nid=0x879 waiting on condition [0x8f3cd000..0x8f3cdfc0]
   java.lang.Thread.State: TIMED_WAITING (sleeping)                                                    
        at java.lang.Thread.sleep(Native Method)                                                        
        at org.mortbay.jetty.servlet.AbstractSessionManager$SessionScavenger.run(AbstractSessionManager.java:587)

"SessionScavenger" daemon prio=10 tid=0x08ab5c00 nid=0x878 waiting on condition [0x8f41e000..0x8f41f040]
   java.lang.Thread.State: TIMED_WAITING (sleeping)                                                    
        at java.lang.Thread.sleep(Native Method)                                                        
        at org.mortbay.jetty.servlet.AbstractSessionManager$SessionScavenger.run(AbstractSessionManager.java:587)

"regionserver/0.0.0.0:60020.leaseChecker" prio=10 tid=0x086d5800 nid=0x877 waiting on condition [0x8f476000..0x8f476ec0]
   java.lang.Thread.State: TIMED_WAITING (parking)                                                                      
        at sun.misc.Unsafe.park(Native Method)                                                                          
        - parking to wait for  <0x93d0e980> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)  
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:198)                                      
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:1963)
        at java.util.concurrent.DelayQueue.poll(DelayQueue.java:201)                                                            
        at org.apache.hadoop.hbase.Leases.run(Leases.java:78)                                                                    

"regionserver/0.0.0.0:60020.majorCompactionChecker" daemon prio=10 tid=0x086d4000 nid=0x876 waiting on condition [0x8f4c7000..0x8f4c7f40]
   java.lang.Thread.State: TIMED_WAITING (sleeping)                                                                                      
        at java.lang.Thread.sleep(Native Method)                                                                                        
        at org.apache.hadoop.hbase.util.Sleeper.sleep(Sleeper.java:74)                                                                  
        at org.apache.hadoop.hbase.Chore.run(Chore.java:72)                                                                              

"LeaseChecker" daemon prio=10 tid=0x0837cc00 nid=0x870 waiting on condition [0x8f6ad000..0x8f6ae040]
   java.lang.Thread.State: TIMED_WAITING (sleeping)                                                
        at java.lang.Thread.sleep(Native Method)                                                    
        at org.apache.hadoop.hdfs.DFSClient$LeaseChecker.run(DFSClient.java:979)                    
        at java.lang.Thread.run(Thread.java:619)                                                    

"DataStreamer for file /hbase/log_192.168.1.13_1235129194745_60020/hlog.dat.1235129195649 block blk_7545556036225037274_1820952" daemon prio=10 tid=0x08375400 nid=0x86f in Object.wait() [0x8f6fe000..0x8f6feec0]                                                                                                      
   java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                                
        at java.lang.Object.wait(Native Method)                                                                                                            
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2166)                                                          
        - locked <0x93d0ea10> (a java.util.LinkedList)                                                                                                      

"DestroyJavaVM" prio=10 tid=0x0805a000 nid=0x855 waiting on condition [0x00000000..0xb7dd2090]
   java.lang.Thread.State: RUNNABLE                                                          

"regionserver/0.0.0.0:60020" prio=10 tid=0x085b2400 nid=0x863 in Object.wait() [0x8f7f7000..0x8f7f7f40]
   java.lang.Thread.State: WAITING (on object monitor)                                                
        at java.lang.Object.wait(Native Method)                                                        
        at java.lang.Object.wait(Object.java:485)                                                      
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.flushInternal(DFSClient.java:3015)        
        - locked <0x93d0ea10> (a java.util.LinkedList)                                                
        - locked <0x93d0ec58> (a org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)                    
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.closeInternal(DFSClient.java:3104)        
        - locked <0x93d0ec58> (a org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)                    
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.close(DFSClient.java:3053)
        at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:59)
        at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:79)
        at org.apache.hadoop.io.SequenceFile$Writer.close(SequenceFile.java:959)
        - locked <0x93d50d20> (a org.apache.hadoop.io.SequenceFile$Writer)
        at org.apache.hadoop.hbase.regionserver.HLog.close(HLog.java:421)
        - locked <0x93d4ce00> (a java.lang.Integer)
        at org.apache.hadoop.hbase.regionserver.HLog.closeAndDelete(HLog.java:404)
        at org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:373)
        at java.lang.Thread.run(Thread.java:619)

"Low Memory Detector" daemon prio=10 tid=0x080dd800 nid=0x85e runnable [0x00000000..0x00000000]
   java.lang.Thread.State: RUNNABLE

"CompilerThread1" daemon prio=10 tid=0x080dc400 nid=0x85d waiting on condition [0x00000000..0x8fbc1588]
   java.lang.Thread.State: RUNNABLE

"CompilerThread0" daemon prio=10 tid=0x080d9c00 nid=0x85c waiting on condition [0x00000000..0x8fc42608]
   java.lang.Thread.State: RUNNABLE

"Signal Dispatcher" daemon prio=10 tid=0x080d8800 nid=0x85b runnable [0x00000000..0x8fc93e80]
   java.lang.Thread.State: RUNNABLE

"Surrogate Locker Thread (CMS)" daemon prio=10 tid=0x080d7800 nid=0x85a waiting on condition [0x00000000..0x8fce522c]
   java.lang.Thread.State: RUNNABLE

"Finalizer" daemon prio=10 tid=0x080bbc00 nid=0x859 in Object.wait() [0x8fd7b000..0x8fd7bec0]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116)
        - locked <0x93d0bdc0> (a java.lang.ref.ReferenceQueue$Lock)
        at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:132)
        at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159)

"Reference Handler" daemon prio=10 tid=0x080ba800 nid=0x858 in Object.wait() [0x8fdcc000..0x8fdccf40]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at java.lang.Object.wait(Object.java:485)
        at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)
        - locked <0x93d0c110> (a java.lang.ref.Reference$Lock)

"VM Thread" prio=10 tid=0x080b7800 nid=0x857 runnable

"Concurrent Mark-Sweep GC Thread" prio=10 tid=0x0806c800 nid=0x856 runnable
"VM Periodic Task Thread" prio=10 tid=0x080df000 nid=0x85f waiting on condition

JNI global references: 922
{noformat}

Notice the DataStreamer thread that seems to work on the concerned block
This force me to kill (-KILL) the region server, and restart the hdfs (if I don't restart hdfs some regions are not available because the block is still not available).

Note that the Hbase master does not end as well, and I have to kill (-KILL). Here is the thread dump of the HMaster process:

h5. Master thread dump:

{noformat}
Full thread dump Java HotSpot(TM) Server VM (10.0-b23 mixed mode):

"Attach Listener" daemon prio=10 tid=0x0805a000 nid=0x2326 waiting on condition [0x00000000..0x00000000]
   java.lang.Thread.State: RUNNABLE                                                                    

"SIGTERM handler" daemon prio=10 tid=0x089f0c00 nid=0x2303 waiting for monitor entry [0x8a6a5000..0x8a6a5ec0]
   java.lang.Thread.State: BLOCKED (on object monitor)                                                      
        at java.lang.Shutdown.exit(Shutdown.java:178)                                                        
        - waiting to lock <0xb0beed60> (a java.lang.Class for java.lang.Shutdown)                            
        at java.lang.Terminator$1.handle(Terminator.java:35)                                                
        at sun.misc.Signal$1.run(Signal.java:195)                                                            
        at java.lang.Thread.run(Thread.java:619)                                                            

"Thread-1" prio=10 tid=0x8f666000 nid=0x653c waiting on condition [0x8f1fe000..0x8f1fef40]
   java.lang.Thread.State: TIMED_WAITING (sleeping)                                      
        at java.lang.Thread.sleep(Native Method)                                          
        at org.apache.hadoop.ipc.Client.stop(Client.java:667)                            
        at org.apache.hadoop.ipc.RPC$ClientCache.stopClient(RPC.java:189)                
        at org.apache.hadoop.ipc.RPC$ClientCache.access$400(RPC.java:138)                
        at org.apache.hadoop.ipc.RPC$Invoker.close(RPC.java:229)                          
        - locked <0x938a1b18> (a org.apache.hadoop.ipc.RPC$Invoker)                      
        at org.apache.hadoop.ipc.RPC$Invoker.access$500(RPC.java:196)                    
        at org.apache.hadoop.ipc.RPC.stopProxy(RPC.java:353)                              
        at org.apache.hadoop.hdfs.DFSClient.close(DFSClient.java:213)                    
        - locked <0x93878470> (a org.apache.hadoop.hdfs.DFSClient)                        
        at org.apache.hadoop.hdfs.DistributedFileSystem.close(DistributedFileSystem.java:243)
        at org.apache.hadoop.fs.FileSystem$Cache.closeAll(FileSystem.java:1413)              
        - locked <0x937d3c00> (a org.apache.hadoop.fs.FileSystem$Cache)                      
        at org.apache.hadoop.fs.FileSystem.closeAll(FileSystem.java:236)                    
        at org.apache.hadoop.fs.FileSystem$ClientFinalizer.run(FileSystem.java:221)          
        - locked <0x937b8a78> (a org.apache.hadoop.fs.FileSystem$ClientFinalizer)            

"DestroyJavaVM" prio=10 tid=0x8f66c800 nid=0x36e9 in Object.wait() [0xb7d7d000..0xb7d7e0e0]
   java.lang.Thread.State: WAITING (on object monitor)                                    
        at java.lang.Object.wait(Native Method)                                            
        at java.lang.Thread.join(Thread.java:1143)                                        
        - locked <0x937b8a78> (a org.apache.hadoop.fs.FileSystem$ClientFinalizer)          
        at java.lang.Thread.join(Thread.java:1196)
        at java.lang.ApplicationShutdownHooks.run(ApplicationShutdownHooks.java:79)
        at java.lang.Shutdown.runHooks(Shutdown.java:89)
        at java.lang.Shutdown.sequence(Shutdown.java:133)
        at java.lang.Shutdown.shutdown(Shutdown.java:200)
        - locked <0xb0beed60> (a java.lang.Class for java.lang.Shutdown)

"Low Memory Detector" daemon prio=10 tid=0x8fe03800 nid=0x36f4 runnable [0x00000000..0x00000000]
   java.lang.Thread.State: RUNNABLE

"CompilerThread1" daemon prio=10 tid=0x8fe01c00 nid=0x36f3 waiting on condition [0x00000000..0x8f9ee488]
   java.lang.Thread.State: RUNNABLE

"CompilerThread0" daemon prio=10 tid=0x8fe00800 nid=0x36f2 waiting on condition [0x00000000..0x8fa6f508]
   java.lang.Thread.State: RUNNABLE

"Signal Dispatcher" daemon prio=10 tid=0x0813e800 nid=0x36f1 runnable [0x00000000..0x8fac0d80]
   java.lang.Thread.State: RUNNABLE

"Surrogate Locker Thread (CMS)" daemon prio=10 tid=0x0813d800 nid=0x36f0 waiting on condition [0x00000000..0x00000000]
   java.lang.Thread.State: RUNNABLE

"Finalizer" daemon prio=10 tid=0x08125000 nid=0x36ef in Object.wait() [0x8fba1000..0x8fba1dc0]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116)
        - locked <0x937bafd8> (a java.lang.ref.ReferenceQueue$Lock)
        at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:132)
        at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159)

"Reference Handler" daemon prio=10 tid=0x08120c00 nid=0x36ee in Object.wait() [0x8fbf2000..0x8fbf2e40]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at java.lang.Object.wait(Object.java:485)
        at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)
        - locked <0x937baf18> (a java.lang.ref.Reference$Lock)

"VM Thread" prio=10 tid=0x0811d800 nid=0x36ed runnable

"Gang worker#0 (Parallel GC Threads)" prio=10 tid=0x0805e000 nid=0x36ea runnable

"Gang worker#1 (Parallel GC Threads)" prio=10 tid=0x0805f000 nid=0x36eb runnable

"Concurrent Mark-Sweep GC Thread" prio=10 tid=0x080cd000 nid=0x36ec runnable
"VM Periodic Task Thread" prio=10 tid=0x8fe05000 nid=0x36f5 waiting on condition

JNI global references: 1044
{noformat}

When I restart dfs, the log shows me that the concerned block is marked as invalidate, and it is therefore deleted:

h5. Namenode log
{noformat}
2009-02-23 10:33:49,245 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.processReport: block blk_7545556036225037274_1820952 on 192.168.1.10:50010 size 27895296 does not
 belong to any file.
2009-02-23 10:33:49,245 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addToInvalidates: blk_7545556036225037274 is added to invalidSet of 192.168.1.10:50010
{noformat}

Finally, when I restart HBase, I can see some of these messages in region server log, and I have some data loss. (this log entry may not be linked with the  block traced before)

h5. Region server log:
{noformat}
2009-02-23 10:38:55,831 WARN org.apache.hadoop.hbase.regionserver.HStore: Exception processing reconstruction log hdfs://lab5-2:9000/hbase/test-D-0.3/510178748/oldlogfile.log opening [B@ff65bf -- continuing.  Probably lack-of-HADOOP-1700 causing DATA LOSS!
java.io.EOFException
        at java.io.DataInputStream.readFully(DataInputStream.java:180)
        at org.apache.hadoop.hbase.io.DataOutputBuffer$Buffer.write(DataOutputBuffer.java:70)
        at org.apache.hadoop.hbase.io.DataOutputBuffer.write(DataOutputBuffer.java:116)
        at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1944)
        at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1844)
        at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1890)
        at org.apache.hadoop.hbase.regionserver.HStore.doReconstructionLog(HStore.java:352)
        at org.apache.hadoop.hbase.regionserver.HStore.runReconstructionLog(HStore.java:297)
        at org.apache.hadoop.hbase.regionserver.HStore.<init>(HStore.java:237)
        at org.apache.hadoop.hbase.regionserver.HRegion.instantiateHStore(HRegion.java:1764)
        at org.apache.hadoop.hbase.regionserver.HRegion.initialize(HRegion.java:276)
        at org.apache.hadoop.hbase.regionserver.HRegionServer.instantiateRegion(HRegionServer.java:1367)
        at org.apache.hadoop.hbase.regionserver.HRegionServer.openRegion(HRegionServer.java:1338)
        at org.apache.hadoop.hbase.regionserver.HRegionServer$Worker.run(HRegionServer.java:1253)
        at java.lang.Thread.run(Thread.java:619)
{noformat}

In my sense, since my hardware is in cause for RegionServer lease miss, I have to deal with the fact that I have to stop/restart hbase + hadoop in such a case. The problem in my sense is that a stop request should be able to perform a clean shutdown, and to avoid dataloss.

Hope this description helps.


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


[jira] Commented: (HBASE-1214) HRegionServer fails to stop if hdfs client it tries to recover a block

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-1214?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12676529#action_12676529 ] 

stack commented on HBASE-1214:
------------------------------

Dave: I've seen DFSClient stuck.  IIRC (I can't find the issue just now), DFSClient keeps trying to go to a server it just marked bad -- it won't move on to the second and third datanodes for the wanted block though it has their addresses to hand.  Is that what you've seen?

> HRegionServer fails to stop if hdfs client it tries to recover a block 
> -----------------------------------------------------------------------
>
>                 Key: HBASE-1214
>                 URL: https://issues.apache.org/jira/browse/HBASE-1214
>             Project: Hadoop HBase
>          Issue Type: Bug
>          Components: master, regionserver
>    Affects Versions: 0.19.0
>         Environment: 4 Node cluster with poor hardware: (DN+RS / DN+RS / RS / MA+NN)
> 1 Gb memory each 
> Ubuntu linux
> Java 6
>            Reporter: Jean-Adrien
>
> One of my region server falls in a long GC time that get it unresponsive during about 10 minutes.
> As I can see in the log, it seems that its DFSClient component was sending a file to hdfs, the sending times out when it recovers from GC:
> h5. Region server log after recovering from GC
> {noformat}
> 2009-02-21 01:22:26,454 WARN org.apache.hadoop.hdfs.DFSClient: DFSOutputStream ResponseProcessor exception  for block blk_7545556036225037274_1820952java.io.IOException: Bad response 1 for block blk_7545556036225037274_1820952 from datanode 192.168.1.10:50010
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:2342)
> {noformat}
> h5. corresponding error in receiving datanode
> {noformat}
> 2009-02-21 01:23:56,608 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Exception in receiveBlock for block blk_7545556036225037274_1820952 java.io.EOFException: while trying to read 65557 bytes
> [...]
> 2009-02-21 01:23:59,076 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 0 for block blk_7545556036225037274_1820952 Interrupted.
> 2009-02-21 01:24:01,484 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 0 for block blk_7545556036225037274_1820952 terminating
> 2009-02-21 01:24:01,485 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: writeBlock blk_7545556036225037274_1820952 received exception java.io.EOFException: while trying to read 65557 bytes
> {noformat}
> Since region server misses its lease to report to the master, it has to close all its regions before recover, and reopens them when the master asks for.
> From this time, it tries to _recover_ this block, as seen in the regionserver log:
> h5. Region server log in an endless loop to recover
> {noformat}
> 009-02-21 01:22:29,327 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_7545556036225037274_1820952 bad datanode[1] 192.168.1.10:50010
> 2009-02-21 01:22:29,327 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_7545556036225037274_1820952 in pipeline 192.168.1.13:50010, 192.
> 168.1.10:50010: bad datanode 192.168.1.10:50010
> 2009-02-21 01:22:29,689 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_7545556036225037274_1820952 failed  because recovery from primary datanode 192.168.1.13:50010 failed 2 times. Will retry...
> {noformat}
> To this _recover_ request, all datanodes fail with this Exception
> {noformat}
> 2009-02-21 01:24:18,650 INFO org.apache.hadoop.ipc.Server: IPC Server handler 1 on 50020, call recoverBlock(blk_7545556036225037274_1820952, false, [Lorg.apache.hadoop.hdfs.protocol.DatanodeInfo;@311c4f) from 192.168.1.13:56968: error: org.apache.hadoop.ipc.RemoteException: java.io.IOException: blk_7545556036225037274_1820952 is already commited, storedBlock == null.
>         at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.nextGenerationStampForBlock(FSNamesystem.java:4536)
>         at org.apache.hadoop.hdfs.server.namenode.NameNode.nextGenerationStamp(NameNode.java:402)
>         at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
> org.apache.hadoop.ipc.RemoteException: java.io.IOException: blk_7545556036225037274_1820952 is already commited, storedBlock == null.
>         at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.nextGenerationStampForBlock(FSNamesystem.java:4536)
>         at org.apache.hadoop.hdfs.server.namenode.NameNode.nextGenerationStamp(NameNode.java:402)
>         at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
>         at org.apache.hadoop.ipc.Client.call(Client.java:696)
>         at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:216)
>         at $Proxy4.nextGenerationStamp(Unknown Source)
>         at org.apache.hadoop.hdfs.server.datanode.DataNode.syncBlock(DataNode.java:1466)
>         at org.apache.hadoop.hdfs.server.datanode.DataNode.recoverBlock(DataNode.java:1440)
>         at org.apache.hadoop.hdfs.server.datanode.DataNode.recoverBlock(DataNode.java:1506)
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
> {noformat}
> The problem is that, in this case, the RegionServer and therefore the master fails to stop when I launch the stop-hbase script.
> A kill (-TERM) on the region server launches the SIGTERM handler thread, but the loop which try to reach the block does not stop.
> here is a thread dump of the region server at this time:
> h5. Thread dump of the region server
> {noformat}
> Full thread dump Java HotSpot(TM) Server VM (10.0-b23 mixed mode):
> "Thread-16" prio=10 tid=0x081c3c00 nid=0x6e81 in Object.wait() [0x8f74f000..0x8f74fec0]
>    java.lang.Thread.State: WAITING (on object monitor)                                
>         at java.lang.Object.wait(Native Method)                                        
>         at java.lang.Thread.join(Thread.java:1143)                                    
>         - locked <0x93d0d060> (a java.lang.Thread)                                    
>         at java.lang.Thread.join(Thread.java:1196)                                    
>         at org.apache.hadoop.hbase.util.Threads.shutdown(Threads.java:78)              
>         at org.apache.hadoop.hbase.util.Threads.shutdown(Threads.java:66)              
>         at org.apache.hadoop.hbase.regionserver.HRegionServer$ShutdownThread.run(HRegionServer.java:814)
> "SIGTERM handler" daemon prio=10 tid=0x08dbcc00 nid=0x6e80 in Object.wait() [0x8edaf000..0x8edaff40]
>    java.lang.Thread.State: WAITING (on object monitor)                                              
>         at java.lang.Object.wait(Native Method)                                                    
>         at java.lang.Thread.join(Thread.java:1143)                                                  
>         - locked <0x93d0f3c0> (a org.apache.hadoop.hbase.regionserver.HRegionServer$ShutdownThread)
>         at java.lang.Thread.join(Thread.java:1196)                                                  
>         at java.lang.ApplicationShutdownHooks.run(ApplicationShutdownHooks.java:79)                
>         at java.lang.Shutdown.runHooks(Shutdown.java:89)                                            
>         at java.lang.Shutdown.sequence(Shutdown.java:133)                                          
>         at java.lang.Shutdown.exit(Shutdown.java:178)                                              
>         - locked <0xb0d3ed60> (a java.lang.Class for java.lang.Shutdown)                            
>         at java.lang.Terminator$1.handle(Terminator.java:35)                                        
>         at sun.misc.Signal$1.run(Signal.java:195)                                                  
>         at java.lang.Thread.run(Thread.java:619)                                                    
> "Attach Listener" daemon prio=10 tid=0x081e8800 nid=0x6e50 waiting on condition [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE                                                                    
> "IPC Client (47) connection to /192.168.1.13:50020 from an unknown user" daemon prio=10 tid=0x8e97a400 nid=0x39d1 in Object.wait() [0x8ec81000..0x8ec81fc0]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                              
>         at java.lang.Object.wait(Native Method)                                                                                                            
>         at org.apache.hadoop.ipc.Client$Connection.waitForWork(Client.java:395)                                                                            
>         - locked <0xa44a36a0> (a org.apache.hadoop.ipc.Client$Connection)                                                                                  
>         at org.apache.hadoop.ipc.Client$Connection.run(Client.java:437)                                                                                    
> "IPC Client (47) connection to /192.168.1.10:50020 from an unknown user" daemon prio=10 tid=0x8e5f1c00 nid=0x523f in Object.wait() [0x8ecd3000..0x8ecd3140]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                              
>         at java.lang.Object.wait(Native Method)                                                                                                            
>         at org.apache.hadoop.ipc.Client$Connection.waitForWork(Client.java:395)                                                                            
>         - locked <0xa448ca50> (a org.apache.hadoop.ipc.Client$Connection)                                                                                  
>         at org.apache.hadoop.ipc.Client$Connection.run(Client.java:437)                                                                                    
> "BlockFSInputStreamReferenceQueueChecker" daemon prio=10 tid=0x085a9400 nid=0x8c8 waiting on condition [0x8ee6c000..0x8ee6cdc0]
>    java.lang.Thread.State: WAITING (parking)                                                                                  
>         at sun.misc.Unsafe.park(Native Method)                                                                                
>         - parking to wait for  <0x93e76068> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)          
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                                  
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)  
>         at java.util.concurrent.DelayQueue.take(DelayQueue.java:160)                                                          
>         at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:582)        
>         at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:575)        
>         at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:946)                                        
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:906)                                    
>         at java.lang.Thread.run(Thread.java:619)                                                                              
> "IPC Server handler 9 on 60020" daemon prio=10 tid=0x0882e000 nid=0x889 waiting on condition [0x8eebd000..0x8eebdfc0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 8 on 60020" daemon prio=10 tid=0x0882cc00 nid=0x888 waiting on condition [0x8ef0e000..0x8ef0f040]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 7 on 60020" daemon prio=10 tid=0x0882b800 nid=0x887 waiting on condition [0x8ef5f000..0x8ef5fec0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 6 on 60020" daemon prio=10 tid=0x0882a400 nid=0x886 waiting on condition [0x8efb0000..0x8efb0f40]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 5 on 60020" daemon prio=10 tid=0x08829000 nid=0x885 waiting on condition [0x8f001000..0x8f001dc0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 4 on 60020" daemon prio=10 tid=0x083bcc00 nid=0x884 waiting on condition [0x8f052000..0x8f052e40]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 3 on 60020" daemon prio=10 tid=0x083bb800 nid=0x883 waiting on condition [0x8f0a3000..0x8f0a40c0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 2 on 60020" daemon prio=10 tid=0x083ba400 nid=0x882 waiting on condition [0x8f0f4000..0x8f0f5140]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 1 on 60020" daemon prio=10 tid=0x083b9400 nid=0x881 waiting on condition [0x8f145000..0x8f145fc0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 0 on 60020" daemon prio=10 tid=0x083b8400 nid=0x880 waiting on condition [0x8f196000..0x8f197040]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server listener on 60020" daemon prio=10 tid=0x083bfc00 nid=0x87f runnable [0x8f1e7000..0x8f1e7ec0]
>    java.lang.Thread.State: RUNNABLE                                                                    
>         at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)                                        
>         at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)                                
>         at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)                            
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)                                
>         - locked <0x93d0c090> (a sun.nio.ch.Util$1)                                                    
>         - locked <0x93d0c120> (a java.util.Collections$UnmodifiableSet)                                
>         - locked <0x93d0c0b0> (a sun.nio.ch.EPollSelectorImpl)                                          
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)                                        
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:84)                                        
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Listener.run(HBaseServer.java:299)                  
> "IPC Server Responder" daemon prio=10 tid=0x083bec00 nid=0x87e runnable [0x8f238000..0x8f238f40]
>    java.lang.Thread.State: RUNNABLE                                                            
>         at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)                                
>         at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)                        
>         at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)                    
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)                        
>         - locked <0x93d0c0f0> (a sun.nio.ch.Util$1)                                            
>         - locked <0x93d0c100> (a java.util.Collections$UnmodifiableSet)                        
>         - locked <0x93d0be30> (a sun.nio.ch.EPollSelectorImpl)                                  
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)                                
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Responder.run(HBaseServer.java:458)          
> "SocketListener0-1" prio=10 tid=0x088d5000 nid=0x87c in Object.wait() [0x8f2da000..0x8f2dae40]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                  
>         at java.lang.Object.wait(Native Method)                                              
>         at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:522)                    
>         - locked <0x93e39bc8> (a org.mortbay.util.ThreadPool$PoolThread)                      
> "SocketListener0-0" prio=10 tid=0x08ab5000 nid=0x87b in Object.wait() [0x8f32b000..0x8f32c0c0]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                  
>         at java.lang.Object.wait(Native Method)                                              
>         at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:522)                    
>         - locked <0x93e39920> (a org.mortbay.util.ThreadPool$PoolThread)                      
> "Acceptor ServerSocket[addr=0.0.0.0/0.0.0.0,port=0,localport=60030]" prio=10 tid=0x084f6800 nid=0x87a runnable [0x8f37c000..0x8f37d140]
>    java.lang.Thread.State: RUNNABLE                                                                                                    
>         at java.net.PlainSocketImpl.socketAccept(Native Method)                                                                        
>         at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:384)                                                                  
>         - locked <0x93e382a8> (a java.net.SocksSocketImpl)                                                                            
>         at java.net.ServerSocket.implAccept(ServerSocket.java:453)                                                                    
>         at java.net.ServerSocket.accept(ServerSocket.java:421)                                                                        
>         at org.mortbay.util.ThreadedServer.acceptSocket(ThreadedServer.java:432)                                                      
>         at org.mortbay.util.ThreadedServer$Acceptor.run(ThreadedServer.java:631)                                                      
> "SessionScavenger" daemon prio=10 tid=0x088f1400 nid=0x879 waiting on condition [0x8f3cd000..0x8f3cdfc0]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                    
>         at java.lang.Thread.sleep(Native Method)                                                        
>         at org.mortbay.jetty.servlet.AbstractSessionManager$SessionScavenger.run(AbstractSessionManager.java:587)
> "SessionScavenger" daemon prio=10 tid=0x08ab5c00 nid=0x878 waiting on condition [0x8f41e000..0x8f41f040]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                    
>         at java.lang.Thread.sleep(Native Method)                                                        
>         at org.mortbay.jetty.servlet.AbstractSessionManager$SessionScavenger.run(AbstractSessionManager.java:587)
> "regionserver/0.0.0.0:60020.leaseChecker" prio=10 tid=0x086d5800 nid=0x877 waiting on condition [0x8f476000..0x8f476ec0]
>    java.lang.Thread.State: TIMED_WAITING (parking)                                                                      
>         at sun.misc.Unsafe.park(Native Method)                                                                          
>         - parking to wait for  <0x93d0e980> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)  
>         at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:198)                                      
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:1963)
>         at java.util.concurrent.DelayQueue.poll(DelayQueue.java:201)                                                            
>         at org.apache.hadoop.hbase.Leases.run(Leases.java:78)                                                                    
> "regionserver/0.0.0.0:60020.majorCompactionChecker" daemon prio=10 tid=0x086d4000 nid=0x876 waiting on condition [0x8f4c7000..0x8f4c7f40]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                                                      
>         at java.lang.Thread.sleep(Native Method)                                                                                        
>         at org.apache.hadoop.hbase.util.Sleeper.sleep(Sleeper.java:74)                                                                  
>         at org.apache.hadoop.hbase.Chore.run(Chore.java:72)                                                                              
> "LeaseChecker" daemon prio=10 tid=0x0837cc00 nid=0x870 waiting on condition [0x8f6ad000..0x8f6ae040]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                
>         at java.lang.Thread.sleep(Native Method)                                                    
>         at org.apache.hadoop.hdfs.DFSClient$LeaseChecker.run(DFSClient.java:979)                    
>         at java.lang.Thread.run(Thread.java:619)                                                    
> "DataStreamer for file /hbase/log_192.168.1.13_1235129194745_60020/hlog.dat.1235129195649 block blk_7545556036225037274_1820952" daemon prio=10 tid=0x08375400 nid=0x86f in Object.wait() [0x8f6fe000..0x8f6feec0]                                                                                                      
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                                
>         at java.lang.Object.wait(Native Method)                                                                                                            
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2166)                                                          
>         - locked <0x93d0ea10> (a java.util.LinkedList)                                                                                                      
> "DestroyJavaVM" prio=10 tid=0x0805a000 nid=0x855 waiting on condition [0x00000000..0xb7dd2090]
>    java.lang.Thread.State: RUNNABLE                                                          
> "regionserver/0.0.0.0:60020" prio=10 tid=0x085b2400 nid=0x863 in Object.wait() [0x8f7f7000..0x8f7f7f40]
>    java.lang.Thread.State: WAITING (on object monitor)                                                
>         at java.lang.Object.wait(Native Method)                                                        
>         at java.lang.Object.wait(Object.java:485)                                                      
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.flushInternal(DFSClient.java:3015)        
>         - locked <0x93d0ea10> (a java.util.LinkedList)                                                
>         - locked <0x93d0ec58> (a org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)                    
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.closeInternal(DFSClient.java:3104)        
>         - locked <0x93d0ec58> (a org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)                    
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.close(DFSClient.java:3053)
>         at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:59)
>         at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:79)
>         at org.apache.hadoop.io.SequenceFile$Writer.close(SequenceFile.java:959)
>         - locked <0x93d50d20> (a org.apache.hadoop.io.SequenceFile$Writer)
>         at org.apache.hadoop.hbase.regionserver.HLog.close(HLog.java:421)
>         - locked <0x93d4ce00> (a java.lang.Integer)
>         at org.apache.hadoop.hbase.regionserver.HLog.closeAndDelete(HLog.java:404)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:373)
>         at java.lang.Thread.run(Thread.java:619)
> "Low Memory Detector" daemon prio=10 tid=0x080dd800 nid=0x85e runnable [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread1" daemon prio=10 tid=0x080dc400 nid=0x85d waiting on condition [0x00000000..0x8fbc1588]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread0" daemon prio=10 tid=0x080d9c00 nid=0x85c waiting on condition [0x00000000..0x8fc42608]
>    java.lang.Thread.State: RUNNABLE
> "Signal Dispatcher" daemon prio=10 tid=0x080d8800 nid=0x85b runnable [0x00000000..0x8fc93e80]
>    java.lang.Thread.State: RUNNABLE
> "Surrogate Locker Thread (CMS)" daemon prio=10 tid=0x080d7800 nid=0x85a waiting on condition [0x00000000..0x8fce522c]
>    java.lang.Thread.State: RUNNABLE
> "Finalizer" daemon prio=10 tid=0x080bbc00 nid=0x859 in Object.wait() [0x8fd7b000..0x8fd7bec0]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116)
>         - locked <0x93d0bdc0> (a java.lang.ref.ReferenceQueue$Lock)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:132)
>         at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159)
> "Reference Handler" daemon prio=10 tid=0x080ba800 nid=0x858 in Object.wait() [0x8fdcc000..0x8fdccf40]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.Object.wait(Object.java:485)
>         at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)
>         - locked <0x93d0c110> (a java.lang.ref.Reference$Lock)
> "VM Thread" prio=10 tid=0x080b7800 nid=0x857 runnable
> "Concurrent Mark-Sweep GC Thread" prio=10 tid=0x0806c800 nid=0x856 runnable
> "VM Periodic Task Thread" prio=10 tid=0x080df000 nid=0x85f waiting on condition
> JNI global references: 922
> {noformat}
> Notice the DataStreamer thread that seems to work on the concerned block
> This force me to kill (-KILL) the region server, and restart the hdfs (if I don't restart hdfs some regions are not available because the block is still not available).
> Note that the Hbase master does not end as well, and I have to kill (-KILL). Here is the thread dump of the HMaster process:
> h5. Master thread dump:
> {noformat}
> Full thread dump Java HotSpot(TM) Server VM (10.0-b23 mixed mode):
> "Attach Listener" daemon prio=10 tid=0x0805a000 nid=0x2326 waiting on condition [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE                                                                    
> "SIGTERM handler" daemon prio=10 tid=0x089f0c00 nid=0x2303 waiting for monitor entry [0x8a6a5000..0x8a6a5ec0]
>    java.lang.Thread.State: BLOCKED (on object monitor)                                                      
>         at java.lang.Shutdown.exit(Shutdown.java:178)                                                        
>         - waiting to lock <0xb0beed60> (a java.lang.Class for java.lang.Shutdown)                            
>         at java.lang.Terminator$1.handle(Terminator.java:35)                                                
>         at sun.misc.Signal$1.run(Signal.java:195)                                                            
>         at java.lang.Thread.run(Thread.java:619)                                                            
> "Thread-1" prio=10 tid=0x8f666000 nid=0x653c waiting on condition [0x8f1fe000..0x8f1fef40]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                      
>         at java.lang.Thread.sleep(Native Method)                                          
>         at org.apache.hadoop.ipc.Client.stop(Client.java:667)                            
>         at org.apache.hadoop.ipc.RPC$ClientCache.stopClient(RPC.java:189)                
>         at org.apache.hadoop.ipc.RPC$ClientCache.access$400(RPC.java:138)                
>         at org.apache.hadoop.ipc.RPC$Invoker.close(RPC.java:229)                          
>         - locked <0x938a1b18> (a org.apache.hadoop.ipc.RPC$Invoker)                      
>         at org.apache.hadoop.ipc.RPC$Invoker.access$500(RPC.java:196)                    
>         at org.apache.hadoop.ipc.RPC.stopProxy(RPC.java:353)                              
>         at org.apache.hadoop.hdfs.DFSClient.close(DFSClient.java:213)                    
>         - locked <0x93878470> (a org.apache.hadoop.hdfs.DFSClient)                        
>         at org.apache.hadoop.hdfs.DistributedFileSystem.close(DistributedFileSystem.java:243)
>         at org.apache.hadoop.fs.FileSystem$Cache.closeAll(FileSystem.java:1413)              
>         - locked <0x937d3c00> (a org.apache.hadoop.fs.FileSystem$Cache)                      
>         at org.apache.hadoop.fs.FileSystem.closeAll(FileSystem.java:236)                    
>         at org.apache.hadoop.fs.FileSystem$ClientFinalizer.run(FileSystem.java:221)          
>         - locked <0x937b8a78> (a org.apache.hadoop.fs.FileSystem$ClientFinalizer)            
> "DestroyJavaVM" prio=10 tid=0x8f66c800 nid=0x36e9 in Object.wait() [0xb7d7d000..0xb7d7e0e0]
>    java.lang.Thread.State: WAITING (on object monitor)                                    
>         at java.lang.Object.wait(Native Method)                                            
>         at java.lang.Thread.join(Thread.java:1143)                                        
>         - locked <0x937b8a78> (a org.apache.hadoop.fs.FileSystem$ClientFinalizer)          
>         at java.lang.Thread.join(Thread.java:1196)
>         at java.lang.ApplicationShutdownHooks.run(ApplicationShutdownHooks.java:79)
>         at java.lang.Shutdown.runHooks(Shutdown.java:89)
>         at java.lang.Shutdown.sequence(Shutdown.java:133)
>         at java.lang.Shutdown.shutdown(Shutdown.java:200)
>         - locked <0xb0beed60> (a java.lang.Class for java.lang.Shutdown)
> "Low Memory Detector" daemon prio=10 tid=0x8fe03800 nid=0x36f4 runnable [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread1" daemon prio=10 tid=0x8fe01c00 nid=0x36f3 waiting on condition [0x00000000..0x8f9ee488]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread0" daemon prio=10 tid=0x8fe00800 nid=0x36f2 waiting on condition [0x00000000..0x8fa6f508]
>    java.lang.Thread.State: RUNNABLE
> "Signal Dispatcher" daemon prio=10 tid=0x0813e800 nid=0x36f1 runnable [0x00000000..0x8fac0d80]
>    java.lang.Thread.State: RUNNABLE
> "Surrogate Locker Thread (CMS)" daemon prio=10 tid=0x0813d800 nid=0x36f0 waiting on condition [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE
> "Finalizer" daemon prio=10 tid=0x08125000 nid=0x36ef in Object.wait() [0x8fba1000..0x8fba1dc0]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116)
>         - locked <0x937bafd8> (a java.lang.ref.ReferenceQueue$Lock)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:132)
>         at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159)
> "Reference Handler" daemon prio=10 tid=0x08120c00 nid=0x36ee in Object.wait() [0x8fbf2000..0x8fbf2e40]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.Object.wait(Object.java:485)
>         at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)
>         - locked <0x937baf18> (a java.lang.ref.Reference$Lock)
> "VM Thread" prio=10 tid=0x0811d800 nid=0x36ed runnable
> "Gang worker#0 (Parallel GC Threads)" prio=10 tid=0x0805e000 nid=0x36ea runnable
> "Gang worker#1 (Parallel GC Threads)" prio=10 tid=0x0805f000 nid=0x36eb runnable
> "Concurrent Mark-Sweep GC Thread" prio=10 tid=0x080cd000 nid=0x36ec runnable
> "VM Periodic Task Thread" prio=10 tid=0x8fe05000 nid=0x36f5 waiting on condition
> JNI global references: 1044
> {noformat}
> When I restart dfs, the log shows me that the concerned block is marked as invalidate, and it is therefore deleted:
> h5. Namenode log
> {noformat}
> 2009-02-23 10:33:49,245 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.processReport: block blk_7545556036225037274_1820952 on 192.168.1.10:50010 size 27895296 does not
>  belong to any file.
> 2009-02-23 10:33:49,245 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addToInvalidates: blk_7545556036225037274 is added to invalidSet of 192.168.1.10:50010
> {noformat}
> Finally, when I restart HBase, I can see some of these messages in region server log, and I have some data loss. (this log entry may not be linked with the  block traced before)
> h5. Region server log:
> {noformat}
> 2009-02-23 10:38:55,831 WARN org.apache.hadoop.hbase.regionserver.HStore: Exception processing reconstruction log hdfs://lab5-2:9000/hbase/test-D-0.3/510178748/oldlogfile.log opening [B@ff65bf -- continuing.  Probably lack-of-HADOOP-1700 causing DATA LOSS!
> java.io.EOFException
>         at java.io.DataInputStream.readFully(DataInputStream.java:180)
>         at org.apache.hadoop.hbase.io.DataOutputBuffer$Buffer.write(DataOutputBuffer.java:70)
>         at org.apache.hadoop.hbase.io.DataOutputBuffer.write(DataOutputBuffer.java:116)
>         at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1944)
>         at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1844)
>         at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1890)
>         at org.apache.hadoop.hbase.regionserver.HStore.doReconstructionLog(HStore.java:352)
>         at org.apache.hadoop.hbase.regionserver.HStore.runReconstructionLog(HStore.java:297)
>         at org.apache.hadoop.hbase.regionserver.HStore.<init>(HStore.java:237)
>         at org.apache.hadoop.hbase.regionserver.HRegion.instantiateHStore(HRegion.java:1764)
>         at org.apache.hadoop.hbase.regionserver.HRegion.initialize(HRegion.java:276)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.instantiateRegion(HRegionServer.java:1367)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.openRegion(HRegionServer.java:1338)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer$Worker.run(HRegionServer.java:1253)
>         at java.lang.Thread.run(Thread.java:619)
> {noformat}
> In my sense, since my hardware is in cause for RegionServer lease miss, I have to deal with the fact that I have to stop/restart hbase + hadoop in such a case. The problem in my sense is that a stop request should be able to perform a clean shutdown, and to avoid dataloss.
> Hope this description helps.

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


[jira] Updated: (HBASE-1214) HRegionServer fails to stop if hdfs client it tries to recover a block

Posted by "stack (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-1214?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

stack updated HBASE-1214:
-------------------------

    Comment: was deleted

> HRegionServer fails to stop if hdfs client it tries to recover a block 
> -----------------------------------------------------------------------
>
>                 Key: HBASE-1214
>                 URL: https://issues.apache.org/jira/browse/HBASE-1214
>             Project: Hadoop HBase
>          Issue Type: Bug
>          Components: master, regionserver
>    Affects Versions: 0.19.0
>         Environment: 4 Node cluster with poor hardware: (DN+RS / DN+RS / RS / MA+NN)
> 1 Gb memory each 
> Ubuntu linux
> Java 6
>            Reporter: Jean-Adrien
>
> One of my region server falls in a long GC time that get it unresponsive during about 10 minutes.
> As I can see in the log, it seems that its DFSClient component was sending a file to hdfs, the sending times out when it recovers from GC:
> h5. Region server log after recovering from GC
> {noformat}
> 2009-02-21 01:22:26,454 WARN org.apache.hadoop.hdfs.DFSClient: DFSOutputStream ResponseProcessor exception  for block blk_7545556036225037274_1820952java.io.IOException: Bad response 1 for block blk_7545556036225037274_1820952 from datanode 192.168.1.10:50010
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:2342)
> {noformat}
> h5. corresponding error in receiving datanode
> {noformat}
> 2009-02-21 01:23:56,608 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Exception in receiveBlock for block blk_7545556036225037274_1820952 java.io.EOFException: while trying to read 65557 bytes
> [...]
> 2009-02-21 01:23:59,076 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 0 for block blk_7545556036225037274_1820952 Interrupted.
> 2009-02-21 01:24:01,484 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 0 for block blk_7545556036225037274_1820952 terminating
> 2009-02-21 01:24:01,485 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: writeBlock blk_7545556036225037274_1820952 received exception java.io.EOFException: while trying to read 65557 bytes
> {noformat}
> Since region server misses its lease to report to the master, it has to close all its regions before recover, and reopens them when the master asks for.
> From this time, it tries to _recover_ this block, as seen in the regionserver log:
> h5. Region server log in an endless loop to recover
> {noformat}
> 009-02-21 01:22:29,327 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_7545556036225037274_1820952 bad datanode[1] 192.168.1.10:50010
> 2009-02-21 01:22:29,327 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_7545556036225037274_1820952 in pipeline 192.168.1.13:50010, 192.
> 168.1.10:50010: bad datanode 192.168.1.10:50010
> 2009-02-21 01:22:29,689 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_7545556036225037274_1820952 failed  because recovery from primary datanode 192.168.1.13:50010 failed 2 times. Will retry...
> {noformat}
> To this _recover_ request, all datanodes fail with this Exception
> {noformat}
> 2009-02-21 01:24:18,650 INFO org.apache.hadoop.ipc.Server: IPC Server handler 1 on 50020, call recoverBlock(blk_7545556036225037274_1820952, false, [Lorg.apache.hadoop.hdfs.protocol.DatanodeInfo;@311c4f) from 192.168.1.13:56968: error: org.apache.hadoop.ipc.RemoteException: java.io.IOException: blk_7545556036225037274_1820952 is already commited, storedBlock == null.
>         at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.nextGenerationStampForBlock(FSNamesystem.java:4536)
>         at org.apache.hadoop.hdfs.server.namenode.NameNode.nextGenerationStamp(NameNode.java:402)
>         at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
> org.apache.hadoop.ipc.RemoteException: java.io.IOException: blk_7545556036225037274_1820952 is already commited, storedBlock == null.
>         at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.nextGenerationStampForBlock(FSNamesystem.java:4536)
>         at org.apache.hadoop.hdfs.server.namenode.NameNode.nextGenerationStamp(NameNode.java:402)
>         at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
>         at org.apache.hadoop.ipc.Client.call(Client.java:696)
>         at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:216)
>         at $Proxy4.nextGenerationStamp(Unknown Source)
>         at org.apache.hadoop.hdfs.server.datanode.DataNode.syncBlock(DataNode.java:1466)
>         at org.apache.hadoop.hdfs.server.datanode.DataNode.recoverBlock(DataNode.java:1440)
>         at org.apache.hadoop.hdfs.server.datanode.DataNode.recoverBlock(DataNode.java:1506)
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
> {noformat}
> The problem is that, in this case, the RegionServer and therefore the master fails to stop when I launch the stop-hbase script.
> A kill (-TERM) on the region server launches the SIGTERM handler thread, but the loop which try to reach the block does not stop.
> here is a thread dump of the region server at this time:
> h5. Thread dump of the region server
> {noformat}
> Full thread dump Java HotSpot(TM) Server VM (10.0-b23 mixed mode):
> "Thread-16" prio=10 tid=0x081c3c00 nid=0x6e81 in Object.wait() [0x8f74f000..0x8f74fec0]
>    java.lang.Thread.State: WAITING (on object monitor)                                
>         at java.lang.Object.wait(Native Method)                                        
>         at java.lang.Thread.join(Thread.java:1143)                                    
>         - locked <0x93d0d060> (a java.lang.Thread)                                    
>         at java.lang.Thread.join(Thread.java:1196)                                    
>         at org.apache.hadoop.hbase.util.Threads.shutdown(Threads.java:78)              
>         at org.apache.hadoop.hbase.util.Threads.shutdown(Threads.java:66)              
>         at org.apache.hadoop.hbase.regionserver.HRegionServer$ShutdownThread.run(HRegionServer.java:814)
> "SIGTERM handler" daemon prio=10 tid=0x08dbcc00 nid=0x6e80 in Object.wait() [0x8edaf000..0x8edaff40]
>    java.lang.Thread.State: WAITING (on object monitor)                                              
>         at java.lang.Object.wait(Native Method)                                                    
>         at java.lang.Thread.join(Thread.java:1143)                                                  
>         - locked <0x93d0f3c0> (a org.apache.hadoop.hbase.regionserver.HRegionServer$ShutdownThread)
>         at java.lang.Thread.join(Thread.java:1196)                                                  
>         at java.lang.ApplicationShutdownHooks.run(ApplicationShutdownHooks.java:79)                
>         at java.lang.Shutdown.runHooks(Shutdown.java:89)                                            
>         at java.lang.Shutdown.sequence(Shutdown.java:133)                                          
>         at java.lang.Shutdown.exit(Shutdown.java:178)                                              
>         - locked <0xb0d3ed60> (a java.lang.Class for java.lang.Shutdown)                            
>         at java.lang.Terminator$1.handle(Terminator.java:35)                                        
>         at sun.misc.Signal$1.run(Signal.java:195)                                                  
>         at java.lang.Thread.run(Thread.java:619)                                                    
> "Attach Listener" daemon prio=10 tid=0x081e8800 nid=0x6e50 waiting on condition [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE                                                                    
> "IPC Client (47) connection to /192.168.1.13:50020 from an unknown user" daemon prio=10 tid=0x8e97a400 nid=0x39d1 in Object.wait() [0x8ec81000..0x8ec81fc0]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                              
>         at java.lang.Object.wait(Native Method)                                                                                                            
>         at org.apache.hadoop.ipc.Client$Connection.waitForWork(Client.java:395)                                                                            
>         - locked <0xa44a36a0> (a org.apache.hadoop.ipc.Client$Connection)                                                                                  
>         at org.apache.hadoop.ipc.Client$Connection.run(Client.java:437)                                                                                    
> "IPC Client (47) connection to /192.168.1.10:50020 from an unknown user" daemon prio=10 tid=0x8e5f1c00 nid=0x523f in Object.wait() [0x8ecd3000..0x8ecd3140]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                              
>         at java.lang.Object.wait(Native Method)                                                                                                            
>         at org.apache.hadoop.ipc.Client$Connection.waitForWork(Client.java:395)                                                                            
>         - locked <0xa448ca50> (a org.apache.hadoop.ipc.Client$Connection)                                                                                  
>         at org.apache.hadoop.ipc.Client$Connection.run(Client.java:437)                                                                                    
> "BlockFSInputStreamReferenceQueueChecker" daemon prio=10 tid=0x085a9400 nid=0x8c8 waiting on condition [0x8ee6c000..0x8ee6cdc0]
>    java.lang.Thread.State: WAITING (parking)                                                                                  
>         at sun.misc.Unsafe.park(Native Method)                                                                                
>         - parking to wait for  <0x93e76068> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)          
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                                  
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)  
>         at java.util.concurrent.DelayQueue.take(DelayQueue.java:160)                                                          
>         at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:582)        
>         at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:575)        
>         at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:946)                                        
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:906)                                    
>         at java.lang.Thread.run(Thread.java:619)                                                                              
> "IPC Server handler 9 on 60020" daemon prio=10 tid=0x0882e000 nid=0x889 waiting on condition [0x8eebd000..0x8eebdfc0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 8 on 60020" daemon prio=10 tid=0x0882cc00 nid=0x888 waiting on condition [0x8ef0e000..0x8ef0f040]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 7 on 60020" daemon prio=10 tid=0x0882b800 nid=0x887 waiting on condition [0x8ef5f000..0x8ef5fec0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 6 on 60020" daemon prio=10 tid=0x0882a400 nid=0x886 waiting on condition [0x8efb0000..0x8efb0f40]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 5 on 60020" daemon prio=10 tid=0x08829000 nid=0x885 waiting on condition [0x8f001000..0x8f001dc0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 4 on 60020" daemon prio=10 tid=0x083bcc00 nid=0x884 waiting on condition [0x8f052000..0x8f052e40]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 3 on 60020" daemon prio=10 tid=0x083bb800 nid=0x883 waiting on condition [0x8f0a3000..0x8f0a40c0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 2 on 60020" daemon prio=10 tid=0x083ba400 nid=0x882 waiting on condition [0x8f0f4000..0x8f0f5140]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 1 on 60020" daemon prio=10 tid=0x083b9400 nid=0x881 waiting on condition [0x8f145000..0x8f145fc0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 0 on 60020" daemon prio=10 tid=0x083b8400 nid=0x880 waiting on condition [0x8f196000..0x8f197040]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server listener on 60020" daemon prio=10 tid=0x083bfc00 nid=0x87f runnable [0x8f1e7000..0x8f1e7ec0]
>    java.lang.Thread.State: RUNNABLE                                                                    
>         at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)                                        
>         at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)                                
>         at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)                            
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)                                
>         - locked <0x93d0c090> (a sun.nio.ch.Util$1)                                                    
>         - locked <0x93d0c120> (a java.util.Collections$UnmodifiableSet)                                
>         - locked <0x93d0c0b0> (a sun.nio.ch.EPollSelectorImpl)                                          
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)                                        
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:84)                                        
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Listener.run(HBaseServer.java:299)                  
> "IPC Server Responder" daemon prio=10 tid=0x083bec00 nid=0x87e runnable [0x8f238000..0x8f238f40]
>    java.lang.Thread.State: RUNNABLE                                                            
>         at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)                                
>         at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)                        
>         at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)                    
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)                        
>         - locked <0x93d0c0f0> (a sun.nio.ch.Util$1)                                            
>         - locked <0x93d0c100> (a java.util.Collections$UnmodifiableSet)                        
>         - locked <0x93d0be30> (a sun.nio.ch.EPollSelectorImpl)                                  
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)                                
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Responder.run(HBaseServer.java:458)          
> "SocketListener0-1" prio=10 tid=0x088d5000 nid=0x87c in Object.wait() [0x8f2da000..0x8f2dae40]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                  
>         at java.lang.Object.wait(Native Method)                                              
>         at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:522)                    
>         - locked <0x93e39bc8> (a org.mortbay.util.ThreadPool$PoolThread)                      
> "SocketListener0-0" prio=10 tid=0x08ab5000 nid=0x87b in Object.wait() [0x8f32b000..0x8f32c0c0]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                  
>         at java.lang.Object.wait(Native Method)                                              
>         at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:522)                    
>         - locked <0x93e39920> (a org.mortbay.util.ThreadPool$PoolThread)                      
> "Acceptor ServerSocket[addr=0.0.0.0/0.0.0.0,port=0,localport=60030]" prio=10 tid=0x084f6800 nid=0x87a runnable [0x8f37c000..0x8f37d140]
>    java.lang.Thread.State: RUNNABLE                                                                                                    
>         at java.net.PlainSocketImpl.socketAccept(Native Method)                                                                        
>         at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:384)                                                                  
>         - locked <0x93e382a8> (a java.net.SocksSocketImpl)                                                                            
>         at java.net.ServerSocket.implAccept(ServerSocket.java:453)                                                                    
>         at java.net.ServerSocket.accept(ServerSocket.java:421)                                                                        
>         at org.mortbay.util.ThreadedServer.acceptSocket(ThreadedServer.java:432)                                                      
>         at org.mortbay.util.ThreadedServer$Acceptor.run(ThreadedServer.java:631)                                                      
> "SessionScavenger" daemon prio=10 tid=0x088f1400 nid=0x879 waiting on condition [0x8f3cd000..0x8f3cdfc0]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                    
>         at java.lang.Thread.sleep(Native Method)                                                        
>         at org.mortbay.jetty.servlet.AbstractSessionManager$SessionScavenger.run(AbstractSessionManager.java:587)
> "SessionScavenger" daemon prio=10 tid=0x08ab5c00 nid=0x878 waiting on condition [0x8f41e000..0x8f41f040]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                    
>         at java.lang.Thread.sleep(Native Method)                                                        
>         at org.mortbay.jetty.servlet.AbstractSessionManager$SessionScavenger.run(AbstractSessionManager.java:587)
> "regionserver/0.0.0.0:60020.leaseChecker" prio=10 tid=0x086d5800 nid=0x877 waiting on condition [0x8f476000..0x8f476ec0]
>    java.lang.Thread.State: TIMED_WAITING (parking)                                                                      
>         at sun.misc.Unsafe.park(Native Method)                                                                          
>         - parking to wait for  <0x93d0e980> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)  
>         at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:198)                                      
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:1963)
>         at java.util.concurrent.DelayQueue.poll(DelayQueue.java:201)                                                            
>         at org.apache.hadoop.hbase.Leases.run(Leases.java:78)                                                                    
> "regionserver/0.0.0.0:60020.majorCompactionChecker" daemon prio=10 tid=0x086d4000 nid=0x876 waiting on condition [0x8f4c7000..0x8f4c7f40]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                                                      
>         at java.lang.Thread.sleep(Native Method)                                                                                        
>         at org.apache.hadoop.hbase.util.Sleeper.sleep(Sleeper.java:74)                                                                  
>         at org.apache.hadoop.hbase.Chore.run(Chore.java:72)                                                                              
> "LeaseChecker" daemon prio=10 tid=0x0837cc00 nid=0x870 waiting on condition [0x8f6ad000..0x8f6ae040]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                
>         at java.lang.Thread.sleep(Native Method)                                                    
>         at org.apache.hadoop.hdfs.DFSClient$LeaseChecker.run(DFSClient.java:979)                    
>         at java.lang.Thread.run(Thread.java:619)                                                    
> "DataStreamer for file /hbase/log_192.168.1.13_1235129194745_60020/hlog.dat.1235129195649 block blk_7545556036225037274_1820952" daemon prio=10 tid=0x08375400 nid=0x86f in Object.wait() [0x8f6fe000..0x8f6feec0]                                                                                                      
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                                
>         at java.lang.Object.wait(Native Method)                                                                                                            
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2166)                                                          
>         - locked <0x93d0ea10> (a java.util.LinkedList)                                                                                                      
> "DestroyJavaVM" prio=10 tid=0x0805a000 nid=0x855 waiting on condition [0x00000000..0xb7dd2090]
>    java.lang.Thread.State: RUNNABLE                                                          
> "regionserver/0.0.0.0:60020" prio=10 tid=0x085b2400 nid=0x863 in Object.wait() [0x8f7f7000..0x8f7f7f40]
>    java.lang.Thread.State: WAITING (on object monitor)                                                
>         at java.lang.Object.wait(Native Method)                                                        
>         at java.lang.Object.wait(Object.java:485)                                                      
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.flushInternal(DFSClient.java:3015)        
>         - locked <0x93d0ea10> (a java.util.LinkedList)                                                
>         - locked <0x93d0ec58> (a org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)                    
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.closeInternal(DFSClient.java:3104)        
>         - locked <0x93d0ec58> (a org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)                    
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.close(DFSClient.java:3053)
>         at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:59)
>         at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:79)
>         at org.apache.hadoop.io.SequenceFile$Writer.close(SequenceFile.java:959)
>         - locked <0x93d50d20> (a org.apache.hadoop.io.SequenceFile$Writer)
>         at org.apache.hadoop.hbase.regionserver.HLog.close(HLog.java:421)
>         - locked <0x93d4ce00> (a java.lang.Integer)
>         at org.apache.hadoop.hbase.regionserver.HLog.closeAndDelete(HLog.java:404)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:373)
>         at java.lang.Thread.run(Thread.java:619)
> "Low Memory Detector" daemon prio=10 tid=0x080dd800 nid=0x85e runnable [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread1" daemon prio=10 tid=0x080dc400 nid=0x85d waiting on condition [0x00000000..0x8fbc1588]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread0" daemon prio=10 tid=0x080d9c00 nid=0x85c waiting on condition [0x00000000..0x8fc42608]
>    java.lang.Thread.State: RUNNABLE
> "Signal Dispatcher" daemon prio=10 tid=0x080d8800 nid=0x85b runnable [0x00000000..0x8fc93e80]
>    java.lang.Thread.State: RUNNABLE
> "Surrogate Locker Thread (CMS)" daemon prio=10 tid=0x080d7800 nid=0x85a waiting on condition [0x00000000..0x8fce522c]
>    java.lang.Thread.State: RUNNABLE
> "Finalizer" daemon prio=10 tid=0x080bbc00 nid=0x859 in Object.wait() [0x8fd7b000..0x8fd7bec0]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116)
>         - locked <0x93d0bdc0> (a java.lang.ref.ReferenceQueue$Lock)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:132)
>         at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159)
> "Reference Handler" daemon prio=10 tid=0x080ba800 nid=0x858 in Object.wait() [0x8fdcc000..0x8fdccf40]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.Object.wait(Object.java:485)
>         at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)
>         - locked <0x93d0c110> (a java.lang.ref.Reference$Lock)
> "VM Thread" prio=10 tid=0x080b7800 nid=0x857 runnable
> "Concurrent Mark-Sweep GC Thread" prio=10 tid=0x0806c800 nid=0x856 runnable
> "VM Periodic Task Thread" prio=10 tid=0x080df000 nid=0x85f waiting on condition
> JNI global references: 922
> {noformat}
> Notice the DataStreamer thread that seems to work on the concerned block
> This force me to kill (-KILL) the region server, and restart the hdfs (if I don't restart hdfs some regions are not available because the block is still not available).
> Note that the Hbase master does not end as well, and I have to kill (-KILL). Here is the thread dump of the HMaster process:
> h5. Master thread dump:
> {noformat}
> Full thread dump Java HotSpot(TM) Server VM (10.0-b23 mixed mode):
> "Attach Listener" daemon prio=10 tid=0x0805a000 nid=0x2326 waiting on condition [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE                                                                    
> "SIGTERM handler" daemon prio=10 tid=0x089f0c00 nid=0x2303 waiting for monitor entry [0x8a6a5000..0x8a6a5ec0]
>    java.lang.Thread.State: BLOCKED (on object monitor)                                                      
>         at java.lang.Shutdown.exit(Shutdown.java:178)                                                        
>         - waiting to lock <0xb0beed60> (a java.lang.Class for java.lang.Shutdown)                            
>         at java.lang.Terminator$1.handle(Terminator.java:35)                                                
>         at sun.misc.Signal$1.run(Signal.java:195)                                                            
>         at java.lang.Thread.run(Thread.java:619)                                                            
> "Thread-1" prio=10 tid=0x8f666000 nid=0x653c waiting on condition [0x8f1fe000..0x8f1fef40]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                      
>         at java.lang.Thread.sleep(Native Method)                                          
>         at org.apache.hadoop.ipc.Client.stop(Client.java:667)                            
>         at org.apache.hadoop.ipc.RPC$ClientCache.stopClient(RPC.java:189)                
>         at org.apache.hadoop.ipc.RPC$ClientCache.access$400(RPC.java:138)                
>         at org.apache.hadoop.ipc.RPC$Invoker.close(RPC.java:229)                          
>         - locked <0x938a1b18> (a org.apache.hadoop.ipc.RPC$Invoker)                      
>         at org.apache.hadoop.ipc.RPC$Invoker.access$500(RPC.java:196)                    
>         at org.apache.hadoop.ipc.RPC.stopProxy(RPC.java:353)                              
>         at org.apache.hadoop.hdfs.DFSClient.close(DFSClient.java:213)                    
>         - locked <0x93878470> (a org.apache.hadoop.hdfs.DFSClient)                        
>         at org.apache.hadoop.hdfs.DistributedFileSystem.close(DistributedFileSystem.java:243)
>         at org.apache.hadoop.fs.FileSystem$Cache.closeAll(FileSystem.java:1413)              
>         - locked <0x937d3c00> (a org.apache.hadoop.fs.FileSystem$Cache)                      
>         at org.apache.hadoop.fs.FileSystem.closeAll(FileSystem.java:236)                    
>         at org.apache.hadoop.fs.FileSystem$ClientFinalizer.run(FileSystem.java:221)          
>         - locked <0x937b8a78> (a org.apache.hadoop.fs.FileSystem$ClientFinalizer)            
> "DestroyJavaVM" prio=10 tid=0x8f66c800 nid=0x36e9 in Object.wait() [0xb7d7d000..0xb7d7e0e0]
>    java.lang.Thread.State: WAITING (on object monitor)                                    
>         at java.lang.Object.wait(Native Method)                                            
>         at java.lang.Thread.join(Thread.java:1143)                                        
>         - locked <0x937b8a78> (a org.apache.hadoop.fs.FileSystem$ClientFinalizer)          
>         at java.lang.Thread.join(Thread.java:1196)
>         at java.lang.ApplicationShutdownHooks.run(ApplicationShutdownHooks.java:79)
>         at java.lang.Shutdown.runHooks(Shutdown.java:89)
>         at java.lang.Shutdown.sequence(Shutdown.java:133)
>         at java.lang.Shutdown.shutdown(Shutdown.java:200)
>         - locked <0xb0beed60> (a java.lang.Class for java.lang.Shutdown)
> "Low Memory Detector" daemon prio=10 tid=0x8fe03800 nid=0x36f4 runnable [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread1" daemon prio=10 tid=0x8fe01c00 nid=0x36f3 waiting on condition [0x00000000..0x8f9ee488]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread0" daemon prio=10 tid=0x8fe00800 nid=0x36f2 waiting on condition [0x00000000..0x8fa6f508]
>    java.lang.Thread.State: RUNNABLE
> "Signal Dispatcher" daemon prio=10 tid=0x0813e800 nid=0x36f1 runnable [0x00000000..0x8fac0d80]
>    java.lang.Thread.State: RUNNABLE
> "Surrogate Locker Thread (CMS)" daemon prio=10 tid=0x0813d800 nid=0x36f0 waiting on condition [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE
> "Finalizer" daemon prio=10 tid=0x08125000 nid=0x36ef in Object.wait() [0x8fba1000..0x8fba1dc0]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116)
>         - locked <0x937bafd8> (a java.lang.ref.ReferenceQueue$Lock)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:132)
>         at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159)
> "Reference Handler" daemon prio=10 tid=0x08120c00 nid=0x36ee in Object.wait() [0x8fbf2000..0x8fbf2e40]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.Object.wait(Object.java:485)
>         at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)
>         - locked <0x937baf18> (a java.lang.ref.Reference$Lock)
> "VM Thread" prio=10 tid=0x0811d800 nid=0x36ed runnable
> "Gang worker#0 (Parallel GC Threads)" prio=10 tid=0x0805e000 nid=0x36ea runnable
> "Gang worker#1 (Parallel GC Threads)" prio=10 tid=0x0805f000 nid=0x36eb runnable
> "Concurrent Mark-Sweep GC Thread" prio=10 tid=0x080cd000 nid=0x36ec runnable
> "VM Periodic Task Thread" prio=10 tid=0x8fe05000 nid=0x36f5 waiting on condition
> JNI global references: 1044
> {noformat}
> When I restart dfs, the log shows me that the concerned block is marked as invalidate, and it is therefore deleted:
> h5. Namenode log
> {noformat}
> 2009-02-23 10:33:49,245 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.processReport: block blk_7545556036225037274_1820952 on 192.168.1.10:50010 size 27895296 does not
>  belong to any file.
> 2009-02-23 10:33:49,245 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addToInvalidates: blk_7545556036225037274 is added to invalidSet of 192.168.1.10:50010
> {noformat}
> Finally, when I restart HBase, I can see some of these messages in region server log, and I have some data loss. (this log entry may not be linked with the  block traced before)
> h5. Region server log:
> {noformat}
> 2009-02-23 10:38:55,831 WARN org.apache.hadoop.hbase.regionserver.HStore: Exception processing reconstruction log hdfs://lab5-2:9000/hbase/test-D-0.3/510178748/oldlogfile.log opening [B@ff65bf -- continuing.  Probably lack-of-HADOOP-1700 causing DATA LOSS!
> java.io.EOFException
>         at java.io.DataInputStream.readFully(DataInputStream.java:180)
>         at org.apache.hadoop.hbase.io.DataOutputBuffer$Buffer.write(DataOutputBuffer.java:70)
>         at org.apache.hadoop.hbase.io.DataOutputBuffer.write(DataOutputBuffer.java:116)
>         at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1944)
>         at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1844)
>         at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1890)
>         at org.apache.hadoop.hbase.regionserver.HStore.doReconstructionLog(HStore.java:352)
>         at org.apache.hadoop.hbase.regionserver.HStore.runReconstructionLog(HStore.java:297)
>         at org.apache.hadoop.hbase.regionserver.HStore.<init>(HStore.java:237)
>         at org.apache.hadoop.hbase.regionserver.HRegion.instantiateHStore(HRegion.java:1764)
>         at org.apache.hadoop.hbase.regionserver.HRegion.initialize(HRegion.java:276)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.instantiateRegion(HRegionServer.java:1367)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.openRegion(HRegionServer.java:1338)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer$Worker.run(HRegionServer.java:1253)
>         at java.lang.Thread.run(Thread.java:619)
> {noformat}
> In my sense, since my hardware is in cause for RegionServer lease miss, I have to deal with the fact that I have to stop/restart hbase + hadoop in such a case. The problem in my sense is that a stop request should be able to perform a clean shutdown, and to avoid dataloss.
> Hope this description helps.

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


[jira] Commented: (HBASE-1214) HRegionServer fails to stop if hdfs client it tries to recover a block

Posted by "Dave Latham (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-1214?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12676332#action_12676332 ] 

Dave Latham commented on HBASE-1214:
------------------------------------

Would it also be considered a bug in HDFS that the DFSClient gets stuck in an endless recover loop?  I think I have observed the same behavior.

> HRegionServer fails to stop if hdfs client it tries to recover a block 
> -----------------------------------------------------------------------
>
>                 Key: HBASE-1214
>                 URL: https://issues.apache.org/jira/browse/HBASE-1214
>             Project: Hadoop HBase
>          Issue Type: Bug
>          Components: master, regionserver
>    Affects Versions: 0.19.0
>         Environment: 4 Node cluster with poor hardware: (DN+RS / DN+RS / RS / MA+NN)
> 1 Gb memory each 
> Ubuntu linux
> Java 6
>            Reporter: Jean-Adrien
>
> One of my region server falls in a long GC time that get it unresponsive during about 10 minutes.
> As I can see in the log, it seems that its DFSClient component was sending a file to hdfs, the sending times out when it recovers from GC:
> h5. Region server log after recovering from GC
> {noformat}
> 2009-02-21 01:22:26,454 WARN org.apache.hadoop.hdfs.DFSClient: DFSOutputStream ResponseProcessor exception  for block blk_7545556036225037274_1820952java.io.IOException: Bad response 1 for block blk_7545556036225037274_1820952 from datanode 192.168.1.10:50010
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:2342)
> {noformat}
> h5. corresponding error in receiving datanode
> {noformat}
> 2009-02-21 01:23:56,608 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Exception in receiveBlock for block blk_7545556036225037274_1820952 java.io.EOFException: while trying to read 65557 bytes
> [...]
> 2009-02-21 01:23:59,076 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 0 for block blk_7545556036225037274_1820952 Interrupted.
> 2009-02-21 01:24:01,484 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 0 for block blk_7545556036225037274_1820952 terminating
> 2009-02-21 01:24:01,485 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: writeBlock blk_7545556036225037274_1820952 received exception java.io.EOFException: while trying to read 65557 bytes
> {noformat}
> Since region server misses its lease to report to the master, it has to close all its regions before recover, and reopens them when the master asks for.
> From this time, it tries to _recover_ this block, as seen in the regionserver log:
> h5. Region server log in an endless loop to recover
> {noformat}
> 009-02-21 01:22:29,327 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_7545556036225037274_1820952 bad datanode[1] 192.168.1.10:50010
> 2009-02-21 01:22:29,327 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_7545556036225037274_1820952 in pipeline 192.168.1.13:50010, 192.
> 168.1.10:50010: bad datanode 192.168.1.10:50010
> 2009-02-21 01:22:29,689 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_7545556036225037274_1820952 failed  because recovery from primary datanode 192.168.1.13:50010 failed 2 times. Will retry...
> {noformat}
> To this _recover_ request, all datanodes fail with this Exception
> {noformat}
> 2009-02-21 01:24:18,650 INFO org.apache.hadoop.ipc.Server: IPC Server handler 1 on 50020, call recoverBlock(blk_7545556036225037274_1820952, false, [Lorg.apache.hadoop.hdfs.protocol.DatanodeInfo;@311c4f) from 192.168.1.13:56968: error: org.apache.hadoop.ipc.RemoteException: java.io.IOException: blk_7545556036225037274_1820952 is already commited, storedBlock == null.
>         at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.nextGenerationStampForBlock(FSNamesystem.java:4536)
>         at org.apache.hadoop.hdfs.server.namenode.NameNode.nextGenerationStamp(NameNode.java:402)
>         at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
> org.apache.hadoop.ipc.RemoteException: java.io.IOException: blk_7545556036225037274_1820952 is already commited, storedBlock == null.
>         at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.nextGenerationStampForBlock(FSNamesystem.java:4536)
>         at org.apache.hadoop.hdfs.server.namenode.NameNode.nextGenerationStamp(NameNode.java:402)
>         at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
>         at org.apache.hadoop.ipc.Client.call(Client.java:696)
>         at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:216)
>         at $Proxy4.nextGenerationStamp(Unknown Source)
>         at org.apache.hadoop.hdfs.server.datanode.DataNode.syncBlock(DataNode.java:1466)
>         at org.apache.hadoop.hdfs.server.datanode.DataNode.recoverBlock(DataNode.java:1440)
>         at org.apache.hadoop.hdfs.server.datanode.DataNode.recoverBlock(DataNode.java:1506)
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
> {noformat}
> The problem is that, in this case, the RegionServer and therefore the master fails to stop when I launch the stop-hbase script.
> A kill (-TERM) on the region server launches the SIGTERM handler thread, but the loop which try to reach the block does not stop.
> here is a thread dump of the region server at this time:
> h5. Thread dump of the region server
> {noformat}
> Full thread dump Java HotSpot(TM) Server VM (10.0-b23 mixed mode):
> "Thread-16" prio=10 tid=0x081c3c00 nid=0x6e81 in Object.wait() [0x8f74f000..0x8f74fec0]
>    java.lang.Thread.State: WAITING (on object monitor)                                
>         at java.lang.Object.wait(Native Method)                                        
>         at java.lang.Thread.join(Thread.java:1143)                                    
>         - locked <0x93d0d060> (a java.lang.Thread)                                    
>         at java.lang.Thread.join(Thread.java:1196)                                    
>         at org.apache.hadoop.hbase.util.Threads.shutdown(Threads.java:78)              
>         at org.apache.hadoop.hbase.util.Threads.shutdown(Threads.java:66)              
>         at org.apache.hadoop.hbase.regionserver.HRegionServer$ShutdownThread.run(HRegionServer.java:814)
> "SIGTERM handler" daemon prio=10 tid=0x08dbcc00 nid=0x6e80 in Object.wait() [0x8edaf000..0x8edaff40]
>    java.lang.Thread.State: WAITING (on object monitor)                                              
>         at java.lang.Object.wait(Native Method)                                                    
>         at java.lang.Thread.join(Thread.java:1143)                                                  
>         - locked <0x93d0f3c0> (a org.apache.hadoop.hbase.regionserver.HRegionServer$ShutdownThread)
>         at java.lang.Thread.join(Thread.java:1196)                                                  
>         at java.lang.ApplicationShutdownHooks.run(ApplicationShutdownHooks.java:79)                
>         at java.lang.Shutdown.runHooks(Shutdown.java:89)                                            
>         at java.lang.Shutdown.sequence(Shutdown.java:133)                                          
>         at java.lang.Shutdown.exit(Shutdown.java:178)                                              
>         - locked <0xb0d3ed60> (a java.lang.Class for java.lang.Shutdown)                            
>         at java.lang.Terminator$1.handle(Terminator.java:35)                                        
>         at sun.misc.Signal$1.run(Signal.java:195)                                                  
>         at java.lang.Thread.run(Thread.java:619)                                                    
> "Attach Listener" daemon prio=10 tid=0x081e8800 nid=0x6e50 waiting on condition [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE                                                                    
> "IPC Client (47) connection to /192.168.1.13:50020 from an unknown user" daemon prio=10 tid=0x8e97a400 nid=0x39d1 in Object.wait() [0x8ec81000..0x8ec81fc0]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                              
>         at java.lang.Object.wait(Native Method)                                                                                                            
>         at org.apache.hadoop.ipc.Client$Connection.waitForWork(Client.java:395)                                                                            
>         - locked <0xa44a36a0> (a org.apache.hadoop.ipc.Client$Connection)                                                                                  
>         at org.apache.hadoop.ipc.Client$Connection.run(Client.java:437)                                                                                    
> "IPC Client (47) connection to /192.168.1.10:50020 from an unknown user" daemon prio=10 tid=0x8e5f1c00 nid=0x523f in Object.wait() [0x8ecd3000..0x8ecd3140]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                              
>         at java.lang.Object.wait(Native Method)                                                                                                            
>         at org.apache.hadoop.ipc.Client$Connection.waitForWork(Client.java:395)                                                                            
>         - locked <0xa448ca50> (a org.apache.hadoop.ipc.Client$Connection)                                                                                  
>         at org.apache.hadoop.ipc.Client$Connection.run(Client.java:437)                                                                                    
> "BlockFSInputStreamReferenceQueueChecker" daemon prio=10 tid=0x085a9400 nid=0x8c8 waiting on condition [0x8ee6c000..0x8ee6cdc0]
>    java.lang.Thread.State: WAITING (parking)                                                                                  
>         at sun.misc.Unsafe.park(Native Method)                                                                                
>         - parking to wait for  <0x93e76068> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)          
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                                  
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)  
>         at java.util.concurrent.DelayQueue.take(DelayQueue.java:160)                                                          
>         at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:582)        
>         at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:575)        
>         at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:946)                                        
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:906)                                    
>         at java.lang.Thread.run(Thread.java:619)                                                                              
> "IPC Server handler 9 on 60020" daemon prio=10 tid=0x0882e000 nid=0x889 waiting on condition [0x8eebd000..0x8eebdfc0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 8 on 60020" daemon prio=10 tid=0x0882cc00 nid=0x888 waiting on condition [0x8ef0e000..0x8ef0f040]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 7 on 60020" daemon prio=10 tid=0x0882b800 nid=0x887 waiting on condition [0x8ef5f000..0x8ef5fec0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 6 on 60020" daemon prio=10 tid=0x0882a400 nid=0x886 waiting on condition [0x8efb0000..0x8efb0f40]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 5 on 60020" daemon prio=10 tid=0x08829000 nid=0x885 waiting on condition [0x8f001000..0x8f001dc0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 4 on 60020" daemon prio=10 tid=0x083bcc00 nid=0x884 waiting on condition [0x8f052000..0x8f052e40]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 3 on 60020" daemon prio=10 tid=0x083bb800 nid=0x883 waiting on condition [0x8f0a3000..0x8f0a40c0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 2 on 60020" daemon prio=10 tid=0x083ba400 nid=0x882 waiting on condition [0x8f0f4000..0x8f0f5140]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 1 on 60020" daemon prio=10 tid=0x083b9400 nid=0x881 waiting on condition [0x8f145000..0x8f145fc0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 0 on 60020" daemon prio=10 tid=0x083b8400 nid=0x880 waiting on condition [0x8f196000..0x8f197040]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server listener on 60020" daemon prio=10 tid=0x083bfc00 nid=0x87f runnable [0x8f1e7000..0x8f1e7ec0]
>    java.lang.Thread.State: RUNNABLE                                                                    
>         at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)                                        
>         at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)                                
>         at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)                            
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)                                
>         - locked <0x93d0c090> (a sun.nio.ch.Util$1)                                                    
>         - locked <0x93d0c120> (a java.util.Collections$UnmodifiableSet)                                
>         - locked <0x93d0c0b0> (a sun.nio.ch.EPollSelectorImpl)                                          
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)                                        
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:84)                                        
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Listener.run(HBaseServer.java:299)                  
> "IPC Server Responder" daemon prio=10 tid=0x083bec00 nid=0x87e runnable [0x8f238000..0x8f238f40]
>    java.lang.Thread.State: RUNNABLE                                                            
>         at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)                                
>         at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)                        
>         at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)                    
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)                        
>         - locked <0x93d0c0f0> (a sun.nio.ch.Util$1)                                            
>         - locked <0x93d0c100> (a java.util.Collections$UnmodifiableSet)                        
>         - locked <0x93d0be30> (a sun.nio.ch.EPollSelectorImpl)                                  
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)                                
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Responder.run(HBaseServer.java:458)          
> "SocketListener0-1" prio=10 tid=0x088d5000 nid=0x87c in Object.wait() [0x8f2da000..0x8f2dae40]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                  
>         at java.lang.Object.wait(Native Method)                                              
>         at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:522)                    
>         - locked <0x93e39bc8> (a org.mortbay.util.ThreadPool$PoolThread)                      
> "SocketListener0-0" prio=10 tid=0x08ab5000 nid=0x87b in Object.wait() [0x8f32b000..0x8f32c0c0]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                  
>         at java.lang.Object.wait(Native Method)                                              
>         at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:522)                    
>         - locked <0x93e39920> (a org.mortbay.util.ThreadPool$PoolThread)                      
> "Acceptor ServerSocket[addr=0.0.0.0/0.0.0.0,port=0,localport=60030]" prio=10 tid=0x084f6800 nid=0x87a runnable [0x8f37c000..0x8f37d140]
>    java.lang.Thread.State: RUNNABLE                                                                                                    
>         at java.net.PlainSocketImpl.socketAccept(Native Method)                                                                        
>         at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:384)                                                                  
>         - locked <0x93e382a8> (a java.net.SocksSocketImpl)                                                                            
>         at java.net.ServerSocket.implAccept(ServerSocket.java:453)                                                                    
>         at java.net.ServerSocket.accept(ServerSocket.java:421)                                                                        
>         at org.mortbay.util.ThreadedServer.acceptSocket(ThreadedServer.java:432)                                                      
>         at org.mortbay.util.ThreadedServer$Acceptor.run(ThreadedServer.java:631)                                                      
> "SessionScavenger" daemon prio=10 tid=0x088f1400 nid=0x879 waiting on condition [0x8f3cd000..0x8f3cdfc0]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                    
>         at java.lang.Thread.sleep(Native Method)                                                        
>         at org.mortbay.jetty.servlet.AbstractSessionManager$SessionScavenger.run(AbstractSessionManager.java:587)
> "SessionScavenger" daemon prio=10 tid=0x08ab5c00 nid=0x878 waiting on condition [0x8f41e000..0x8f41f040]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                    
>         at java.lang.Thread.sleep(Native Method)                                                        
>         at org.mortbay.jetty.servlet.AbstractSessionManager$SessionScavenger.run(AbstractSessionManager.java:587)
> "regionserver/0.0.0.0:60020.leaseChecker" prio=10 tid=0x086d5800 nid=0x877 waiting on condition [0x8f476000..0x8f476ec0]
>    java.lang.Thread.State: TIMED_WAITING (parking)                                                                      
>         at sun.misc.Unsafe.park(Native Method)                                                                          
>         - parking to wait for  <0x93d0e980> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)  
>         at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:198)                                      
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:1963)
>         at java.util.concurrent.DelayQueue.poll(DelayQueue.java:201)                                                            
>         at org.apache.hadoop.hbase.Leases.run(Leases.java:78)                                                                    
> "regionserver/0.0.0.0:60020.majorCompactionChecker" daemon prio=10 tid=0x086d4000 nid=0x876 waiting on condition [0x8f4c7000..0x8f4c7f40]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                                                      
>         at java.lang.Thread.sleep(Native Method)                                                                                        
>         at org.apache.hadoop.hbase.util.Sleeper.sleep(Sleeper.java:74)                                                                  
>         at org.apache.hadoop.hbase.Chore.run(Chore.java:72)                                                                              
> "LeaseChecker" daemon prio=10 tid=0x0837cc00 nid=0x870 waiting on condition [0x8f6ad000..0x8f6ae040]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                
>         at java.lang.Thread.sleep(Native Method)                                                    
>         at org.apache.hadoop.hdfs.DFSClient$LeaseChecker.run(DFSClient.java:979)                    
>         at java.lang.Thread.run(Thread.java:619)                                                    
> "DataStreamer for file /hbase/log_192.168.1.13_1235129194745_60020/hlog.dat.1235129195649 block blk_7545556036225037274_1820952" daemon prio=10 tid=0x08375400 nid=0x86f in Object.wait() [0x8f6fe000..0x8f6feec0]                                                                                                      
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                                
>         at java.lang.Object.wait(Native Method)                                                                                                            
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2166)                                                          
>         - locked <0x93d0ea10> (a java.util.LinkedList)                                                                                                      
> "DestroyJavaVM" prio=10 tid=0x0805a000 nid=0x855 waiting on condition [0x00000000..0xb7dd2090]
>    java.lang.Thread.State: RUNNABLE                                                          
> "regionserver/0.0.0.0:60020" prio=10 tid=0x085b2400 nid=0x863 in Object.wait() [0x8f7f7000..0x8f7f7f40]
>    java.lang.Thread.State: WAITING (on object monitor)                                                
>         at java.lang.Object.wait(Native Method)                                                        
>         at java.lang.Object.wait(Object.java:485)                                                      
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.flushInternal(DFSClient.java:3015)        
>         - locked <0x93d0ea10> (a java.util.LinkedList)                                                
>         - locked <0x93d0ec58> (a org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)                    
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.closeInternal(DFSClient.java:3104)        
>         - locked <0x93d0ec58> (a org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)                    
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.close(DFSClient.java:3053)
>         at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:59)
>         at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:79)
>         at org.apache.hadoop.io.SequenceFile$Writer.close(SequenceFile.java:959)
>         - locked <0x93d50d20> (a org.apache.hadoop.io.SequenceFile$Writer)
>         at org.apache.hadoop.hbase.regionserver.HLog.close(HLog.java:421)
>         - locked <0x93d4ce00> (a java.lang.Integer)
>         at org.apache.hadoop.hbase.regionserver.HLog.closeAndDelete(HLog.java:404)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:373)
>         at java.lang.Thread.run(Thread.java:619)
> "Low Memory Detector" daemon prio=10 tid=0x080dd800 nid=0x85e runnable [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread1" daemon prio=10 tid=0x080dc400 nid=0x85d waiting on condition [0x00000000..0x8fbc1588]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread0" daemon prio=10 tid=0x080d9c00 nid=0x85c waiting on condition [0x00000000..0x8fc42608]
>    java.lang.Thread.State: RUNNABLE
> "Signal Dispatcher" daemon prio=10 tid=0x080d8800 nid=0x85b runnable [0x00000000..0x8fc93e80]
>    java.lang.Thread.State: RUNNABLE
> "Surrogate Locker Thread (CMS)" daemon prio=10 tid=0x080d7800 nid=0x85a waiting on condition [0x00000000..0x8fce522c]
>    java.lang.Thread.State: RUNNABLE
> "Finalizer" daemon prio=10 tid=0x080bbc00 nid=0x859 in Object.wait() [0x8fd7b000..0x8fd7bec0]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116)
>         - locked <0x93d0bdc0> (a java.lang.ref.ReferenceQueue$Lock)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:132)
>         at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159)
> "Reference Handler" daemon prio=10 tid=0x080ba800 nid=0x858 in Object.wait() [0x8fdcc000..0x8fdccf40]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.Object.wait(Object.java:485)
>         at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)
>         - locked <0x93d0c110> (a java.lang.ref.Reference$Lock)
> "VM Thread" prio=10 tid=0x080b7800 nid=0x857 runnable
> "Concurrent Mark-Sweep GC Thread" prio=10 tid=0x0806c800 nid=0x856 runnable
> "VM Periodic Task Thread" prio=10 tid=0x080df000 nid=0x85f waiting on condition
> JNI global references: 922
> {noformat}
> Notice the DataStreamer thread that seems to work on the concerned block
> This force me to kill (-KILL) the region server, and restart the hdfs (if I don't restart hdfs some regions are not available because the block is still not available).
> Note that the Hbase master does not end as well, and I have to kill (-KILL). Here is the thread dump of the HMaster process:
> h5. Master thread dump:
> {noformat}
> Full thread dump Java HotSpot(TM) Server VM (10.0-b23 mixed mode):
> "Attach Listener" daemon prio=10 tid=0x0805a000 nid=0x2326 waiting on condition [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE                                                                    
> "SIGTERM handler" daemon prio=10 tid=0x089f0c00 nid=0x2303 waiting for monitor entry [0x8a6a5000..0x8a6a5ec0]
>    java.lang.Thread.State: BLOCKED (on object monitor)                                                      
>         at java.lang.Shutdown.exit(Shutdown.java:178)                                                        
>         - waiting to lock <0xb0beed60> (a java.lang.Class for java.lang.Shutdown)                            
>         at java.lang.Terminator$1.handle(Terminator.java:35)                                                
>         at sun.misc.Signal$1.run(Signal.java:195)                                                            
>         at java.lang.Thread.run(Thread.java:619)                                                            
> "Thread-1" prio=10 tid=0x8f666000 nid=0x653c waiting on condition [0x8f1fe000..0x8f1fef40]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                      
>         at java.lang.Thread.sleep(Native Method)                                          
>         at org.apache.hadoop.ipc.Client.stop(Client.java:667)                            
>         at org.apache.hadoop.ipc.RPC$ClientCache.stopClient(RPC.java:189)                
>         at org.apache.hadoop.ipc.RPC$ClientCache.access$400(RPC.java:138)                
>         at org.apache.hadoop.ipc.RPC$Invoker.close(RPC.java:229)                          
>         - locked <0x938a1b18> (a org.apache.hadoop.ipc.RPC$Invoker)                      
>         at org.apache.hadoop.ipc.RPC$Invoker.access$500(RPC.java:196)                    
>         at org.apache.hadoop.ipc.RPC.stopProxy(RPC.java:353)                              
>         at org.apache.hadoop.hdfs.DFSClient.close(DFSClient.java:213)                    
>         - locked <0x93878470> (a org.apache.hadoop.hdfs.DFSClient)                        
>         at org.apache.hadoop.hdfs.DistributedFileSystem.close(DistributedFileSystem.java:243)
>         at org.apache.hadoop.fs.FileSystem$Cache.closeAll(FileSystem.java:1413)              
>         - locked <0x937d3c00> (a org.apache.hadoop.fs.FileSystem$Cache)                      
>         at org.apache.hadoop.fs.FileSystem.closeAll(FileSystem.java:236)                    
>         at org.apache.hadoop.fs.FileSystem$ClientFinalizer.run(FileSystem.java:221)          
>         - locked <0x937b8a78> (a org.apache.hadoop.fs.FileSystem$ClientFinalizer)            
> "DestroyJavaVM" prio=10 tid=0x8f66c800 nid=0x36e9 in Object.wait() [0xb7d7d000..0xb7d7e0e0]
>    java.lang.Thread.State: WAITING (on object monitor)                                    
>         at java.lang.Object.wait(Native Method)                                            
>         at java.lang.Thread.join(Thread.java:1143)                                        
>         - locked <0x937b8a78> (a org.apache.hadoop.fs.FileSystem$ClientFinalizer)          
>         at java.lang.Thread.join(Thread.java:1196)
>         at java.lang.ApplicationShutdownHooks.run(ApplicationShutdownHooks.java:79)
>         at java.lang.Shutdown.runHooks(Shutdown.java:89)
>         at java.lang.Shutdown.sequence(Shutdown.java:133)
>         at java.lang.Shutdown.shutdown(Shutdown.java:200)
>         - locked <0xb0beed60> (a java.lang.Class for java.lang.Shutdown)
> "Low Memory Detector" daemon prio=10 tid=0x8fe03800 nid=0x36f4 runnable [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread1" daemon prio=10 tid=0x8fe01c00 nid=0x36f3 waiting on condition [0x00000000..0x8f9ee488]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread0" daemon prio=10 tid=0x8fe00800 nid=0x36f2 waiting on condition [0x00000000..0x8fa6f508]
>    java.lang.Thread.State: RUNNABLE
> "Signal Dispatcher" daemon prio=10 tid=0x0813e800 nid=0x36f1 runnable [0x00000000..0x8fac0d80]
>    java.lang.Thread.State: RUNNABLE
> "Surrogate Locker Thread (CMS)" daemon prio=10 tid=0x0813d800 nid=0x36f0 waiting on condition [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE
> "Finalizer" daemon prio=10 tid=0x08125000 nid=0x36ef in Object.wait() [0x8fba1000..0x8fba1dc0]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116)
>         - locked <0x937bafd8> (a java.lang.ref.ReferenceQueue$Lock)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:132)
>         at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159)
> "Reference Handler" daemon prio=10 tid=0x08120c00 nid=0x36ee in Object.wait() [0x8fbf2000..0x8fbf2e40]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.Object.wait(Object.java:485)
>         at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)
>         - locked <0x937baf18> (a java.lang.ref.Reference$Lock)
> "VM Thread" prio=10 tid=0x0811d800 nid=0x36ed runnable
> "Gang worker#0 (Parallel GC Threads)" prio=10 tid=0x0805e000 nid=0x36ea runnable
> "Gang worker#1 (Parallel GC Threads)" prio=10 tid=0x0805f000 nid=0x36eb runnable
> "Concurrent Mark-Sweep GC Thread" prio=10 tid=0x080cd000 nid=0x36ec runnable
> "VM Periodic Task Thread" prio=10 tid=0x8fe05000 nid=0x36f5 waiting on condition
> JNI global references: 1044
> {noformat}
> When I restart dfs, the log shows me that the concerned block is marked as invalidate, and it is therefore deleted:
> h5. Namenode log
> {noformat}
> 2009-02-23 10:33:49,245 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.processReport: block blk_7545556036225037274_1820952 on 192.168.1.10:50010 size 27895296 does not
>  belong to any file.
> 2009-02-23 10:33:49,245 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addToInvalidates: blk_7545556036225037274 is added to invalidSet of 192.168.1.10:50010
> {noformat}
> Finally, when I restart HBase, I can see some of these messages in region server log, and I have some data loss. (this log entry may not be linked with the  block traced before)
> h5. Region server log:
> {noformat}
> 2009-02-23 10:38:55,831 WARN org.apache.hadoop.hbase.regionserver.HStore: Exception processing reconstruction log hdfs://lab5-2:9000/hbase/test-D-0.3/510178748/oldlogfile.log opening [B@ff65bf -- continuing.  Probably lack-of-HADOOP-1700 causing DATA LOSS!
> java.io.EOFException
>         at java.io.DataInputStream.readFully(DataInputStream.java:180)
>         at org.apache.hadoop.hbase.io.DataOutputBuffer$Buffer.write(DataOutputBuffer.java:70)
>         at org.apache.hadoop.hbase.io.DataOutputBuffer.write(DataOutputBuffer.java:116)
>         at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1944)
>         at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1844)
>         at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1890)
>         at org.apache.hadoop.hbase.regionserver.HStore.doReconstructionLog(HStore.java:352)
>         at org.apache.hadoop.hbase.regionserver.HStore.runReconstructionLog(HStore.java:297)
>         at org.apache.hadoop.hbase.regionserver.HStore.<init>(HStore.java:237)
>         at org.apache.hadoop.hbase.regionserver.HRegion.instantiateHStore(HRegion.java:1764)
>         at org.apache.hadoop.hbase.regionserver.HRegion.initialize(HRegion.java:276)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.instantiateRegion(HRegionServer.java:1367)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.openRegion(HRegionServer.java:1338)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer$Worker.run(HRegionServer.java:1253)
>         at java.lang.Thread.run(Thread.java:619)
> {noformat}
> In my sense, since my hardware is in cause for RegionServer lease miss, I have to deal with the fact that I have to stop/restart hbase + hadoop in such a case. The problem in my sense is that a stop request should be able to perform a clean shutdown, and to avoid dataloss.
> Hope this description helps.

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


[jira] Updated: (HBASE-1214) HRegionServer fails to stop if hdfs client it tries to recover a block

Posted by "Jean-Adrien (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-1214?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jean-Adrien updated HBASE-1214:
-------------------------------

    Description: 
One of my region server falls in a long GC time that get it unresponsive during about 10 minutes.
As I can see in the log, it seems that its DFSClient component was sending a file to hdfs, the sending times out when it recovers from GC:

h5. 1. Region server log after recovering from GC
{noformat}
2009-02-21 01:22:26,454 WARN org.apache.hadoop.hdfs.DFSClient: DFSOutputStream ResponseProcessor exception  
    for block blk_7545556036225037274_1820952java.io.IOException: Bad response 1 for block 
    blk_7545556036225037274_1820952 from datanode 192.168.1.10:50010
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:2342)
{noformat}

h5. 2. corresponding error in receiving datanode
{noformat}
2009-02-21 01:23:56,608 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Exception in receiveBlock for block 
    blk_7545556036225037274_1820952 java.io.EOFException: while trying to read 65557 bytes
[...]
2009-02-21 01:23:59,076 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 0 for block 
    blk_7545556036225037274_1820952 Interrupted.
2009-02-21 01:24:01,484 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 0 for block 
    blk_7545556036225037274_1820952 terminating
2009-02-21 01:24:01,485 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: writeBlock 
   blk_7545556036225037274_1820952 received exception java.io.EOFException: while trying to read 65557 bytes
{noformat}

Since region server misses its lease to report to the master, it has to close all its regions before recover, and reopens them when the master asks for.
>From this time, it tries to _recover_ this block, as seen in the regionserver log:

h5. 3. Region server log in an endless loop to recover
{noformat}
009-02-21 01:22:29,327 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block 
    blk_7545556036225037274_1820952 bad datanode[1] 192.168.1.10:50010
2009-02-21 01:22:29,327 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block 
    blk_7545556036225037274_1820952 in pipeline 192.168.1.13:50010, 
   192.168.1.10:50010: bad datanode 192.168.1.10:50010
2009-02-21 01:22:29,689 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block 
    blk_7545556036225037274_1820952 failed  because recovery from primary datanode 192.168.1.13:50010 failed 2 times. Will retry...
{noformat}

To this _recover_ request, all datanodes fail with this Exception

h5. 4.
{noformat}
2009-02-21 01:24:18,650 INFO org.apache.hadoop.ipc.Server: IPC Server handler 1 on 50020, call 
   recoverBlock(blk_7545556036225037274_1820952, false, [Lorg.apache.hadoop.hdfs.protocol.DatanodeInfo;@311c4f) from 
   192.168.1.13:56968: error: org.apache.hadoop.ipc.RemoteException: java.io.IOException: 
   blk_7545556036225037274_1820952 is already commited, storedBlock == null.
        at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.nextGenerationStampForBlock(FSNamesystem.java:4536)
        at org.apache.hadoop.hdfs.server.namenode.NameNode.nextGenerationStamp(NameNode.java:402)
        at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)

org.apache.hadoop.ipc.RemoteException: java.io.IOException: blk_7545556036225037274_1820952 is already commited, 
   storedBlock == null.
        at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.nextGenerationStampForBlock(FSNamesystem.java:4536)
        at org.apache.hadoop.hdfs.server.namenode.NameNode.nextGenerationStamp(NameNode.java:402)
        at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)

        at org.apache.hadoop.ipc.Client.call(Client.java:696)
        at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:216)
        at $Proxy4.nextGenerationStamp(Unknown Source)
        at org.apache.hadoop.hdfs.server.datanode.DataNode.syncBlock(DataNode.java:1466)
        at org.apache.hadoop.hdfs.server.datanode.DataNode.recoverBlock(DataNode.java:1440)
        at org.apache.hadoop.hdfs.server.datanode.DataNode.recoverBlock(DataNode.java:1506)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
{noformat}

The problem is that, in this case, the RegionServer and therefore the master fails to stop when I launch the stop-hbase script.
A kill (-TERM) on the region server launches the SIGTERM handler thread, but the loop which try to reach the block does not stop.
here is a thread dump of the region server at this time:

h5. 5. Thread dump of the region server
{noformat}
Full thread dump Java HotSpot(TM) Server VM (10.0-b23 mixed mode):

"Thread-16" prio=10 tid=0x081c3c00 nid=0x6e81 in Object.wait() [0x8f74f000..0x8f74fec0]
   java.lang.Thread.State: WAITING (on object monitor)                                
        at java.lang.Object.wait(Native Method)                                        
        at java.lang.Thread.join(Thread.java:1143)                                    
        - locked <0x93d0d060> (a java.lang.Thread)                                    
        at java.lang.Thread.join(Thread.java:1196)                                    
        at org.apache.hadoop.hbase.util.Threads.shutdown(Threads.java:78)              
        at org.apache.hadoop.hbase.util.Threads.shutdown(Threads.java:66)              
        at org.apache.hadoop.hbase.regionserver.HRegionServer$ShutdownThread.run(HRegionServer.java:814)

"SIGTERM handler" daemon prio=10 tid=0x08dbcc00 nid=0x6e80 in Object.wait() [0x8edaf000..0x8edaff40]
   java.lang.Thread.State: WAITING (on object monitor)                                              
        at java.lang.Object.wait(Native Method)                                                    
        at java.lang.Thread.join(Thread.java:1143)                                                  
        - locked <0x93d0f3c0> (a org.apache.hadoop.hbase.regionserver.HRegionServer$ShutdownThread)
        at java.lang.Thread.join(Thread.java:1196)                                                  
        at java.lang.ApplicationShutdownHooks.run(ApplicationShutdownHooks.java:79)                
        at java.lang.Shutdown.runHooks(Shutdown.java:89)                                            
        at java.lang.Shutdown.sequence(Shutdown.java:133)                                          
        at java.lang.Shutdown.exit(Shutdown.java:178)                                              
        - locked <0xb0d3ed60> (a java.lang.Class for java.lang.Shutdown)                            
        at java.lang.Terminator$1.handle(Terminator.java:35)                                        
        at sun.misc.Signal$1.run(Signal.java:195)                                                  
        at java.lang.Thread.run(Thread.java:619)                                                    

"Attach Listener" daemon prio=10 tid=0x081e8800 nid=0x6e50 waiting on condition [0x00000000..0x00000000]
   java.lang.Thread.State: RUNNABLE                                                                    

"IPC Client (47) connection to /192.168.1.13:50020 from an unknown user" daemon prio=10 tid=0x8e97a400 nid=0x39d1 in Object.wait() [0x8ec81000..0x8ec81fc0]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                              
        at java.lang.Object.wait(Native Method)                                                                                                            
        at org.apache.hadoop.ipc.Client$Connection.waitForWork(Client.java:395)                                                                            
        - locked <0xa44a36a0> (a org.apache.hadoop.ipc.Client$Connection)                                                                                  
        at org.apache.hadoop.ipc.Client$Connection.run(Client.java:437)                                                                                    

"IPC Client (47) connection to /192.168.1.10:50020 from an unknown user" daemon prio=10 tid=0x8e5f1c00 nid=0x523f in Object.wait() [0x8ecd3000..0x8ecd3140]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                              
        at java.lang.Object.wait(Native Method)                                                                                                            
        at org.apache.hadoop.ipc.Client$Connection.waitForWork(Client.java:395)                                                                            
        - locked <0xa448ca50> (a org.apache.hadoop.ipc.Client$Connection)                                                                                  
        at org.apache.hadoop.ipc.Client$Connection.run(Client.java:437)                                                                                    

"BlockFSInputStreamReferenceQueueChecker" daemon prio=10 tid=0x085a9400 nid=0x8c8 waiting on condition [0x8ee6c000..0x8ee6cdc0]
   java.lang.Thread.State: WAITING (parking)                                                                                  
        at sun.misc.Unsafe.park(Native Method)                                                                                
        - parking to wait for  <0x93e76068> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)          
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                                  
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)  
        at java.util.concurrent.DelayQueue.take(DelayQueue.java:160)                                                          
        at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:582)        
        at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:575)        
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:946)                                        
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:906)                                    
        at java.lang.Thread.run(Thread.java:619)                                                                              

"IPC Server handler 9 on 60020" daemon prio=10 tid=0x0882e000 nid=0x889 waiting on condition [0x8eebd000..0x8eebdfc0]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 8 on 60020" daemon prio=10 tid=0x0882cc00 nid=0x888 waiting on condition [0x8ef0e000..0x8ef0f040]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 7 on 60020" daemon prio=10 tid=0x0882b800 nid=0x887 waiting on condition [0x8ef5f000..0x8ef5fec0]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 6 on 60020" daemon prio=10 tid=0x0882a400 nid=0x886 waiting on condition [0x8efb0000..0x8efb0f40]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 5 on 60020" daemon prio=10 tid=0x08829000 nid=0x885 waiting on condition [0x8f001000..0x8f001dc0]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 4 on 60020" daemon prio=10 tid=0x083bcc00 nid=0x884 waiting on condition [0x8f052000..0x8f052e40]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 3 on 60020" daemon prio=10 tid=0x083bb800 nid=0x883 waiting on condition [0x8f0a3000..0x8f0a40c0]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 2 on 60020" daemon prio=10 tid=0x083ba400 nid=0x882 waiting on condition [0x8f0f4000..0x8f0f5140]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 1 on 60020" daemon prio=10 tid=0x083b9400 nid=0x881 waiting on condition [0x8f145000..0x8f145fc0]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 0 on 60020" daemon prio=10 tid=0x083b8400 nid=0x880 waiting on condition [0x8f196000..0x8f197040]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server listener on 60020" daemon prio=10 tid=0x083bfc00 nid=0x87f runnable [0x8f1e7000..0x8f1e7ec0]
   java.lang.Thread.State: RUNNABLE                                                                    
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)                                        
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)                                
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)                            
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)                                
        - locked <0x93d0c090> (a sun.nio.ch.Util$1)                                                    
        - locked <0x93d0c120> (a java.util.Collections$UnmodifiableSet)                                
        - locked <0x93d0c0b0> (a sun.nio.ch.EPollSelectorImpl)                                          
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)                                        
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:84)                                        
        at org.apache.hadoop.hbase.ipc.HBaseServer$Listener.run(HBaseServer.java:299)                  

"IPC Server Responder" daemon prio=10 tid=0x083bec00 nid=0x87e runnable [0x8f238000..0x8f238f40]
   java.lang.Thread.State: RUNNABLE                                                            
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)                                
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)                        
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)                    
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)                        
        - locked <0x93d0c0f0> (a sun.nio.ch.Util$1)                                            
        - locked <0x93d0c100> (a java.util.Collections$UnmodifiableSet)                        
        - locked <0x93d0be30> (a sun.nio.ch.EPollSelectorImpl)                                  
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)                                
        at org.apache.hadoop.hbase.ipc.HBaseServer$Responder.run(HBaseServer.java:458)          

"SocketListener0-1" prio=10 tid=0x088d5000 nid=0x87c in Object.wait() [0x8f2da000..0x8f2dae40]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)                                  
        at java.lang.Object.wait(Native Method)                                              
        at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:522)                    
        - locked <0x93e39bc8> (a org.mortbay.util.ThreadPool$PoolThread)                      

"SocketListener0-0" prio=10 tid=0x08ab5000 nid=0x87b in Object.wait() [0x8f32b000..0x8f32c0c0]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)                                  
        at java.lang.Object.wait(Native Method)                                              
        at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:522)                    
        - locked <0x93e39920> (a org.mortbay.util.ThreadPool$PoolThread)                      

"Acceptor ServerSocket[addr=0.0.0.0/0.0.0.0,port=0,localport=60030]" prio=10 tid=0x084f6800 nid=0x87a 
   runnable [0x8f37c000..0x8f37d140]
   java.lang.Thread.State: RUNNABLE                                                                                                    
        at java.net.PlainSocketImpl.socketAccept(Native Method)                                                                        
        at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:384)                                                                  
        - locked <0x93e382a8> (a java.net.SocksSocketImpl)                                                                            
        at java.net.ServerSocket.implAccept(ServerSocket.java:453)                                                                    
        at java.net.ServerSocket.accept(ServerSocket.java:421)                                                                        
        at org.mortbay.util.ThreadedServer.acceptSocket(ThreadedServer.java:432)                                                      
        at org.mortbay.util.ThreadedServer$Acceptor.run(ThreadedServer.java:631)                                                      

"SessionScavenger" daemon prio=10 tid=0x088f1400 nid=0x879 waiting on condition [0x8f3cd000..0x8f3cdfc0]
   java.lang.Thread.State: TIMED_WAITING (sleeping)                                                    
        at java.lang.Thread.sleep(Native Method)                                                        
        at org.mortbay.jetty.servlet.AbstractSessionManager$SessionScavenger.run(AbstractSessionManager.java:587)

"SessionScavenger" daemon prio=10 tid=0x08ab5c00 nid=0x878 waiting on condition [0x8f41e000..0x8f41f040]
   java.lang.Thread.State: TIMED_WAITING (sleeping)                                                    
        at java.lang.Thread.sleep(Native Method)                                                        
        at org.mortbay.jetty.servlet.AbstractSessionManager$SessionScavenger.run(AbstractSessionManager.java:587)

"regionserver/0.0.0.0:60020.leaseChecker" prio=10 tid=0x086d5800 nid=0x877 waiting on condition [0x8f476000..0x8f476ec0]
   java.lang.Thread.State: TIMED_WAITING (parking)                                                                      
        at sun.misc.Unsafe.park(Native Method)                                                                          
        - parking to wait for  <0x93d0e980> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)  
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:198)                                      
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:1963)
        at java.util.concurrent.DelayQueue.poll(DelayQueue.java:201)                                                            
        at org.apache.hadoop.hbase.Leases.run(Leases.java:78)                                                                    

"regionserver/0.0.0.0:60020.majorCompactionChecker" daemon prio=10 tid=0x086d4000 nid=0x876 waiting on 
   condition [0x8f4c7000..0x8f4c7f40]
   java.lang.Thread.State: TIMED_WAITING (sleeping)                                                                                      
        at java.lang.Thread.sleep(Native Method)                                                                                        
        at org.apache.hadoop.hbase.util.Sleeper.sleep(Sleeper.java:74)                                                                  
        at org.apache.hadoop.hbase.Chore.run(Chore.java:72)                                                                              

"LeaseChecker" daemon prio=10 tid=0x0837cc00 nid=0x870 waiting on condition [0x8f6ad000..0x8f6ae040]
   java.lang.Thread.State: TIMED_WAITING (sleeping)                                                
        at java.lang.Thread.sleep(Native Method)                                                    
        at org.apache.hadoop.hdfs.DFSClient$LeaseChecker.run(DFSClient.java:979)                    
        at java.lang.Thread.run(Thread.java:619)                                                    

"DataStreamer for file /hbase/log_192.168.1.13_1235129194745_60020/hlog.dat.1235129195649 block blk_7545556036225037274_1820952" 
   daemon prio=10 tid=0x08375400 nid=0x86f in Object.wait() [0x8f6fe000..0x8f6feec0]                                                                                                      
   java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                                
        at java.lang.Object.wait(Native Method)                                                                                                            
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2166)                                                          
        - locked <0x93d0ea10> (a java.util.LinkedList)                                                                                                      

"DestroyJavaVM" prio=10 tid=0x0805a000 nid=0x855 waiting on condition [0x00000000..0xb7dd2090]
   java.lang.Thread.State: RUNNABLE                                                          

"regionserver/0.0.0.0:60020" prio=10 tid=0x085b2400 nid=0x863 in Object.wait() [0x8f7f7000..0x8f7f7f40]
   java.lang.Thread.State: WAITING (on object monitor)                                                
        at java.lang.Object.wait(Native Method)                                                        
        at java.lang.Object.wait(Object.java:485)                                                      
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.flushInternal(DFSClient.java:3015)        
        - locked <0x93d0ea10> (a java.util.LinkedList)                                                
        - locked <0x93d0ec58> (a org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)                    
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.closeInternal(DFSClient.java:3104)        
        - locked <0x93d0ec58> (a org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)                    
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.close(DFSClient.java:3053)
        at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:59)
        at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:79)
        at org.apache.hadoop.io.SequenceFile$Writer.close(SequenceFile.java:959)
        - locked <0x93d50d20> (a org.apache.hadoop.io.SequenceFile$Writer)
        at org.apache.hadoop.hbase.regionserver.HLog.close(HLog.java:421)
        - locked <0x93d4ce00> (a java.lang.Integer)
        at org.apache.hadoop.hbase.regionserver.HLog.closeAndDelete(HLog.java:404)
        at org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:373)
        at java.lang.Thread.run(Thread.java:619)

"Low Memory Detector" daemon prio=10 tid=0x080dd800 nid=0x85e runnable [0x00000000..0x00000000]
   java.lang.Thread.State: RUNNABLE

"CompilerThread1" daemon prio=10 tid=0x080dc400 nid=0x85d waiting on condition [0x00000000..0x8fbc1588]
   java.lang.Thread.State: RUNNABLE

"CompilerThread0" daemon prio=10 tid=0x080d9c00 nid=0x85c waiting on condition [0x00000000..0x8fc42608]
   java.lang.Thread.State: RUNNABLE

"Signal Dispatcher" daemon prio=10 tid=0x080d8800 nid=0x85b runnable [0x00000000..0x8fc93e80]
   java.lang.Thread.State: RUNNABLE

"Surrogate Locker Thread (CMS)" daemon prio=10 tid=0x080d7800 nid=0x85a waiting on condition [0x00000000..0x8fce522c]
   java.lang.Thread.State: RUNNABLE

"Finalizer" daemon prio=10 tid=0x080bbc00 nid=0x859 in Object.wait() [0x8fd7b000..0x8fd7bec0]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116)
        - locked <0x93d0bdc0> (a java.lang.ref.ReferenceQueue$Lock)
        at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:132)
        at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159)

"Reference Handler" daemon prio=10 tid=0x080ba800 nid=0x858 in Object.wait() [0x8fdcc000..0x8fdccf40]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at java.lang.Object.wait(Object.java:485)
        at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)
        - locked <0x93d0c110> (a java.lang.ref.Reference$Lock)

"VM Thread" prio=10 tid=0x080b7800 nid=0x857 runnable

"Concurrent Mark-Sweep GC Thread" prio=10 tid=0x0806c800 nid=0x856 runnable
"VM Periodic Task Thread" prio=10 tid=0x080df000 nid=0x85f waiting on condition

JNI global references: 922
{noformat}

Notice the DataStreamer thread that seems to work on the concerned block
This force me to kill (-KILL) the region server, and restart the hdfs (if I don't restart hdfs some regions are not available because the block is still not available).

Note that the Hbase master does not end as well, and I have to kill (-KILL). Here is the thread dump of the HMaster process:

h5. 6. Master thread dump:

{noformat}
Full thread dump Java HotSpot(TM) Server VM (10.0-b23 mixed mode):

"Attach Listener" daemon prio=10 tid=0x0805a000 nid=0x2326 waiting on condition [0x00000000..0x00000000]
   java.lang.Thread.State: RUNNABLE                                                                    

"SIGTERM handler" daemon prio=10 tid=0x089f0c00 nid=0x2303 waiting for monitor entry [0x8a6a5000..0x8a6a5ec0]
   java.lang.Thread.State: BLOCKED (on object monitor)                                                      
        at java.lang.Shutdown.exit(Shutdown.java:178)                                                        
        - waiting to lock <0xb0beed60> (a java.lang.Class for java.lang.Shutdown)                            
        at java.lang.Terminator$1.handle(Terminator.java:35)                                                
        at sun.misc.Signal$1.run(Signal.java:195)                                                            
        at java.lang.Thread.run(Thread.java:619)                                                            

"Thread-1" prio=10 tid=0x8f666000 nid=0x653c waiting on condition [0x8f1fe000..0x8f1fef40]
   java.lang.Thread.State: TIMED_WAITING (sleeping)                                      
        at java.lang.Thread.sleep(Native Method)                                          
        at org.apache.hadoop.ipc.Client.stop(Client.java:667)                            
        at org.apache.hadoop.ipc.RPC$ClientCache.stopClient(RPC.java:189)                
        at org.apache.hadoop.ipc.RPC$ClientCache.access$400(RPC.java:138)                
        at org.apache.hadoop.ipc.RPC$Invoker.close(RPC.java:229)                          
        - locked <0x938a1b18> (a org.apache.hadoop.ipc.RPC$Invoker)                      
        at org.apache.hadoop.ipc.RPC$Invoker.access$500(RPC.java:196)                    
        at org.apache.hadoop.ipc.RPC.stopProxy(RPC.java:353)                              
        at org.apache.hadoop.hdfs.DFSClient.close(DFSClient.java:213)                    
        - locked <0x93878470> (a org.apache.hadoop.hdfs.DFSClient)                        
        at org.apache.hadoop.hdfs.DistributedFileSystem.close(DistributedFileSystem.java:243)
        at org.apache.hadoop.fs.FileSystem$Cache.closeAll(FileSystem.java:1413)              
        - locked <0x937d3c00> (a org.apache.hadoop.fs.FileSystem$Cache)                      
        at org.apache.hadoop.fs.FileSystem.closeAll(FileSystem.java:236)                    
        at org.apache.hadoop.fs.FileSystem$ClientFinalizer.run(FileSystem.java:221)          
        - locked <0x937b8a78> (a org.apache.hadoop.fs.FileSystem$ClientFinalizer)            

"DestroyJavaVM" prio=10 tid=0x8f66c800 nid=0x36e9 in Object.wait() [0xb7d7d000..0xb7d7e0e0]
   java.lang.Thread.State: WAITING (on object monitor)                                    
        at java.lang.Object.wait(Native Method)                                            
        at java.lang.Thread.join(Thread.java:1143)                                        
        - locked <0x937b8a78> (a org.apache.hadoop.fs.FileSystem$ClientFinalizer)          
        at java.lang.Thread.join(Thread.java:1196)
        at java.lang.ApplicationShutdownHooks.run(ApplicationShutdownHooks.java:79)
        at java.lang.Shutdown.runHooks(Shutdown.java:89)
        at java.lang.Shutdown.sequence(Shutdown.java:133)
        at java.lang.Shutdown.shutdown(Shutdown.java:200)
        - locked <0xb0beed60> (a java.lang.Class for java.lang.Shutdown)

"Low Memory Detector" daemon prio=10 tid=0x8fe03800 nid=0x36f4 runnable [0x00000000..0x00000000]
   java.lang.Thread.State: RUNNABLE

"CompilerThread1" daemon prio=10 tid=0x8fe01c00 nid=0x36f3 waiting on condition [0x00000000..0x8f9ee488]
   java.lang.Thread.State: RUNNABLE

"CompilerThread0" daemon prio=10 tid=0x8fe00800 nid=0x36f2 waiting on condition [0x00000000..0x8fa6f508]
   java.lang.Thread.State: RUNNABLE

"Signal Dispatcher" daemon prio=10 tid=0x0813e800 nid=0x36f1 runnable [0x00000000..0x8fac0d80]
   java.lang.Thread.State: RUNNABLE

"Surrogate Locker Thread (CMS)" daemon prio=10 tid=0x0813d800 nid=0x36f0 waiting on condition [0x00000000..0x00000000]
   java.lang.Thread.State: RUNNABLE

"Finalizer" daemon prio=10 tid=0x08125000 nid=0x36ef in Object.wait() [0x8fba1000..0x8fba1dc0]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116)
        - locked <0x937bafd8> (a java.lang.ref.ReferenceQueue$Lock)
        at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:132)
        at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159)

"Reference Handler" daemon prio=10 tid=0x08120c00 nid=0x36ee in Object.wait() [0x8fbf2000..0x8fbf2e40]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at java.lang.Object.wait(Object.java:485)
        at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)
        - locked <0x937baf18> (a java.lang.ref.Reference$Lock)

"VM Thread" prio=10 tid=0x0811d800 nid=0x36ed runnable

"Gang worker#0 (Parallel GC Threads)" prio=10 tid=0x0805e000 nid=0x36ea runnable

"Gang worker#1 (Parallel GC Threads)" prio=10 tid=0x0805f000 nid=0x36eb runnable

"Concurrent Mark-Sweep GC Thread" prio=10 tid=0x080cd000 nid=0x36ec runnable
"VM Periodic Task Thread" prio=10 tid=0x8fe05000 nid=0x36f5 waiting on condition

JNI global references: 1044
{noformat}

When I restart dfs, the log shows me that the concerned block is marked as invalidate, and it is therefore deleted:

h5. 7. Namenode log
{noformat}
2009-02-23 10:33:49,245 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.processReport: block 
   blk_7545556036225037274_1820952 on 192.168.1.10:50010 size 27895296 does not
 belong to any file.
2009-02-23 10:33:49,245 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addToInvalidates: 
   blk_7545556036225037274 is added to invalidSet of 192.168.1.10:50010
{noformat}

Finally, when I restart HBase, I can see some of these messages in region server log, and I have some data loss. (this log entry may not be linked with the  block traced before)

h5. 8. Region server log:
{noformat}
2009-02-23 10:38:55,831 WARN org.apache.hadoop.hbase.regionserver.HStore: Exception processing reconstruction log 
   hdfs://lab5-2:9000/hbase/test-D-0.3/510178748/oldlogfile.log opening [B@ff65bf -- continuing.  Probably lack-of-HADOOP-1700 
   causing DATA LOSS!
java.io.EOFException
        at java.io.DataInputStream.readFully(DataInputStream.java:180)
        at org.apache.hadoop.hbase.io.DataOutputBuffer$Buffer.write(DataOutputBuffer.java:70)
        at org.apache.hadoop.hbase.io.DataOutputBuffer.write(DataOutputBuffer.java:116)
        at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1944)
        at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1844)
        at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1890)
        at org.apache.hadoop.hbase.regionserver.HStore.doReconstructionLog(HStore.java:352)
        at org.apache.hadoop.hbase.regionserver.HStore.runReconstructionLog(HStore.java:297)
        at org.apache.hadoop.hbase.regionserver.HStore.<init>(HStore.java:237)
        at org.apache.hadoop.hbase.regionserver.HRegion.instantiateHStore(HRegion.java:1764)
        at org.apache.hadoop.hbase.regionserver.HRegion.initialize(HRegion.java:276)
        at org.apache.hadoop.hbase.regionserver.HRegionServer.instantiateRegion(HRegionServer.java:1367)
        at org.apache.hadoop.hbase.regionserver.HRegionServer.openRegion(HRegionServer.java:1338)
        at org.apache.hadoop.hbase.regionserver.HRegionServer$Worker.run(HRegionServer.java:1253)
        at java.lang.Thread.run(Thread.java:619)
{noformat}

In my sense, since my hardware is in cause for RegionServer lease miss, I have to deal with the fact that I have to stop/restart hbase + hadoop in such a case. The problem in my sense is that a stop request should be able to perform a clean shutdown, and to avoid dataloss.

Hope this description helps.


  was:
One of my region server falls in a long GC time that get it unresponsive during about 10 minutes.
As I can see in the log, it seems that its DFSClient component was sending a file to hdfs, the sending times out when it recovers from GC:

h5. Region server log after recovering from GC
{noformat}
2009-02-21 01:22:26,454 WARN org.apache.hadoop.hdfs.DFSClient: DFSOutputStream ResponseProcessor exception  for block blk_7545556036225037274_1820952java.io.IOException: Bad response 1 for block blk_7545556036225037274_1820952 from datanode 192.168.1.10:50010
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:2342)
{noformat}

h5. corresponding error in receiving datanode
{noformat}
2009-02-21 01:23:56,608 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Exception in receiveBlock for block blk_7545556036225037274_1820952 java.io.EOFException: while trying to read 65557 bytes
[...]
2009-02-21 01:23:59,076 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 0 for block blk_7545556036225037274_1820952 Interrupted.
2009-02-21 01:24:01,484 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 0 for block blk_7545556036225037274_1820952 terminating
2009-02-21 01:24:01,485 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: writeBlock blk_7545556036225037274_1820952 received exception java.io.EOFException: while trying to read 65557 bytes
{noformat}

Since region server misses its lease to report to the master, it has to close all its regions before recover, and reopens them when the master asks for.
>From this time, it tries to _recover_ this block, as seen in the regionserver log:

h5. Region server log in an endless loop to recover
{noformat}
009-02-21 01:22:29,327 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_7545556036225037274_1820952 bad datanode[1] 192.168.1.10:50010
2009-02-21 01:22:29,327 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_7545556036225037274_1820952 in pipeline 192.168.1.13:50010, 192.
168.1.10:50010: bad datanode 192.168.1.10:50010
2009-02-21 01:22:29,689 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_7545556036225037274_1820952 failed  because recovery from primary datanode 192.168.1.13:50010 failed 2 times. Will retry...
{noformat}

To this _recover_ request, all datanodes fail with this Exception

{noformat}
2009-02-21 01:24:18,650 INFO org.apache.hadoop.ipc.Server: IPC Server handler 1 on 50020, call recoverBlock(blk_7545556036225037274_1820952, false, [Lorg.apache.hadoop.hdfs.protocol.DatanodeInfo;@311c4f) from 192.168.1.13:56968: error: org.apache.hadoop.ipc.RemoteException: java.io.IOException: blk_7545556036225037274_1820952 is already commited, storedBlock == null.
        at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.nextGenerationStampForBlock(FSNamesystem.java:4536)
        at org.apache.hadoop.hdfs.server.namenode.NameNode.nextGenerationStamp(NameNode.java:402)
        at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)

org.apache.hadoop.ipc.RemoteException: java.io.IOException: blk_7545556036225037274_1820952 is already commited, storedBlock == null.
        at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.nextGenerationStampForBlock(FSNamesystem.java:4536)
        at org.apache.hadoop.hdfs.server.namenode.NameNode.nextGenerationStamp(NameNode.java:402)
        at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)

        at org.apache.hadoop.ipc.Client.call(Client.java:696)
        at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:216)
        at $Proxy4.nextGenerationStamp(Unknown Source)
        at org.apache.hadoop.hdfs.server.datanode.DataNode.syncBlock(DataNode.java:1466)
        at org.apache.hadoop.hdfs.server.datanode.DataNode.recoverBlock(DataNode.java:1440)
        at org.apache.hadoop.hdfs.server.datanode.DataNode.recoverBlock(DataNode.java:1506)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
{noformat}

The problem is that, in this case, the RegionServer and therefore the master fails to stop when I launch the stop-hbase script.
A kill (-TERM) on the region server launches the SIGTERM handler thread, but the loop which try to reach the block does not stop.
here is a thread dump of the region server at this time:

h5. Thread dump of the region server
{noformat}
Full thread dump Java HotSpot(TM) Server VM (10.0-b23 mixed mode):

"Thread-16" prio=10 tid=0x081c3c00 nid=0x6e81 in Object.wait() [0x8f74f000..0x8f74fec0]
   java.lang.Thread.State: WAITING (on object monitor)                                
        at java.lang.Object.wait(Native Method)                                        
        at java.lang.Thread.join(Thread.java:1143)                                    
        - locked <0x93d0d060> (a java.lang.Thread)                                    
        at java.lang.Thread.join(Thread.java:1196)                                    
        at org.apache.hadoop.hbase.util.Threads.shutdown(Threads.java:78)              
        at org.apache.hadoop.hbase.util.Threads.shutdown(Threads.java:66)              
        at org.apache.hadoop.hbase.regionserver.HRegionServer$ShutdownThread.run(HRegionServer.java:814)

"SIGTERM handler" daemon prio=10 tid=0x08dbcc00 nid=0x6e80 in Object.wait() [0x8edaf000..0x8edaff40]
   java.lang.Thread.State: WAITING (on object monitor)                                              
        at java.lang.Object.wait(Native Method)                                                    
        at java.lang.Thread.join(Thread.java:1143)                                                  
        - locked <0x93d0f3c0> (a org.apache.hadoop.hbase.regionserver.HRegionServer$ShutdownThread)
        at java.lang.Thread.join(Thread.java:1196)                                                  
        at java.lang.ApplicationShutdownHooks.run(ApplicationShutdownHooks.java:79)                
        at java.lang.Shutdown.runHooks(Shutdown.java:89)                                            
        at java.lang.Shutdown.sequence(Shutdown.java:133)                                          
        at java.lang.Shutdown.exit(Shutdown.java:178)                                              
        - locked <0xb0d3ed60> (a java.lang.Class for java.lang.Shutdown)                            
        at java.lang.Terminator$1.handle(Terminator.java:35)                                        
        at sun.misc.Signal$1.run(Signal.java:195)                                                  
        at java.lang.Thread.run(Thread.java:619)                                                    

"Attach Listener" daemon prio=10 tid=0x081e8800 nid=0x6e50 waiting on condition [0x00000000..0x00000000]
   java.lang.Thread.State: RUNNABLE                                                                    

"IPC Client (47) connection to /192.168.1.13:50020 from an unknown user" daemon prio=10 tid=0x8e97a400 nid=0x39d1 in Object.wait() [0x8ec81000..0x8ec81fc0]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                              
        at java.lang.Object.wait(Native Method)                                                                                                            
        at org.apache.hadoop.ipc.Client$Connection.waitForWork(Client.java:395)                                                                            
        - locked <0xa44a36a0> (a org.apache.hadoop.ipc.Client$Connection)                                                                                  
        at org.apache.hadoop.ipc.Client$Connection.run(Client.java:437)                                                                                    

"IPC Client (47) connection to /192.168.1.10:50020 from an unknown user" daemon prio=10 tid=0x8e5f1c00 nid=0x523f in Object.wait() [0x8ecd3000..0x8ecd3140]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                              
        at java.lang.Object.wait(Native Method)                                                                                                            
        at org.apache.hadoop.ipc.Client$Connection.waitForWork(Client.java:395)                                                                            
        - locked <0xa448ca50> (a org.apache.hadoop.ipc.Client$Connection)                                                                                  
        at org.apache.hadoop.ipc.Client$Connection.run(Client.java:437)                                                                                    

"BlockFSInputStreamReferenceQueueChecker" daemon prio=10 tid=0x085a9400 nid=0x8c8 waiting on condition [0x8ee6c000..0x8ee6cdc0]
   java.lang.Thread.State: WAITING (parking)                                                                                  
        at sun.misc.Unsafe.park(Native Method)                                                                                
        - parking to wait for  <0x93e76068> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)          
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                                  
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)  
        at java.util.concurrent.DelayQueue.take(DelayQueue.java:160)                                                          
        at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:582)        
        at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:575)        
        at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:946)                                        
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:906)                                    
        at java.lang.Thread.run(Thread.java:619)                                                                              

"IPC Server handler 9 on 60020" daemon prio=10 tid=0x0882e000 nid=0x889 waiting on condition [0x8eebd000..0x8eebdfc0]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 8 on 60020" daemon prio=10 tid=0x0882cc00 nid=0x888 waiting on condition [0x8ef0e000..0x8ef0f040]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 7 on 60020" daemon prio=10 tid=0x0882b800 nid=0x887 waiting on condition [0x8ef5f000..0x8ef5fec0]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 6 on 60020" daemon prio=10 tid=0x0882a400 nid=0x886 waiting on condition [0x8efb0000..0x8efb0f40]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 5 on 60020" daemon prio=10 tid=0x08829000 nid=0x885 waiting on condition [0x8f001000..0x8f001dc0]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 4 on 60020" daemon prio=10 tid=0x083bcc00 nid=0x884 waiting on condition [0x8f052000..0x8f052e40]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 3 on 60020" daemon prio=10 tid=0x083bb800 nid=0x883 waiting on condition [0x8f0a3000..0x8f0a40c0]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 2 on 60020" daemon prio=10 tid=0x083ba400 nid=0x882 waiting on condition [0x8f0f4000..0x8f0f5140]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 1 on 60020" daemon prio=10 tid=0x083b9400 nid=0x881 waiting on condition [0x8f145000..0x8f145fc0]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server handler 0 on 60020" daemon prio=10 tid=0x083b8400 nid=0x880 waiting on condition [0x8f196000..0x8f197040]
   java.lang.Thread.State: WAITING (parking)                                                                        
        at sun.misc.Unsafe.park(Native Method)                                                                      
        - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
        at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        

"IPC Server listener on 60020" daemon prio=10 tid=0x083bfc00 nid=0x87f runnable [0x8f1e7000..0x8f1e7ec0]
   java.lang.Thread.State: RUNNABLE                                                                    
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)                                        
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)                                
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)                            
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)                                
        - locked <0x93d0c090> (a sun.nio.ch.Util$1)                                                    
        - locked <0x93d0c120> (a java.util.Collections$UnmodifiableSet)                                
        - locked <0x93d0c0b0> (a sun.nio.ch.EPollSelectorImpl)                                          
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)                                        
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:84)                                        
        at org.apache.hadoop.hbase.ipc.HBaseServer$Listener.run(HBaseServer.java:299)                  

"IPC Server Responder" daemon prio=10 tid=0x083bec00 nid=0x87e runnable [0x8f238000..0x8f238f40]
   java.lang.Thread.State: RUNNABLE                                                            
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)                                
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)                        
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)                    
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)                        
        - locked <0x93d0c0f0> (a sun.nio.ch.Util$1)                                            
        - locked <0x93d0c100> (a java.util.Collections$UnmodifiableSet)                        
        - locked <0x93d0be30> (a sun.nio.ch.EPollSelectorImpl)                                  
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)                                
        at org.apache.hadoop.hbase.ipc.HBaseServer$Responder.run(HBaseServer.java:458)          

"SocketListener0-1" prio=10 tid=0x088d5000 nid=0x87c in Object.wait() [0x8f2da000..0x8f2dae40]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)                                  
        at java.lang.Object.wait(Native Method)                                              
        at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:522)                    
        - locked <0x93e39bc8> (a org.mortbay.util.ThreadPool$PoolThread)                      

"SocketListener0-0" prio=10 tid=0x08ab5000 nid=0x87b in Object.wait() [0x8f32b000..0x8f32c0c0]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)                                  
        at java.lang.Object.wait(Native Method)                                              
        at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:522)                    
        - locked <0x93e39920> (a org.mortbay.util.ThreadPool$PoolThread)                      

"Acceptor ServerSocket[addr=0.0.0.0/0.0.0.0,port=0,localport=60030]" prio=10 tid=0x084f6800 nid=0x87a runnable [0x8f37c000..0x8f37d140]
   java.lang.Thread.State: RUNNABLE                                                                                                    
        at java.net.PlainSocketImpl.socketAccept(Native Method)                                                                        
        at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:384)                                                                  
        - locked <0x93e382a8> (a java.net.SocksSocketImpl)                                                                            
        at java.net.ServerSocket.implAccept(ServerSocket.java:453)                                                                    
        at java.net.ServerSocket.accept(ServerSocket.java:421)                                                                        
        at org.mortbay.util.ThreadedServer.acceptSocket(ThreadedServer.java:432)                                                      
        at org.mortbay.util.ThreadedServer$Acceptor.run(ThreadedServer.java:631)                                                      

"SessionScavenger" daemon prio=10 tid=0x088f1400 nid=0x879 waiting on condition [0x8f3cd000..0x8f3cdfc0]
   java.lang.Thread.State: TIMED_WAITING (sleeping)                                                    
        at java.lang.Thread.sleep(Native Method)                                                        
        at org.mortbay.jetty.servlet.AbstractSessionManager$SessionScavenger.run(AbstractSessionManager.java:587)

"SessionScavenger" daemon prio=10 tid=0x08ab5c00 nid=0x878 waiting on condition [0x8f41e000..0x8f41f040]
   java.lang.Thread.State: TIMED_WAITING (sleeping)                                                    
        at java.lang.Thread.sleep(Native Method)                                                        
        at org.mortbay.jetty.servlet.AbstractSessionManager$SessionScavenger.run(AbstractSessionManager.java:587)

"regionserver/0.0.0.0:60020.leaseChecker" prio=10 tid=0x086d5800 nid=0x877 waiting on condition [0x8f476000..0x8f476ec0]
   java.lang.Thread.State: TIMED_WAITING (parking)                                                                      
        at sun.misc.Unsafe.park(Native Method)                                                                          
        - parking to wait for  <0x93d0e980> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)  
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:198)                                      
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:1963)
        at java.util.concurrent.DelayQueue.poll(DelayQueue.java:201)                                                            
        at org.apache.hadoop.hbase.Leases.run(Leases.java:78)                                                                    

"regionserver/0.0.0.0:60020.majorCompactionChecker" daemon prio=10 tid=0x086d4000 nid=0x876 waiting on condition [0x8f4c7000..0x8f4c7f40]
   java.lang.Thread.State: TIMED_WAITING (sleeping)                                                                                      
        at java.lang.Thread.sleep(Native Method)                                                                                        
        at org.apache.hadoop.hbase.util.Sleeper.sleep(Sleeper.java:74)                                                                  
        at org.apache.hadoop.hbase.Chore.run(Chore.java:72)                                                                              

"LeaseChecker" daemon prio=10 tid=0x0837cc00 nid=0x870 waiting on condition [0x8f6ad000..0x8f6ae040]
   java.lang.Thread.State: TIMED_WAITING (sleeping)                                                
        at java.lang.Thread.sleep(Native Method)                                                    
        at org.apache.hadoop.hdfs.DFSClient$LeaseChecker.run(DFSClient.java:979)                    
        at java.lang.Thread.run(Thread.java:619)                                                    

"DataStreamer for file /hbase/log_192.168.1.13_1235129194745_60020/hlog.dat.1235129195649 block blk_7545556036225037274_1820952" daemon prio=10 tid=0x08375400 nid=0x86f in Object.wait() [0x8f6fe000..0x8f6feec0]                                                                                                      
   java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                                
        at java.lang.Object.wait(Native Method)                                                                                                            
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2166)                                                          
        - locked <0x93d0ea10> (a java.util.LinkedList)                                                                                                      

"DestroyJavaVM" prio=10 tid=0x0805a000 nid=0x855 waiting on condition [0x00000000..0xb7dd2090]
   java.lang.Thread.State: RUNNABLE                                                          

"regionserver/0.0.0.0:60020" prio=10 tid=0x085b2400 nid=0x863 in Object.wait() [0x8f7f7000..0x8f7f7f40]
   java.lang.Thread.State: WAITING (on object monitor)                                                
        at java.lang.Object.wait(Native Method)                                                        
        at java.lang.Object.wait(Object.java:485)                                                      
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.flushInternal(DFSClient.java:3015)        
        - locked <0x93d0ea10> (a java.util.LinkedList)                                                
        - locked <0x93d0ec58> (a org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)                    
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.closeInternal(DFSClient.java:3104)        
        - locked <0x93d0ec58> (a org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)                    
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.close(DFSClient.java:3053)
        at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:59)
        at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:79)
        at org.apache.hadoop.io.SequenceFile$Writer.close(SequenceFile.java:959)
        - locked <0x93d50d20> (a org.apache.hadoop.io.SequenceFile$Writer)
        at org.apache.hadoop.hbase.regionserver.HLog.close(HLog.java:421)
        - locked <0x93d4ce00> (a java.lang.Integer)
        at org.apache.hadoop.hbase.regionserver.HLog.closeAndDelete(HLog.java:404)
        at org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:373)
        at java.lang.Thread.run(Thread.java:619)

"Low Memory Detector" daemon prio=10 tid=0x080dd800 nid=0x85e runnable [0x00000000..0x00000000]
   java.lang.Thread.State: RUNNABLE

"CompilerThread1" daemon prio=10 tid=0x080dc400 nid=0x85d waiting on condition [0x00000000..0x8fbc1588]
   java.lang.Thread.State: RUNNABLE

"CompilerThread0" daemon prio=10 tid=0x080d9c00 nid=0x85c waiting on condition [0x00000000..0x8fc42608]
   java.lang.Thread.State: RUNNABLE

"Signal Dispatcher" daemon prio=10 tid=0x080d8800 nid=0x85b runnable [0x00000000..0x8fc93e80]
   java.lang.Thread.State: RUNNABLE

"Surrogate Locker Thread (CMS)" daemon prio=10 tid=0x080d7800 nid=0x85a waiting on condition [0x00000000..0x8fce522c]
   java.lang.Thread.State: RUNNABLE

"Finalizer" daemon prio=10 tid=0x080bbc00 nid=0x859 in Object.wait() [0x8fd7b000..0x8fd7bec0]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116)
        - locked <0x93d0bdc0> (a java.lang.ref.ReferenceQueue$Lock)
        at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:132)
        at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159)

"Reference Handler" daemon prio=10 tid=0x080ba800 nid=0x858 in Object.wait() [0x8fdcc000..0x8fdccf40]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at java.lang.Object.wait(Object.java:485)
        at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)
        - locked <0x93d0c110> (a java.lang.ref.Reference$Lock)

"VM Thread" prio=10 tid=0x080b7800 nid=0x857 runnable

"Concurrent Mark-Sweep GC Thread" prio=10 tid=0x0806c800 nid=0x856 runnable
"VM Periodic Task Thread" prio=10 tid=0x080df000 nid=0x85f waiting on condition

JNI global references: 922
{noformat}

Notice the DataStreamer thread that seems to work on the concerned block
This force me to kill (-KILL) the region server, and restart the hdfs (if I don't restart hdfs some regions are not available because the block is still not available).

Note that the Hbase master does not end as well, and I have to kill (-KILL). Here is the thread dump of the HMaster process:

h5. Master thread dump:

{noformat}
Full thread dump Java HotSpot(TM) Server VM (10.0-b23 mixed mode):

"Attach Listener" daemon prio=10 tid=0x0805a000 nid=0x2326 waiting on condition [0x00000000..0x00000000]
   java.lang.Thread.State: RUNNABLE                                                                    

"SIGTERM handler" daemon prio=10 tid=0x089f0c00 nid=0x2303 waiting for monitor entry [0x8a6a5000..0x8a6a5ec0]
   java.lang.Thread.State: BLOCKED (on object monitor)                                                      
        at java.lang.Shutdown.exit(Shutdown.java:178)                                                        
        - waiting to lock <0xb0beed60> (a java.lang.Class for java.lang.Shutdown)                            
        at java.lang.Terminator$1.handle(Terminator.java:35)                                                
        at sun.misc.Signal$1.run(Signal.java:195)                                                            
        at java.lang.Thread.run(Thread.java:619)                                                            

"Thread-1" prio=10 tid=0x8f666000 nid=0x653c waiting on condition [0x8f1fe000..0x8f1fef40]
   java.lang.Thread.State: TIMED_WAITING (sleeping)                                      
        at java.lang.Thread.sleep(Native Method)                                          
        at org.apache.hadoop.ipc.Client.stop(Client.java:667)                            
        at org.apache.hadoop.ipc.RPC$ClientCache.stopClient(RPC.java:189)                
        at org.apache.hadoop.ipc.RPC$ClientCache.access$400(RPC.java:138)                
        at org.apache.hadoop.ipc.RPC$Invoker.close(RPC.java:229)                          
        - locked <0x938a1b18> (a org.apache.hadoop.ipc.RPC$Invoker)                      
        at org.apache.hadoop.ipc.RPC$Invoker.access$500(RPC.java:196)                    
        at org.apache.hadoop.ipc.RPC.stopProxy(RPC.java:353)                              
        at org.apache.hadoop.hdfs.DFSClient.close(DFSClient.java:213)                    
        - locked <0x93878470> (a org.apache.hadoop.hdfs.DFSClient)                        
        at org.apache.hadoop.hdfs.DistributedFileSystem.close(DistributedFileSystem.java:243)
        at org.apache.hadoop.fs.FileSystem$Cache.closeAll(FileSystem.java:1413)              
        - locked <0x937d3c00> (a org.apache.hadoop.fs.FileSystem$Cache)                      
        at org.apache.hadoop.fs.FileSystem.closeAll(FileSystem.java:236)                    
        at org.apache.hadoop.fs.FileSystem$ClientFinalizer.run(FileSystem.java:221)          
        - locked <0x937b8a78> (a org.apache.hadoop.fs.FileSystem$ClientFinalizer)            

"DestroyJavaVM" prio=10 tid=0x8f66c800 nid=0x36e9 in Object.wait() [0xb7d7d000..0xb7d7e0e0]
   java.lang.Thread.State: WAITING (on object monitor)                                    
        at java.lang.Object.wait(Native Method)                                            
        at java.lang.Thread.join(Thread.java:1143)                                        
        - locked <0x937b8a78> (a org.apache.hadoop.fs.FileSystem$ClientFinalizer)          
        at java.lang.Thread.join(Thread.java:1196)
        at java.lang.ApplicationShutdownHooks.run(ApplicationShutdownHooks.java:79)
        at java.lang.Shutdown.runHooks(Shutdown.java:89)
        at java.lang.Shutdown.sequence(Shutdown.java:133)
        at java.lang.Shutdown.shutdown(Shutdown.java:200)
        - locked <0xb0beed60> (a java.lang.Class for java.lang.Shutdown)

"Low Memory Detector" daemon prio=10 tid=0x8fe03800 nid=0x36f4 runnable [0x00000000..0x00000000]
   java.lang.Thread.State: RUNNABLE

"CompilerThread1" daemon prio=10 tid=0x8fe01c00 nid=0x36f3 waiting on condition [0x00000000..0x8f9ee488]
   java.lang.Thread.State: RUNNABLE

"CompilerThread0" daemon prio=10 tid=0x8fe00800 nid=0x36f2 waiting on condition [0x00000000..0x8fa6f508]
   java.lang.Thread.State: RUNNABLE

"Signal Dispatcher" daemon prio=10 tid=0x0813e800 nid=0x36f1 runnable [0x00000000..0x8fac0d80]
   java.lang.Thread.State: RUNNABLE

"Surrogate Locker Thread (CMS)" daemon prio=10 tid=0x0813d800 nid=0x36f0 waiting on condition [0x00000000..0x00000000]
   java.lang.Thread.State: RUNNABLE

"Finalizer" daemon prio=10 tid=0x08125000 nid=0x36ef in Object.wait() [0x8fba1000..0x8fba1dc0]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116)
        - locked <0x937bafd8> (a java.lang.ref.ReferenceQueue$Lock)
        at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:132)
        at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159)

"Reference Handler" daemon prio=10 tid=0x08120c00 nid=0x36ee in Object.wait() [0x8fbf2000..0x8fbf2e40]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at java.lang.Object.wait(Object.java:485)
        at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)
        - locked <0x937baf18> (a java.lang.ref.Reference$Lock)

"VM Thread" prio=10 tid=0x0811d800 nid=0x36ed runnable

"Gang worker#0 (Parallel GC Threads)" prio=10 tid=0x0805e000 nid=0x36ea runnable

"Gang worker#1 (Parallel GC Threads)" prio=10 tid=0x0805f000 nid=0x36eb runnable

"Concurrent Mark-Sweep GC Thread" prio=10 tid=0x080cd000 nid=0x36ec runnable
"VM Periodic Task Thread" prio=10 tid=0x8fe05000 nid=0x36f5 waiting on condition

JNI global references: 1044
{noformat}

When I restart dfs, the log shows me that the concerned block is marked as invalidate, and it is therefore deleted:

h5. Namenode log
{noformat}
2009-02-23 10:33:49,245 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.processReport: block blk_7545556036225037274_1820952 on 192.168.1.10:50010 size 27895296 does not
 belong to any file.
2009-02-23 10:33:49,245 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addToInvalidates: blk_7545556036225037274 is added to invalidSet of 192.168.1.10:50010
{noformat}

Finally, when I restart HBase, I can see some of these messages in region server log, and I have some data loss. (this log entry may not be linked with the  block traced before)

h5. Region server log:
{noformat}
2009-02-23 10:38:55,831 WARN org.apache.hadoop.hbase.regionserver.HStore: Exception processing reconstruction log hdfs://lab5-2:9000/hbase/test-D-0.3/510178748/oldlogfile.log opening [B@ff65bf -- continuing.  Probably lack-of-HADOOP-1700 causing DATA LOSS!
java.io.EOFException
        at java.io.DataInputStream.readFully(DataInputStream.java:180)
        at org.apache.hadoop.hbase.io.DataOutputBuffer$Buffer.write(DataOutputBuffer.java:70)
        at org.apache.hadoop.hbase.io.DataOutputBuffer.write(DataOutputBuffer.java:116)
        at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1944)
        at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1844)
        at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1890)
        at org.apache.hadoop.hbase.regionserver.HStore.doReconstructionLog(HStore.java:352)
        at org.apache.hadoop.hbase.regionserver.HStore.runReconstructionLog(HStore.java:297)
        at org.apache.hadoop.hbase.regionserver.HStore.<init>(HStore.java:237)
        at org.apache.hadoop.hbase.regionserver.HRegion.instantiateHStore(HRegion.java:1764)
        at org.apache.hadoop.hbase.regionserver.HRegion.initialize(HRegion.java:276)
        at org.apache.hadoop.hbase.regionserver.HRegionServer.instantiateRegion(HRegionServer.java:1367)
        at org.apache.hadoop.hbase.regionserver.HRegionServer.openRegion(HRegionServer.java:1338)
        at org.apache.hadoop.hbase.regionserver.HRegionServer$Worker.run(HRegionServer.java:1253)
        at java.lang.Thread.run(Thread.java:619)
{noformat}

In my sense, since my hardware is in cause for RegionServer lease miss, I have to deal with the fact that I have to stop/restart hbase + hadoop in such a case. The problem in my sense is that a stop request should be able to perform a clean shutdown, and to avoid dataloss.

Hope this description helps.



The only IOException I have in regionserver log is the orignial one when the block write fails because of a timeout *log extract #1*

Notice that the error *log extract #3* loops until I kill the regionserver

(I put numbering and some line break in the description above)

> HRegionServer fails to stop if hdfs client it tries to recover a block 
> -----------------------------------------------------------------------
>
>                 Key: HBASE-1214
>                 URL: https://issues.apache.org/jira/browse/HBASE-1214
>             Project: Hadoop HBase
>          Issue Type: Bug
>          Components: master, regionserver
>    Affects Versions: 0.19.0
>         Environment: 4 Node cluster with poor hardware: (DN+RS / DN+RS / RS / MA+NN)
> 1 Gb memory each 
> Ubuntu linux
> Java 6
>            Reporter: Jean-Adrien
>            Priority: Critical
>
> One of my region server falls in a long GC time that get it unresponsive during about 10 minutes.
> As I can see in the log, it seems that its DFSClient component was sending a file to hdfs, the sending times out when it recovers from GC:
> h5. 1. Region server log after recovering from GC
> {noformat}
> 2009-02-21 01:22:26,454 WARN org.apache.hadoop.hdfs.DFSClient: DFSOutputStream ResponseProcessor exception  
>     for block blk_7545556036225037274_1820952java.io.IOException: Bad response 1 for block 
>     blk_7545556036225037274_1820952 from datanode 192.168.1.10:50010
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:2342)
> {noformat}
> h5. 2. corresponding error in receiving datanode
> {noformat}
> 2009-02-21 01:23:56,608 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Exception in receiveBlock for block 
>     blk_7545556036225037274_1820952 java.io.EOFException: while trying to read 65557 bytes
> [...]
> 2009-02-21 01:23:59,076 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 0 for block 
>     blk_7545556036225037274_1820952 Interrupted.
> 2009-02-21 01:24:01,484 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 0 for block 
>     blk_7545556036225037274_1820952 terminating
> 2009-02-21 01:24:01,485 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: writeBlock 
>    blk_7545556036225037274_1820952 received exception java.io.EOFException: while trying to read 65557 bytes
> {noformat}
> Since region server misses its lease to report to the master, it has to close all its regions before recover, and reopens them when the master asks for.
> From this time, it tries to _recover_ this block, as seen in the regionserver log:
> h5. 3. Region server log in an endless loop to recover
> {noformat}
> 009-02-21 01:22:29,327 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block 
>     blk_7545556036225037274_1820952 bad datanode[1] 192.168.1.10:50010
> 2009-02-21 01:22:29,327 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block 
>     blk_7545556036225037274_1820952 in pipeline 192.168.1.13:50010, 
>    192.168.1.10:50010: bad datanode 192.168.1.10:50010
> 2009-02-21 01:22:29,689 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block 
>     blk_7545556036225037274_1820952 failed  because recovery from primary datanode 192.168.1.13:50010 failed 2 times. Will retry...
> {noformat}
> To this _recover_ request, all datanodes fail with this Exception
> h5. 4.
> {noformat}
> 2009-02-21 01:24:18,650 INFO org.apache.hadoop.ipc.Server: IPC Server handler 1 on 50020, call 
>    recoverBlock(blk_7545556036225037274_1820952, false, [Lorg.apache.hadoop.hdfs.protocol.DatanodeInfo;@311c4f) from 
>    192.168.1.13:56968: error: org.apache.hadoop.ipc.RemoteException: java.io.IOException: 
>    blk_7545556036225037274_1820952 is already commited, storedBlock == null.
>         at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.nextGenerationStampForBlock(FSNamesystem.java:4536)
>         at org.apache.hadoop.hdfs.server.namenode.NameNode.nextGenerationStamp(NameNode.java:402)
>         at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
> org.apache.hadoop.ipc.RemoteException: java.io.IOException: blk_7545556036225037274_1820952 is already commited, 
>    storedBlock == null.
>         at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.nextGenerationStampForBlock(FSNamesystem.java:4536)
>         at org.apache.hadoop.hdfs.server.namenode.NameNode.nextGenerationStamp(NameNode.java:402)
>         at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
>         at org.apache.hadoop.ipc.Client.call(Client.java:696)
>         at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:216)
>         at $Proxy4.nextGenerationStamp(Unknown Source)
>         at org.apache.hadoop.hdfs.server.datanode.DataNode.syncBlock(DataNode.java:1466)
>         at org.apache.hadoop.hdfs.server.datanode.DataNode.recoverBlock(DataNode.java:1440)
>         at org.apache.hadoop.hdfs.server.datanode.DataNode.recoverBlock(DataNode.java:1506)
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
> {noformat}
> The problem is that, in this case, the RegionServer and therefore the master fails to stop when I launch the stop-hbase script.
> A kill (-TERM) on the region server launches the SIGTERM handler thread, but the loop which try to reach the block does not stop.
> here is a thread dump of the region server at this time:
> h5. 5. Thread dump of the region server
> {noformat}
> Full thread dump Java HotSpot(TM) Server VM (10.0-b23 mixed mode):
> "Thread-16" prio=10 tid=0x081c3c00 nid=0x6e81 in Object.wait() [0x8f74f000..0x8f74fec0]
>    java.lang.Thread.State: WAITING (on object monitor)                                
>         at java.lang.Object.wait(Native Method)                                        
>         at java.lang.Thread.join(Thread.java:1143)                                    
>         - locked <0x93d0d060> (a java.lang.Thread)                                    
>         at java.lang.Thread.join(Thread.java:1196)                                    
>         at org.apache.hadoop.hbase.util.Threads.shutdown(Threads.java:78)              
>         at org.apache.hadoop.hbase.util.Threads.shutdown(Threads.java:66)              
>         at org.apache.hadoop.hbase.regionserver.HRegionServer$ShutdownThread.run(HRegionServer.java:814)
> "SIGTERM handler" daemon prio=10 tid=0x08dbcc00 nid=0x6e80 in Object.wait() [0x8edaf000..0x8edaff40]
>    java.lang.Thread.State: WAITING (on object monitor)                                              
>         at java.lang.Object.wait(Native Method)                                                    
>         at java.lang.Thread.join(Thread.java:1143)                                                  
>         - locked <0x93d0f3c0> (a org.apache.hadoop.hbase.regionserver.HRegionServer$ShutdownThread)
>         at java.lang.Thread.join(Thread.java:1196)                                                  
>         at java.lang.ApplicationShutdownHooks.run(ApplicationShutdownHooks.java:79)                
>         at java.lang.Shutdown.runHooks(Shutdown.java:89)                                            
>         at java.lang.Shutdown.sequence(Shutdown.java:133)                                          
>         at java.lang.Shutdown.exit(Shutdown.java:178)                                              
>         - locked <0xb0d3ed60> (a java.lang.Class for java.lang.Shutdown)                            
>         at java.lang.Terminator$1.handle(Terminator.java:35)                                        
>         at sun.misc.Signal$1.run(Signal.java:195)                                                  
>         at java.lang.Thread.run(Thread.java:619)                                                    
> "Attach Listener" daemon prio=10 tid=0x081e8800 nid=0x6e50 waiting on condition [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE                                                                    
> "IPC Client (47) connection to /192.168.1.13:50020 from an unknown user" daemon prio=10 tid=0x8e97a400 nid=0x39d1 in Object.wait() [0x8ec81000..0x8ec81fc0]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                              
>         at java.lang.Object.wait(Native Method)                                                                                                            
>         at org.apache.hadoop.ipc.Client$Connection.waitForWork(Client.java:395)                                                                            
>         - locked <0xa44a36a0> (a org.apache.hadoop.ipc.Client$Connection)                                                                                  
>         at org.apache.hadoop.ipc.Client$Connection.run(Client.java:437)                                                                                    
> "IPC Client (47) connection to /192.168.1.10:50020 from an unknown user" daemon prio=10 tid=0x8e5f1c00 nid=0x523f in Object.wait() [0x8ecd3000..0x8ecd3140]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                              
>         at java.lang.Object.wait(Native Method)                                                                                                            
>         at org.apache.hadoop.ipc.Client$Connection.waitForWork(Client.java:395)                                                                            
>         - locked <0xa448ca50> (a org.apache.hadoop.ipc.Client$Connection)                                                                                  
>         at org.apache.hadoop.ipc.Client$Connection.run(Client.java:437)                                                                                    
> "BlockFSInputStreamReferenceQueueChecker" daemon prio=10 tid=0x085a9400 nid=0x8c8 waiting on condition [0x8ee6c000..0x8ee6cdc0]
>    java.lang.Thread.State: WAITING (parking)                                                                                  
>         at sun.misc.Unsafe.park(Native Method)                                                                                
>         - parking to wait for  <0x93e76068> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)          
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                                  
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)  
>         at java.util.concurrent.DelayQueue.take(DelayQueue.java:160)                                                          
>         at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:582)        
>         at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:575)        
>         at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:946)                                        
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:906)                                    
>         at java.lang.Thread.run(Thread.java:619)                                                                              
> "IPC Server handler 9 on 60020" daemon prio=10 tid=0x0882e000 nid=0x889 waiting on condition [0x8eebd000..0x8eebdfc0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 8 on 60020" daemon prio=10 tid=0x0882cc00 nid=0x888 waiting on condition [0x8ef0e000..0x8ef0f040]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 7 on 60020" daemon prio=10 tid=0x0882b800 nid=0x887 waiting on condition [0x8ef5f000..0x8ef5fec0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 6 on 60020" daemon prio=10 tid=0x0882a400 nid=0x886 waiting on condition [0x8efb0000..0x8efb0f40]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 5 on 60020" daemon prio=10 tid=0x08829000 nid=0x885 waiting on condition [0x8f001000..0x8f001dc0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 4 on 60020" daemon prio=10 tid=0x083bcc00 nid=0x884 waiting on condition [0x8f052000..0x8f052e40]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 3 on 60020" daemon prio=10 tid=0x083bb800 nid=0x883 waiting on condition [0x8f0a3000..0x8f0a40c0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 2 on 60020" daemon prio=10 tid=0x083ba400 nid=0x882 waiting on condition [0x8f0f4000..0x8f0f5140]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 1 on 60020" daemon prio=10 tid=0x083b9400 nid=0x881 waiting on condition [0x8f145000..0x8f145fc0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 0 on 60020" daemon prio=10 tid=0x083b8400 nid=0x880 waiting on condition [0x8f196000..0x8f197040]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server listener on 60020" daemon prio=10 tid=0x083bfc00 nid=0x87f runnable [0x8f1e7000..0x8f1e7ec0]
>    java.lang.Thread.State: RUNNABLE                                                                    
>         at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)                                        
>         at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)                                
>         at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)                            
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)                                
>         - locked <0x93d0c090> (a sun.nio.ch.Util$1)                                                    
>         - locked <0x93d0c120> (a java.util.Collections$UnmodifiableSet)                                
>         - locked <0x93d0c0b0> (a sun.nio.ch.EPollSelectorImpl)                                          
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)                                        
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:84)                                        
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Listener.run(HBaseServer.java:299)                  
> "IPC Server Responder" daemon prio=10 tid=0x083bec00 nid=0x87e runnable [0x8f238000..0x8f238f40]
>    java.lang.Thread.State: RUNNABLE                                                            
>         at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)                                
>         at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)                        
>         at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)                    
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)                        
>         - locked <0x93d0c0f0> (a sun.nio.ch.Util$1)                                            
>         - locked <0x93d0c100> (a java.util.Collections$UnmodifiableSet)                        
>         - locked <0x93d0be30> (a sun.nio.ch.EPollSelectorImpl)                                  
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)                                
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Responder.run(HBaseServer.java:458)          
> "SocketListener0-1" prio=10 tid=0x088d5000 nid=0x87c in Object.wait() [0x8f2da000..0x8f2dae40]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                  
>         at java.lang.Object.wait(Native Method)                                              
>         at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:522)                    
>         - locked <0x93e39bc8> (a org.mortbay.util.ThreadPool$PoolThread)                      
> "SocketListener0-0" prio=10 tid=0x08ab5000 nid=0x87b in Object.wait() [0x8f32b000..0x8f32c0c0]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                  
>         at java.lang.Object.wait(Native Method)                                              
>         at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:522)                    
>         - locked <0x93e39920> (a org.mortbay.util.ThreadPool$PoolThread)                      
> "Acceptor ServerSocket[addr=0.0.0.0/0.0.0.0,port=0,localport=60030]" prio=10 tid=0x084f6800 nid=0x87a 
>    runnable [0x8f37c000..0x8f37d140]
>    java.lang.Thread.State: RUNNABLE                                                                                                    
>         at java.net.PlainSocketImpl.socketAccept(Native Method)                                                                        
>         at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:384)                                                                  
>         - locked <0x93e382a8> (a java.net.SocksSocketImpl)                                                                            
>         at java.net.ServerSocket.implAccept(ServerSocket.java:453)                                                                    
>         at java.net.ServerSocket.accept(ServerSocket.java:421)                                                                        
>         at org.mortbay.util.ThreadedServer.acceptSocket(ThreadedServer.java:432)                                                      
>         at org.mortbay.util.ThreadedServer$Acceptor.run(ThreadedServer.java:631)                                                      
> "SessionScavenger" daemon prio=10 tid=0x088f1400 nid=0x879 waiting on condition [0x8f3cd000..0x8f3cdfc0]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                    
>         at java.lang.Thread.sleep(Native Method)                                                        
>         at org.mortbay.jetty.servlet.AbstractSessionManager$SessionScavenger.run(AbstractSessionManager.java:587)
> "SessionScavenger" daemon prio=10 tid=0x08ab5c00 nid=0x878 waiting on condition [0x8f41e000..0x8f41f040]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                    
>         at java.lang.Thread.sleep(Native Method)                                                        
>         at org.mortbay.jetty.servlet.AbstractSessionManager$SessionScavenger.run(AbstractSessionManager.java:587)
> "regionserver/0.0.0.0:60020.leaseChecker" prio=10 tid=0x086d5800 nid=0x877 waiting on condition [0x8f476000..0x8f476ec0]
>    java.lang.Thread.State: TIMED_WAITING (parking)                                                                      
>         at sun.misc.Unsafe.park(Native Method)                                                                          
>         - parking to wait for  <0x93d0e980> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)  
>         at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:198)                                      
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:1963)
>         at java.util.concurrent.DelayQueue.poll(DelayQueue.java:201)                                                            
>         at org.apache.hadoop.hbase.Leases.run(Leases.java:78)                                                                    
> "regionserver/0.0.0.0:60020.majorCompactionChecker" daemon prio=10 tid=0x086d4000 nid=0x876 waiting on 
>    condition [0x8f4c7000..0x8f4c7f40]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                                                      
>         at java.lang.Thread.sleep(Native Method)                                                                                        
>         at org.apache.hadoop.hbase.util.Sleeper.sleep(Sleeper.java:74)                                                                  
>         at org.apache.hadoop.hbase.Chore.run(Chore.java:72)                                                                              
> "LeaseChecker" daemon prio=10 tid=0x0837cc00 nid=0x870 waiting on condition [0x8f6ad000..0x8f6ae040]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                
>         at java.lang.Thread.sleep(Native Method)                                                    
>         at org.apache.hadoop.hdfs.DFSClient$LeaseChecker.run(DFSClient.java:979)                    
>         at java.lang.Thread.run(Thread.java:619)                                                    
> "DataStreamer for file /hbase/log_192.168.1.13_1235129194745_60020/hlog.dat.1235129195649 block blk_7545556036225037274_1820952" 
>    daemon prio=10 tid=0x08375400 nid=0x86f in Object.wait() [0x8f6fe000..0x8f6feec0]                                                                                                      
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                                
>         at java.lang.Object.wait(Native Method)                                                                                                            
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2166)                                                          
>         - locked <0x93d0ea10> (a java.util.LinkedList)                                                                                                      
> "DestroyJavaVM" prio=10 tid=0x0805a000 nid=0x855 waiting on condition [0x00000000..0xb7dd2090]
>    java.lang.Thread.State: RUNNABLE                                                          
> "regionserver/0.0.0.0:60020" prio=10 tid=0x085b2400 nid=0x863 in Object.wait() [0x8f7f7000..0x8f7f7f40]
>    java.lang.Thread.State: WAITING (on object monitor)                                                
>         at java.lang.Object.wait(Native Method)                                                        
>         at java.lang.Object.wait(Object.java:485)                                                      
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.flushInternal(DFSClient.java:3015)        
>         - locked <0x93d0ea10> (a java.util.LinkedList)                                                
>         - locked <0x93d0ec58> (a org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)                    
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.closeInternal(DFSClient.java:3104)        
>         - locked <0x93d0ec58> (a org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)                    
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.close(DFSClient.java:3053)
>         at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:59)
>         at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:79)
>         at org.apache.hadoop.io.SequenceFile$Writer.close(SequenceFile.java:959)
>         - locked <0x93d50d20> (a org.apache.hadoop.io.SequenceFile$Writer)
>         at org.apache.hadoop.hbase.regionserver.HLog.close(HLog.java:421)
>         - locked <0x93d4ce00> (a java.lang.Integer)
>         at org.apache.hadoop.hbase.regionserver.HLog.closeAndDelete(HLog.java:404)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:373)
>         at java.lang.Thread.run(Thread.java:619)
> "Low Memory Detector" daemon prio=10 tid=0x080dd800 nid=0x85e runnable [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread1" daemon prio=10 tid=0x080dc400 nid=0x85d waiting on condition [0x00000000..0x8fbc1588]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread0" daemon prio=10 tid=0x080d9c00 nid=0x85c waiting on condition [0x00000000..0x8fc42608]
>    java.lang.Thread.State: RUNNABLE
> "Signal Dispatcher" daemon prio=10 tid=0x080d8800 nid=0x85b runnable [0x00000000..0x8fc93e80]
>    java.lang.Thread.State: RUNNABLE
> "Surrogate Locker Thread (CMS)" daemon prio=10 tid=0x080d7800 nid=0x85a waiting on condition [0x00000000..0x8fce522c]
>    java.lang.Thread.State: RUNNABLE
> "Finalizer" daemon prio=10 tid=0x080bbc00 nid=0x859 in Object.wait() [0x8fd7b000..0x8fd7bec0]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116)
>         - locked <0x93d0bdc0> (a java.lang.ref.ReferenceQueue$Lock)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:132)
>         at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159)
> "Reference Handler" daemon prio=10 tid=0x080ba800 nid=0x858 in Object.wait() [0x8fdcc000..0x8fdccf40]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.Object.wait(Object.java:485)
>         at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)
>         - locked <0x93d0c110> (a java.lang.ref.Reference$Lock)
> "VM Thread" prio=10 tid=0x080b7800 nid=0x857 runnable
> "Concurrent Mark-Sweep GC Thread" prio=10 tid=0x0806c800 nid=0x856 runnable
> "VM Periodic Task Thread" prio=10 tid=0x080df000 nid=0x85f waiting on condition
> JNI global references: 922
> {noformat}
> Notice the DataStreamer thread that seems to work on the concerned block
> This force me to kill (-KILL) the region server, and restart the hdfs (if I don't restart hdfs some regions are not available because the block is still not available).
> Note that the Hbase master does not end as well, and I have to kill (-KILL). Here is the thread dump of the HMaster process:
> h5. 6. Master thread dump:
> {noformat}
> Full thread dump Java HotSpot(TM) Server VM (10.0-b23 mixed mode):
> "Attach Listener" daemon prio=10 tid=0x0805a000 nid=0x2326 waiting on condition [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE                                                                    
> "SIGTERM handler" daemon prio=10 tid=0x089f0c00 nid=0x2303 waiting for monitor entry [0x8a6a5000..0x8a6a5ec0]
>    java.lang.Thread.State: BLOCKED (on object monitor)                                                      
>         at java.lang.Shutdown.exit(Shutdown.java:178)                                                        
>         - waiting to lock <0xb0beed60> (a java.lang.Class for java.lang.Shutdown)                            
>         at java.lang.Terminator$1.handle(Terminator.java:35)                                                
>         at sun.misc.Signal$1.run(Signal.java:195)                                                            
>         at java.lang.Thread.run(Thread.java:619)                                                            
> "Thread-1" prio=10 tid=0x8f666000 nid=0x653c waiting on condition [0x8f1fe000..0x8f1fef40]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                      
>         at java.lang.Thread.sleep(Native Method)                                          
>         at org.apache.hadoop.ipc.Client.stop(Client.java:667)                            
>         at org.apache.hadoop.ipc.RPC$ClientCache.stopClient(RPC.java:189)                
>         at org.apache.hadoop.ipc.RPC$ClientCache.access$400(RPC.java:138)                
>         at org.apache.hadoop.ipc.RPC$Invoker.close(RPC.java:229)                          
>         - locked <0x938a1b18> (a org.apache.hadoop.ipc.RPC$Invoker)                      
>         at org.apache.hadoop.ipc.RPC$Invoker.access$500(RPC.java:196)                    
>         at org.apache.hadoop.ipc.RPC.stopProxy(RPC.java:353)                              
>         at org.apache.hadoop.hdfs.DFSClient.close(DFSClient.java:213)                    
>         - locked <0x93878470> (a org.apache.hadoop.hdfs.DFSClient)                        
>         at org.apache.hadoop.hdfs.DistributedFileSystem.close(DistributedFileSystem.java:243)
>         at org.apache.hadoop.fs.FileSystem$Cache.closeAll(FileSystem.java:1413)              
>         - locked <0x937d3c00> (a org.apache.hadoop.fs.FileSystem$Cache)                      
>         at org.apache.hadoop.fs.FileSystem.closeAll(FileSystem.java:236)                    
>         at org.apache.hadoop.fs.FileSystem$ClientFinalizer.run(FileSystem.java:221)          
>         - locked <0x937b8a78> (a org.apache.hadoop.fs.FileSystem$ClientFinalizer)            
> "DestroyJavaVM" prio=10 tid=0x8f66c800 nid=0x36e9 in Object.wait() [0xb7d7d000..0xb7d7e0e0]
>    java.lang.Thread.State: WAITING (on object monitor)                                    
>         at java.lang.Object.wait(Native Method)                                            
>         at java.lang.Thread.join(Thread.java:1143)                                        
>         - locked <0x937b8a78> (a org.apache.hadoop.fs.FileSystem$ClientFinalizer)          
>         at java.lang.Thread.join(Thread.java:1196)
>         at java.lang.ApplicationShutdownHooks.run(ApplicationShutdownHooks.java:79)
>         at java.lang.Shutdown.runHooks(Shutdown.java:89)
>         at java.lang.Shutdown.sequence(Shutdown.java:133)
>         at java.lang.Shutdown.shutdown(Shutdown.java:200)
>         - locked <0xb0beed60> (a java.lang.Class for java.lang.Shutdown)
> "Low Memory Detector" daemon prio=10 tid=0x8fe03800 nid=0x36f4 runnable [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread1" daemon prio=10 tid=0x8fe01c00 nid=0x36f3 waiting on condition [0x00000000..0x8f9ee488]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread0" daemon prio=10 tid=0x8fe00800 nid=0x36f2 waiting on condition [0x00000000..0x8fa6f508]
>    java.lang.Thread.State: RUNNABLE
> "Signal Dispatcher" daemon prio=10 tid=0x0813e800 nid=0x36f1 runnable [0x00000000..0x8fac0d80]
>    java.lang.Thread.State: RUNNABLE
> "Surrogate Locker Thread (CMS)" daemon prio=10 tid=0x0813d800 nid=0x36f0 waiting on condition [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE
> "Finalizer" daemon prio=10 tid=0x08125000 nid=0x36ef in Object.wait() [0x8fba1000..0x8fba1dc0]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116)
>         - locked <0x937bafd8> (a java.lang.ref.ReferenceQueue$Lock)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:132)
>         at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159)
> "Reference Handler" daemon prio=10 tid=0x08120c00 nid=0x36ee in Object.wait() [0x8fbf2000..0x8fbf2e40]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.Object.wait(Object.java:485)
>         at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)
>         - locked <0x937baf18> (a java.lang.ref.Reference$Lock)
> "VM Thread" prio=10 tid=0x0811d800 nid=0x36ed runnable
> "Gang worker#0 (Parallel GC Threads)" prio=10 tid=0x0805e000 nid=0x36ea runnable
> "Gang worker#1 (Parallel GC Threads)" prio=10 tid=0x0805f000 nid=0x36eb runnable
> "Concurrent Mark-Sweep GC Thread" prio=10 tid=0x080cd000 nid=0x36ec runnable
> "VM Periodic Task Thread" prio=10 tid=0x8fe05000 nid=0x36f5 waiting on condition
> JNI global references: 1044
> {noformat}
> When I restart dfs, the log shows me that the concerned block is marked as invalidate, and it is therefore deleted:
> h5. 7. Namenode log
> {noformat}
> 2009-02-23 10:33:49,245 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.processReport: block 
>    blk_7545556036225037274_1820952 on 192.168.1.10:50010 size 27895296 does not
>  belong to any file.
> 2009-02-23 10:33:49,245 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addToInvalidates: 
>    blk_7545556036225037274 is added to invalidSet of 192.168.1.10:50010
> {noformat}
> Finally, when I restart HBase, I can see some of these messages in region server log, and I have some data loss. (this log entry may not be linked with the  block traced before)
> h5. 8. Region server log:
> {noformat}
> 2009-02-23 10:38:55,831 WARN org.apache.hadoop.hbase.regionserver.HStore: Exception processing reconstruction log 
>    hdfs://lab5-2:9000/hbase/test-D-0.3/510178748/oldlogfile.log opening [B@ff65bf -- continuing.  Probably lack-of-HADOOP-1700 
>    causing DATA LOSS!
> java.io.EOFException
>         at java.io.DataInputStream.readFully(DataInputStream.java:180)
>         at org.apache.hadoop.hbase.io.DataOutputBuffer$Buffer.write(DataOutputBuffer.java:70)
>         at org.apache.hadoop.hbase.io.DataOutputBuffer.write(DataOutputBuffer.java:116)
>         at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1944)
>         at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1844)
>         at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1890)
>         at org.apache.hadoop.hbase.regionserver.HStore.doReconstructionLog(HStore.java:352)
>         at org.apache.hadoop.hbase.regionserver.HStore.runReconstructionLog(HStore.java:297)
>         at org.apache.hadoop.hbase.regionserver.HStore.<init>(HStore.java:237)
>         at org.apache.hadoop.hbase.regionserver.HRegion.instantiateHStore(HRegion.java:1764)
>         at org.apache.hadoop.hbase.regionserver.HRegion.initialize(HRegion.java:276)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.instantiateRegion(HRegionServer.java:1367)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.openRegion(HRegionServer.java:1338)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer$Worker.run(HRegionServer.java:1253)
>         at java.lang.Thread.run(Thread.java:619)
> {noformat}
> In my sense, since my hardware is in cause for RegionServer lease miss, I have to deal with the fact that I have to stop/restart hbase + hadoop in such a case. The problem in my sense is that a stop request should be able to perform a clean shutdown, and to avoid dataloss.
> Hope this description helps.

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


[jira] Commented: (HBASE-1214) HRegionServer fails to stop if hdfs client it tries to recover a block

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-1214?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12677049#action_12677049 ] 

stack commented on HBASE-1214:
------------------------------

I may be wrong but I don't think the IOE in "log extract #1" makes it out to the regionserver?  Its caught down in DFSCilent?

> HRegionServer fails to stop if hdfs client it tries to recover a block 
> -----------------------------------------------------------------------
>
>                 Key: HBASE-1214
>                 URL: https://issues.apache.org/jira/browse/HBASE-1214
>             Project: Hadoop HBase
>          Issue Type: Bug
>          Components: master, regionserver
>    Affects Versions: 0.19.0
>         Environment: 4 Node cluster with poor hardware: (DN+RS / DN+RS / RS / MA+NN)
> 1 Gb memory each 
> Ubuntu linux
> Java 6
>            Reporter: Jean-Adrien
>            Priority: Critical
>
> One of my region server falls in a long GC time that get it unresponsive during about 10 minutes.
> As I can see in the log, it seems that its DFSClient component was sending a file to hdfs, the sending times out when it recovers from GC:
> h5. 1. Region server log after recovering from GC
> {noformat}
> 2009-02-21 01:22:26,454 WARN org.apache.hadoop.hdfs.DFSClient: DFSOutputStream ResponseProcessor exception  
>     for block blk_7545556036225037274_1820952java.io.IOException: Bad response 1 for block 
>     blk_7545556036225037274_1820952 from datanode 192.168.1.10:50010
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:2342)
> {noformat}
> h5. 2. corresponding error in receiving datanode
> {noformat}
> 2009-02-21 01:23:56,608 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Exception in receiveBlock for block 
>     blk_7545556036225037274_1820952 java.io.EOFException: while trying to read 65557 bytes
> [...]
> 2009-02-21 01:23:59,076 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 0 for block 
>     blk_7545556036225037274_1820952 Interrupted.
> 2009-02-21 01:24:01,484 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 0 for block 
>     blk_7545556036225037274_1820952 terminating
> 2009-02-21 01:24:01,485 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: writeBlock 
>    blk_7545556036225037274_1820952 received exception java.io.EOFException: while trying to read 65557 bytes
> {noformat}
> Since region server misses its lease to report to the master, it has to close all its regions before recover, and reopens them when the master asks for.
> From this time, it tries to _recover_ this block, as seen in the regionserver log:
> h5. 3. Region server log in an endless loop to recover
> {noformat}
> 009-02-21 01:22:29,327 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block 
>     blk_7545556036225037274_1820952 bad datanode[1] 192.168.1.10:50010
> 2009-02-21 01:22:29,327 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block 
>     blk_7545556036225037274_1820952 in pipeline 192.168.1.13:50010, 
>    192.168.1.10:50010: bad datanode 192.168.1.10:50010
> 2009-02-21 01:22:29,689 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block 
>     blk_7545556036225037274_1820952 failed  because recovery from primary datanode 192.168.1.13:50010 failed 2 times. Will retry...
> {noformat}
> To this _recover_ request, all datanodes fail with this Exception
> h5. 4.
> {noformat}
> 2009-02-21 01:24:18,650 INFO org.apache.hadoop.ipc.Server: IPC Server handler 1 on 50020, call 
>    recoverBlock(blk_7545556036225037274_1820952, false, [Lorg.apache.hadoop.hdfs.protocol.DatanodeInfo;@311c4f) from 
>    192.168.1.13:56968: error: org.apache.hadoop.ipc.RemoteException: java.io.IOException: 
>    blk_7545556036225037274_1820952 is already commited, storedBlock == null.
>         at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.nextGenerationStampForBlock(FSNamesystem.java:4536)
>         at org.apache.hadoop.hdfs.server.namenode.NameNode.nextGenerationStamp(NameNode.java:402)
>         at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
> org.apache.hadoop.ipc.RemoteException: java.io.IOException: blk_7545556036225037274_1820952 is already commited, 
>    storedBlock == null.
>         at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.nextGenerationStampForBlock(FSNamesystem.java:4536)
>         at org.apache.hadoop.hdfs.server.namenode.NameNode.nextGenerationStamp(NameNode.java:402)
>         at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
>         at org.apache.hadoop.ipc.Client.call(Client.java:696)
>         at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:216)
>         at $Proxy4.nextGenerationStamp(Unknown Source)
>         at org.apache.hadoop.hdfs.server.datanode.DataNode.syncBlock(DataNode.java:1466)
>         at org.apache.hadoop.hdfs.server.datanode.DataNode.recoverBlock(DataNode.java:1440)
>         at org.apache.hadoop.hdfs.server.datanode.DataNode.recoverBlock(DataNode.java:1506)
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
> {noformat}
> The problem is that, in this case, the RegionServer and therefore the master fails to stop when I launch the stop-hbase script.
> A kill (-TERM) on the region server launches the SIGTERM handler thread, but the loop which try to reach the block does not stop.
> here is a thread dump of the region server at this time:
> h5. 5. Thread dump of the region server
> {noformat}
> Full thread dump Java HotSpot(TM) Server VM (10.0-b23 mixed mode):
> "Thread-16" prio=10 tid=0x081c3c00 nid=0x6e81 in Object.wait() [0x8f74f000..0x8f74fec0]
>    java.lang.Thread.State: WAITING (on object monitor)                                
>         at java.lang.Object.wait(Native Method)                                        
>         at java.lang.Thread.join(Thread.java:1143)                                    
>         - locked <0x93d0d060> (a java.lang.Thread)                                    
>         at java.lang.Thread.join(Thread.java:1196)                                    
>         at org.apache.hadoop.hbase.util.Threads.shutdown(Threads.java:78)              
>         at org.apache.hadoop.hbase.util.Threads.shutdown(Threads.java:66)              
>         at org.apache.hadoop.hbase.regionserver.HRegionServer$ShutdownThread.run(HRegionServer.java:814)
> "SIGTERM handler" daemon prio=10 tid=0x08dbcc00 nid=0x6e80 in Object.wait() [0x8edaf000..0x8edaff40]
>    java.lang.Thread.State: WAITING (on object monitor)                                              
>         at java.lang.Object.wait(Native Method)                                                    
>         at java.lang.Thread.join(Thread.java:1143)                                                  
>         - locked <0x93d0f3c0> (a org.apache.hadoop.hbase.regionserver.HRegionServer$ShutdownThread)
>         at java.lang.Thread.join(Thread.java:1196)                                                  
>         at java.lang.ApplicationShutdownHooks.run(ApplicationShutdownHooks.java:79)                
>         at java.lang.Shutdown.runHooks(Shutdown.java:89)                                            
>         at java.lang.Shutdown.sequence(Shutdown.java:133)                                          
>         at java.lang.Shutdown.exit(Shutdown.java:178)                                              
>         - locked <0xb0d3ed60> (a java.lang.Class for java.lang.Shutdown)                            
>         at java.lang.Terminator$1.handle(Terminator.java:35)                                        
>         at sun.misc.Signal$1.run(Signal.java:195)                                                  
>         at java.lang.Thread.run(Thread.java:619)                                                    
> "Attach Listener" daemon prio=10 tid=0x081e8800 nid=0x6e50 waiting on condition [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE                                                                    
> "IPC Client (47) connection to /192.168.1.13:50020 from an unknown user" daemon prio=10 tid=0x8e97a400 nid=0x39d1 in Object.wait() [0x8ec81000..0x8ec81fc0]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                              
>         at java.lang.Object.wait(Native Method)                                                                                                            
>         at org.apache.hadoop.ipc.Client$Connection.waitForWork(Client.java:395)                                                                            
>         - locked <0xa44a36a0> (a org.apache.hadoop.ipc.Client$Connection)                                                                                  
>         at org.apache.hadoop.ipc.Client$Connection.run(Client.java:437)                                                                                    
> "IPC Client (47) connection to /192.168.1.10:50020 from an unknown user" daemon prio=10 tid=0x8e5f1c00 nid=0x523f in Object.wait() [0x8ecd3000..0x8ecd3140]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                              
>         at java.lang.Object.wait(Native Method)                                                                                                            
>         at org.apache.hadoop.ipc.Client$Connection.waitForWork(Client.java:395)                                                                            
>         - locked <0xa448ca50> (a org.apache.hadoop.ipc.Client$Connection)                                                                                  
>         at org.apache.hadoop.ipc.Client$Connection.run(Client.java:437)                                                                                    
> "BlockFSInputStreamReferenceQueueChecker" daemon prio=10 tid=0x085a9400 nid=0x8c8 waiting on condition [0x8ee6c000..0x8ee6cdc0]
>    java.lang.Thread.State: WAITING (parking)                                                                                  
>         at sun.misc.Unsafe.park(Native Method)                                                                                
>         - parking to wait for  <0x93e76068> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)          
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                                  
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)  
>         at java.util.concurrent.DelayQueue.take(DelayQueue.java:160)                                                          
>         at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:582)        
>         at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:575)        
>         at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:946)                                        
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:906)                                    
>         at java.lang.Thread.run(Thread.java:619)                                                                              
> "IPC Server handler 9 on 60020" daemon prio=10 tid=0x0882e000 nid=0x889 waiting on condition [0x8eebd000..0x8eebdfc0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 8 on 60020" daemon prio=10 tid=0x0882cc00 nid=0x888 waiting on condition [0x8ef0e000..0x8ef0f040]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 7 on 60020" daemon prio=10 tid=0x0882b800 nid=0x887 waiting on condition [0x8ef5f000..0x8ef5fec0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 6 on 60020" daemon prio=10 tid=0x0882a400 nid=0x886 waiting on condition [0x8efb0000..0x8efb0f40]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 5 on 60020" daemon prio=10 tid=0x08829000 nid=0x885 waiting on condition [0x8f001000..0x8f001dc0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 4 on 60020" daemon prio=10 tid=0x083bcc00 nid=0x884 waiting on condition [0x8f052000..0x8f052e40]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 3 on 60020" daemon prio=10 tid=0x083bb800 nid=0x883 waiting on condition [0x8f0a3000..0x8f0a40c0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 2 on 60020" daemon prio=10 tid=0x083ba400 nid=0x882 waiting on condition [0x8f0f4000..0x8f0f5140]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 1 on 60020" daemon prio=10 tid=0x083b9400 nid=0x881 waiting on condition [0x8f145000..0x8f145fc0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 0 on 60020" daemon prio=10 tid=0x083b8400 nid=0x880 waiting on condition [0x8f196000..0x8f197040]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server listener on 60020" daemon prio=10 tid=0x083bfc00 nid=0x87f runnable [0x8f1e7000..0x8f1e7ec0]
>    java.lang.Thread.State: RUNNABLE                                                                    
>         at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)                                        
>         at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)                                
>         at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)                            
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)                                
>         - locked <0x93d0c090> (a sun.nio.ch.Util$1)                                                    
>         - locked <0x93d0c120> (a java.util.Collections$UnmodifiableSet)                                
>         - locked <0x93d0c0b0> (a sun.nio.ch.EPollSelectorImpl)                                          
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)                                        
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:84)                                        
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Listener.run(HBaseServer.java:299)                  
> "IPC Server Responder" daemon prio=10 tid=0x083bec00 nid=0x87e runnable [0x8f238000..0x8f238f40]
>    java.lang.Thread.State: RUNNABLE                                                            
>         at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)                                
>         at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)                        
>         at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)                    
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)                        
>         - locked <0x93d0c0f0> (a sun.nio.ch.Util$1)                                            
>         - locked <0x93d0c100> (a java.util.Collections$UnmodifiableSet)                        
>         - locked <0x93d0be30> (a sun.nio.ch.EPollSelectorImpl)                                  
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)                                
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Responder.run(HBaseServer.java:458)          
> "SocketListener0-1" prio=10 tid=0x088d5000 nid=0x87c in Object.wait() [0x8f2da000..0x8f2dae40]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                  
>         at java.lang.Object.wait(Native Method)                                              
>         at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:522)                    
>         - locked <0x93e39bc8> (a org.mortbay.util.ThreadPool$PoolThread)                      
> "SocketListener0-0" prio=10 tid=0x08ab5000 nid=0x87b in Object.wait() [0x8f32b000..0x8f32c0c0]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                  
>         at java.lang.Object.wait(Native Method)                                              
>         at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:522)                    
>         - locked <0x93e39920> (a org.mortbay.util.ThreadPool$PoolThread)                      
> "Acceptor ServerSocket[addr=0.0.0.0/0.0.0.0,port=0,localport=60030]" prio=10 tid=0x084f6800 nid=0x87a 
>    runnable [0x8f37c000..0x8f37d140]
>    java.lang.Thread.State: RUNNABLE                                                                                                    
>         at java.net.PlainSocketImpl.socketAccept(Native Method)                                                                        
>         at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:384)                                                                  
>         - locked <0x93e382a8> (a java.net.SocksSocketImpl)                                                                            
>         at java.net.ServerSocket.implAccept(ServerSocket.java:453)                                                                    
>         at java.net.ServerSocket.accept(ServerSocket.java:421)                                                                        
>         at org.mortbay.util.ThreadedServer.acceptSocket(ThreadedServer.java:432)                                                      
>         at org.mortbay.util.ThreadedServer$Acceptor.run(ThreadedServer.java:631)                                                      
> "SessionScavenger" daemon prio=10 tid=0x088f1400 nid=0x879 waiting on condition [0x8f3cd000..0x8f3cdfc0]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                    
>         at java.lang.Thread.sleep(Native Method)                                                        
>         at org.mortbay.jetty.servlet.AbstractSessionManager$SessionScavenger.run(AbstractSessionManager.java:587)
> "SessionScavenger" daemon prio=10 tid=0x08ab5c00 nid=0x878 waiting on condition [0x8f41e000..0x8f41f040]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                    
>         at java.lang.Thread.sleep(Native Method)                                                        
>         at org.mortbay.jetty.servlet.AbstractSessionManager$SessionScavenger.run(AbstractSessionManager.java:587)
> "regionserver/0.0.0.0:60020.leaseChecker" prio=10 tid=0x086d5800 nid=0x877 waiting on condition [0x8f476000..0x8f476ec0]
>    java.lang.Thread.State: TIMED_WAITING (parking)                                                                      
>         at sun.misc.Unsafe.park(Native Method)                                                                          
>         - parking to wait for  <0x93d0e980> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)  
>         at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:198)                                      
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:1963)
>         at java.util.concurrent.DelayQueue.poll(DelayQueue.java:201)                                                            
>         at org.apache.hadoop.hbase.Leases.run(Leases.java:78)                                                                    
> "regionserver/0.0.0.0:60020.majorCompactionChecker" daemon prio=10 tid=0x086d4000 nid=0x876 waiting on 
>    condition [0x8f4c7000..0x8f4c7f40]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                                                      
>         at java.lang.Thread.sleep(Native Method)                                                                                        
>         at org.apache.hadoop.hbase.util.Sleeper.sleep(Sleeper.java:74)                                                                  
>         at org.apache.hadoop.hbase.Chore.run(Chore.java:72)                                                                              
> "LeaseChecker" daemon prio=10 tid=0x0837cc00 nid=0x870 waiting on condition [0x8f6ad000..0x8f6ae040]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                
>         at java.lang.Thread.sleep(Native Method)                                                    
>         at org.apache.hadoop.hdfs.DFSClient$LeaseChecker.run(DFSClient.java:979)                    
>         at java.lang.Thread.run(Thread.java:619)                                                    
> "DataStreamer for file /hbase/log_192.168.1.13_1235129194745_60020/hlog.dat.1235129195649 block blk_7545556036225037274_1820952" 
>    daemon prio=10 tid=0x08375400 nid=0x86f in Object.wait() [0x8f6fe000..0x8f6feec0]                                                                                                      
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                                
>         at java.lang.Object.wait(Native Method)                                                                                                            
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2166)                                                          
>         - locked <0x93d0ea10> (a java.util.LinkedList)                                                                                                      
> "DestroyJavaVM" prio=10 tid=0x0805a000 nid=0x855 waiting on condition [0x00000000..0xb7dd2090]
>    java.lang.Thread.State: RUNNABLE                                                          
> "regionserver/0.0.0.0:60020" prio=10 tid=0x085b2400 nid=0x863 in Object.wait() [0x8f7f7000..0x8f7f7f40]
>    java.lang.Thread.State: WAITING (on object monitor)                                                
>         at java.lang.Object.wait(Native Method)                                                        
>         at java.lang.Object.wait(Object.java:485)                                                      
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.flushInternal(DFSClient.java:3015)        
>         - locked <0x93d0ea10> (a java.util.LinkedList)                                                
>         - locked <0x93d0ec58> (a org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)                    
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.closeInternal(DFSClient.java:3104)        
>         - locked <0x93d0ec58> (a org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)                    
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.close(DFSClient.java:3053)
>         at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:59)
>         at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:79)
>         at org.apache.hadoop.io.SequenceFile$Writer.close(SequenceFile.java:959)
>         - locked <0x93d50d20> (a org.apache.hadoop.io.SequenceFile$Writer)
>         at org.apache.hadoop.hbase.regionserver.HLog.close(HLog.java:421)
>         - locked <0x93d4ce00> (a java.lang.Integer)
>         at org.apache.hadoop.hbase.regionserver.HLog.closeAndDelete(HLog.java:404)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:373)
>         at java.lang.Thread.run(Thread.java:619)
> "Low Memory Detector" daemon prio=10 tid=0x080dd800 nid=0x85e runnable [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread1" daemon prio=10 tid=0x080dc400 nid=0x85d waiting on condition [0x00000000..0x8fbc1588]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread0" daemon prio=10 tid=0x080d9c00 nid=0x85c waiting on condition [0x00000000..0x8fc42608]
>    java.lang.Thread.State: RUNNABLE
> "Signal Dispatcher" daemon prio=10 tid=0x080d8800 nid=0x85b runnable [0x00000000..0x8fc93e80]
>    java.lang.Thread.State: RUNNABLE
> "Surrogate Locker Thread (CMS)" daemon prio=10 tid=0x080d7800 nid=0x85a waiting on condition [0x00000000..0x8fce522c]
>    java.lang.Thread.State: RUNNABLE
> "Finalizer" daemon prio=10 tid=0x080bbc00 nid=0x859 in Object.wait() [0x8fd7b000..0x8fd7bec0]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116)
>         - locked <0x93d0bdc0> (a java.lang.ref.ReferenceQueue$Lock)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:132)
>         at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159)
> "Reference Handler" daemon prio=10 tid=0x080ba800 nid=0x858 in Object.wait() [0x8fdcc000..0x8fdccf40]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.Object.wait(Object.java:485)
>         at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)
>         - locked <0x93d0c110> (a java.lang.ref.Reference$Lock)
> "VM Thread" prio=10 tid=0x080b7800 nid=0x857 runnable
> "Concurrent Mark-Sweep GC Thread" prio=10 tid=0x0806c800 nid=0x856 runnable
> "VM Periodic Task Thread" prio=10 tid=0x080df000 nid=0x85f waiting on condition
> JNI global references: 922
> {noformat}
> Notice the DataStreamer thread that seems to work on the concerned block
> This force me to kill (-KILL) the region server, and restart the hdfs (if I don't restart hdfs some regions are not available because the block is still not available).
> Note that the Hbase master does not end as well, and I have to kill (-KILL). Here is the thread dump of the HMaster process:
> h5. 6. Master thread dump:
> {noformat}
> Full thread dump Java HotSpot(TM) Server VM (10.0-b23 mixed mode):
> "Attach Listener" daemon prio=10 tid=0x0805a000 nid=0x2326 waiting on condition [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE                                                                    
> "SIGTERM handler" daemon prio=10 tid=0x089f0c00 nid=0x2303 waiting for monitor entry [0x8a6a5000..0x8a6a5ec0]
>    java.lang.Thread.State: BLOCKED (on object monitor)                                                      
>         at java.lang.Shutdown.exit(Shutdown.java:178)                                                        
>         - waiting to lock <0xb0beed60> (a java.lang.Class for java.lang.Shutdown)                            
>         at java.lang.Terminator$1.handle(Terminator.java:35)                                                
>         at sun.misc.Signal$1.run(Signal.java:195)                                                            
>         at java.lang.Thread.run(Thread.java:619)                                                            
> "Thread-1" prio=10 tid=0x8f666000 nid=0x653c waiting on condition [0x8f1fe000..0x8f1fef40]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                      
>         at java.lang.Thread.sleep(Native Method)                                          
>         at org.apache.hadoop.ipc.Client.stop(Client.java:667)                            
>         at org.apache.hadoop.ipc.RPC$ClientCache.stopClient(RPC.java:189)                
>         at org.apache.hadoop.ipc.RPC$ClientCache.access$400(RPC.java:138)                
>         at org.apache.hadoop.ipc.RPC$Invoker.close(RPC.java:229)                          
>         - locked <0x938a1b18> (a org.apache.hadoop.ipc.RPC$Invoker)                      
>         at org.apache.hadoop.ipc.RPC$Invoker.access$500(RPC.java:196)                    
>         at org.apache.hadoop.ipc.RPC.stopProxy(RPC.java:353)                              
>         at org.apache.hadoop.hdfs.DFSClient.close(DFSClient.java:213)                    
>         - locked <0x93878470> (a org.apache.hadoop.hdfs.DFSClient)                        
>         at org.apache.hadoop.hdfs.DistributedFileSystem.close(DistributedFileSystem.java:243)
>         at org.apache.hadoop.fs.FileSystem$Cache.closeAll(FileSystem.java:1413)              
>         - locked <0x937d3c00> (a org.apache.hadoop.fs.FileSystem$Cache)                      
>         at org.apache.hadoop.fs.FileSystem.closeAll(FileSystem.java:236)                    
>         at org.apache.hadoop.fs.FileSystem$ClientFinalizer.run(FileSystem.java:221)          
>         - locked <0x937b8a78> (a org.apache.hadoop.fs.FileSystem$ClientFinalizer)            
> "DestroyJavaVM" prio=10 tid=0x8f66c800 nid=0x36e9 in Object.wait() [0xb7d7d000..0xb7d7e0e0]
>    java.lang.Thread.State: WAITING (on object monitor)                                    
>         at java.lang.Object.wait(Native Method)                                            
>         at java.lang.Thread.join(Thread.java:1143)                                        
>         - locked <0x937b8a78> (a org.apache.hadoop.fs.FileSystem$ClientFinalizer)          
>         at java.lang.Thread.join(Thread.java:1196)
>         at java.lang.ApplicationShutdownHooks.run(ApplicationShutdownHooks.java:79)
>         at java.lang.Shutdown.runHooks(Shutdown.java:89)
>         at java.lang.Shutdown.sequence(Shutdown.java:133)
>         at java.lang.Shutdown.shutdown(Shutdown.java:200)
>         - locked <0xb0beed60> (a java.lang.Class for java.lang.Shutdown)
> "Low Memory Detector" daemon prio=10 tid=0x8fe03800 nid=0x36f4 runnable [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread1" daemon prio=10 tid=0x8fe01c00 nid=0x36f3 waiting on condition [0x00000000..0x8f9ee488]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread0" daemon prio=10 tid=0x8fe00800 nid=0x36f2 waiting on condition [0x00000000..0x8fa6f508]
>    java.lang.Thread.State: RUNNABLE
> "Signal Dispatcher" daemon prio=10 tid=0x0813e800 nid=0x36f1 runnable [0x00000000..0x8fac0d80]
>    java.lang.Thread.State: RUNNABLE
> "Surrogate Locker Thread (CMS)" daemon prio=10 tid=0x0813d800 nid=0x36f0 waiting on condition [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE
> "Finalizer" daemon prio=10 tid=0x08125000 nid=0x36ef in Object.wait() [0x8fba1000..0x8fba1dc0]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116)
>         - locked <0x937bafd8> (a java.lang.ref.ReferenceQueue$Lock)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:132)
>         at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159)
> "Reference Handler" daemon prio=10 tid=0x08120c00 nid=0x36ee in Object.wait() [0x8fbf2000..0x8fbf2e40]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.Object.wait(Object.java:485)
>         at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)
>         - locked <0x937baf18> (a java.lang.ref.Reference$Lock)
> "VM Thread" prio=10 tid=0x0811d800 nid=0x36ed runnable
> "Gang worker#0 (Parallel GC Threads)" prio=10 tid=0x0805e000 nid=0x36ea runnable
> "Gang worker#1 (Parallel GC Threads)" prio=10 tid=0x0805f000 nid=0x36eb runnable
> "Concurrent Mark-Sweep GC Thread" prio=10 tid=0x080cd000 nid=0x36ec runnable
> "VM Periodic Task Thread" prio=10 tid=0x8fe05000 nid=0x36f5 waiting on condition
> JNI global references: 1044
> {noformat}
> When I restart dfs, the log shows me that the concerned block is marked as invalidate, and it is therefore deleted:
> h5. 7. Namenode log
> {noformat}
> 2009-02-23 10:33:49,245 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.processReport: block 
>    blk_7545556036225037274_1820952 on 192.168.1.10:50010 size 27895296 does not
>  belong to any file.
> 2009-02-23 10:33:49,245 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addToInvalidates: 
>    blk_7545556036225037274 is added to invalidSet of 192.168.1.10:50010
> {noformat}
> Finally, when I restart HBase, I can see some of these messages in region server log, and I have some data loss. (this log entry may not be linked with the  block traced before)
> h5. 8. Region server log:
> {noformat}
> 2009-02-23 10:38:55,831 WARN org.apache.hadoop.hbase.regionserver.HStore: Exception processing reconstruction log 
>    hdfs://lab5-2:9000/hbase/test-D-0.3/510178748/oldlogfile.log opening [B@ff65bf -- continuing.  Probably lack-of-HADOOP-1700 
>    causing DATA LOSS!
> java.io.EOFException
>         at java.io.DataInputStream.readFully(DataInputStream.java:180)
>         at org.apache.hadoop.hbase.io.DataOutputBuffer$Buffer.write(DataOutputBuffer.java:70)
>         at org.apache.hadoop.hbase.io.DataOutputBuffer.write(DataOutputBuffer.java:116)
>         at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1944)
>         at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1844)
>         at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1890)
>         at org.apache.hadoop.hbase.regionserver.HStore.doReconstructionLog(HStore.java:352)
>         at org.apache.hadoop.hbase.regionserver.HStore.runReconstructionLog(HStore.java:297)
>         at org.apache.hadoop.hbase.regionserver.HStore.<init>(HStore.java:237)
>         at org.apache.hadoop.hbase.regionserver.HRegion.instantiateHStore(HRegion.java:1764)
>         at org.apache.hadoop.hbase.regionserver.HRegion.initialize(HRegion.java:276)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.instantiateRegion(HRegionServer.java:1367)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.openRegion(HRegionServer.java:1338)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer$Worker.run(HRegionServer.java:1253)
>         at java.lang.Thread.run(Thread.java:619)
> {noformat}
> In my sense, since my hardware is in cause for RegionServer lease miss, I have to deal with the fact that I have to stop/restart hbase + hadoop in such a case. The problem in my sense is that a stop request should be able to perform a clean shutdown, and to avoid dataloss.
> Hope this description helps.

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


[jira] Commented: (HBASE-1214) HRegionServer fails to stop if hdfs client it tries to recover a block

Posted by "Jean-Adrien (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-1214?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12677058#action_12677058 ] 

Jean-Adrien commented on HBASE-1214:
------------------------------------

Sure. Regarding to the log, the IOE comes from DFSClient, if I understand correctly, it is the response from the DataNode. HBase cannot be shamed for that, moreover since the connection was lost due to a problem of hardware...

The only problem in my sense is the subsequent state, when the 'retries' of the DFSClient that never stops prevent to process to a clean shutdown. 
Since I'm not aware of the DFSClient / HBase interface and component responsibilities, I can't tell if the infinite retry loop is the normal behavior of DFSClient or if it's a bug in Hadoop that avoids the DFSClient component to stop if it falls in such a state. 

Regarding to [HADOOP-4866], they says that the retries should eventually stop, unless "a client application [continuously tries] to open/append to the file", in this case it might be a bug of HBase, otherwise, I agree, this is not an HBase issue. Maybe it is possible to kill the DFSClient thread to allow the RS to shut itself down, but I guess that it could yield to the same dataloss, and therefore the kill command is preferable.

> HRegionServer fails to stop if hdfs client it tries to recover a block 
> -----------------------------------------------------------------------
>
>                 Key: HBASE-1214
>                 URL: https://issues.apache.org/jira/browse/HBASE-1214
>             Project: Hadoop HBase
>          Issue Type: Bug
>          Components: master, regionserver
>    Affects Versions: 0.19.0
>         Environment: 4 Node cluster with poor hardware: (DN+RS / DN+RS / RS / MA+NN)
> 1 Gb memory each 
> Ubuntu linux
> Java 6
>            Reporter: Jean-Adrien
>            Priority: Critical
>
> One of my region server falls in a long GC time that get it unresponsive during about 10 minutes.
> As I can see in the log, it seems that its DFSClient component was sending a file to hdfs, the sending times out when it recovers from GC:
> h5. 1. Region server log after recovering from GC
> {noformat}
> 2009-02-21 01:22:26,454 WARN org.apache.hadoop.hdfs.DFSClient: DFSOutputStream ResponseProcessor exception  
>     for block blk_7545556036225037274_1820952java.io.IOException: Bad response 1 for block 
>     blk_7545556036225037274_1820952 from datanode 192.168.1.10:50010
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:2342)
> {noformat}
> h5. 2. corresponding error in receiving datanode
> {noformat}
> 2009-02-21 01:23:56,608 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Exception in receiveBlock for block 
>     blk_7545556036225037274_1820952 java.io.EOFException: while trying to read 65557 bytes
> [...]
> 2009-02-21 01:23:59,076 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 0 for block 
>     blk_7545556036225037274_1820952 Interrupted.
> 2009-02-21 01:24:01,484 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 0 for block 
>     blk_7545556036225037274_1820952 terminating
> 2009-02-21 01:24:01,485 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: writeBlock 
>    blk_7545556036225037274_1820952 received exception java.io.EOFException: while trying to read 65557 bytes
> {noformat}
> Since region server misses its lease to report to the master, it has to close all its regions before recover, and reopens them when the master asks for.
> From this time, it tries to _recover_ this block, as seen in the regionserver log:
> h5. 3. Region server log in an endless loop to recover
> {noformat}
> 009-02-21 01:22:29,327 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block 
>     blk_7545556036225037274_1820952 bad datanode[1] 192.168.1.10:50010
> 2009-02-21 01:22:29,327 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block 
>     blk_7545556036225037274_1820952 in pipeline 192.168.1.13:50010, 
>    192.168.1.10:50010: bad datanode 192.168.1.10:50010
> 2009-02-21 01:22:29,689 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block 
>     blk_7545556036225037274_1820952 failed  because recovery from primary datanode 192.168.1.13:50010 failed 2 times. Will retry...
> {noformat}
> To this _recover_ request, all datanodes fail with this Exception
> h5. 4.
> {noformat}
> 2009-02-21 01:24:18,650 INFO org.apache.hadoop.ipc.Server: IPC Server handler 1 on 50020, call 
>    recoverBlock(blk_7545556036225037274_1820952, false, [Lorg.apache.hadoop.hdfs.protocol.DatanodeInfo;@311c4f) from 
>    192.168.1.13:56968: error: org.apache.hadoop.ipc.RemoteException: java.io.IOException: 
>    blk_7545556036225037274_1820952 is already commited, storedBlock == null.
>         at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.nextGenerationStampForBlock(FSNamesystem.java:4536)
>         at org.apache.hadoop.hdfs.server.namenode.NameNode.nextGenerationStamp(NameNode.java:402)
>         at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
> org.apache.hadoop.ipc.RemoteException: java.io.IOException: blk_7545556036225037274_1820952 is already commited, 
>    storedBlock == null.
>         at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.nextGenerationStampForBlock(FSNamesystem.java:4536)
>         at org.apache.hadoop.hdfs.server.namenode.NameNode.nextGenerationStamp(NameNode.java:402)
>         at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
>         at org.apache.hadoop.ipc.Client.call(Client.java:696)
>         at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:216)
>         at $Proxy4.nextGenerationStamp(Unknown Source)
>         at org.apache.hadoop.hdfs.server.datanode.DataNode.syncBlock(DataNode.java:1466)
>         at org.apache.hadoop.hdfs.server.datanode.DataNode.recoverBlock(DataNode.java:1440)
>         at org.apache.hadoop.hdfs.server.datanode.DataNode.recoverBlock(DataNode.java:1506)
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
> {noformat}
> The problem is that, in this case, the RegionServer and therefore the master fails to stop when I launch the stop-hbase script.
> A kill (-TERM) on the region server launches the SIGTERM handler thread, but the loop which try to reach the block does not stop.
> here is a thread dump of the region server at this time:
> h5. 5. Thread dump of the region server
> {noformat}
> Full thread dump Java HotSpot(TM) Server VM (10.0-b23 mixed mode):
> "Thread-16" prio=10 tid=0x081c3c00 nid=0x6e81 in Object.wait() [0x8f74f000..0x8f74fec0]
>    java.lang.Thread.State: WAITING (on object monitor)                                
>         at java.lang.Object.wait(Native Method)                                        
>         at java.lang.Thread.join(Thread.java:1143)                                    
>         - locked <0x93d0d060> (a java.lang.Thread)                                    
>         at java.lang.Thread.join(Thread.java:1196)                                    
>         at org.apache.hadoop.hbase.util.Threads.shutdown(Threads.java:78)              
>         at org.apache.hadoop.hbase.util.Threads.shutdown(Threads.java:66)              
>         at org.apache.hadoop.hbase.regionserver.HRegionServer$ShutdownThread.run(HRegionServer.java:814)
> "SIGTERM handler" daemon prio=10 tid=0x08dbcc00 nid=0x6e80 in Object.wait() [0x8edaf000..0x8edaff40]
>    java.lang.Thread.State: WAITING (on object monitor)                                              
>         at java.lang.Object.wait(Native Method)                                                    
>         at java.lang.Thread.join(Thread.java:1143)                                                  
>         - locked <0x93d0f3c0> (a org.apache.hadoop.hbase.regionserver.HRegionServer$ShutdownThread)
>         at java.lang.Thread.join(Thread.java:1196)                                                  
>         at java.lang.ApplicationShutdownHooks.run(ApplicationShutdownHooks.java:79)                
>         at java.lang.Shutdown.runHooks(Shutdown.java:89)                                            
>         at java.lang.Shutdown.sequence(Shutdown.java:133)                                          
>         at java.lang.Shutdown.exit(Shutdown.java:178)                                              
>         - locked <0xb0d3ed60> (a java.lang.Class for java.lang.Shutdown)                            
>         at java.lang.Terminator$1.handle(Terminator.java:35)                                        
>         at sun.misc.Signal$1.run(Signal.java:195)                                                  
>         at java.lang.Thread.run(Thread.java:619)                                                    
> "Attach Listener" daemon prio=10 tid=0x081e8800 nid=0x6e50 waiting on condition [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE                                                                    
> "IPC Client (47) connection to /192.168.1.13:50020 from an unknown user" daemon prio=10 tid=0x8e97a400 nid=0x39d1 in Object.wait() [0x8ec81000..0x8ec81fc0]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                              
>         at java.lang.Object.wait(Native Method)                                                                                                            
>         at org.apache.hadoop.ipc.Client$Connection.waitForWork(Client.java:395)                                                                            
>         - locked <0xa44a36a0> (a org.apache.hadoop.ipc.Client$Connection)                                                                                  
>         at org.apache.hadoop.ipc.Client$Connection.run(Client.java:437)                                                                                    
> "IPC Client (47) connection to /192.168.1.10:50020 from an unknown user" daemon prio=10 tid=0x8e5f1c00 nid=0x523f in Object.wait() [0x8ecd3000..0x8ecd3140]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                              
>         at java.lang.Object.wait(Native Method)                                                                                                            
>         at org.apache.hadoop.ipc.Client$Connection.waitForWork(Client.java:395)                                                                            
>         - locked <0xa448ca50> (a org.apache.hadoop.ipc.Client$Connection)                                                                                  
>         at org.apache.hadoop.ipc.Client$Connection.run(Client.java:437)                                                                                    
> "BlockFSInputStreamReferenceQueueChecker" daemon prio=10 tid=0x085a9400 nid=0x8c8 waiting on condition [0x8ee6c000..0x8ee6cdc0]
>    java.lang.Thread.State: WAITING (parking)                                                                                  
>         at sun.misc.Unsafe.park(Native Method)                                                                                
>         - parking to wait for  <0x93e76068> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)          
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                                  
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)  
>         at java.util.concurrent.DelayQueue.take(DelayQueue.java:160)                                                          
>         at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:582)        
>         at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:575)        
>         at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:946)                                        
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:906)                                    
>         at java.lang.Thread.run(Thread.java:619)                                                                              
> "IPC Server handler 9 on 60020" daemon prio=10 tid=0x0882e000 nid=0x889 waiting on condition [0x8eebd000..0x8eebdfc0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 8 on 60020" daemon prio=10 tid=0x0882cc00 nid=0x888 waiting on condition [0x8ef0e000..0x8ef0f040]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 7 on 60020" daemon prio=10 tid=0x0882b800 nid=0x887 waiting on condition [0x8ef5f000..0x8ef5fec0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 6 on 60020" daemon prio=10 tid=0x0882a400 nid=0x886 waiting on condition [0x8efb0000..0x8efb0f40]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 5 on 60020" daemon prio=10 tid=0x08829000 nid=0x885 waiting on condition [0x8f001000..0x8f001dc0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 4 on 60020" daemon prio=10 tid=0x083bcc00 nid=0x884 waiting on condition [0x8f052000..0x8f052e40]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 3 on 60020" daemon prio=10 tid=0x083bb800 nid=0x883 waiting on condition [0x8f0a3000..0x8f0a40c0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 2 on 60020" daemon prio=10 tid=0x083ba400 nid=0x882 waiting on condition [0x8f0f4000..0x8f0f5140]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 1 on 60020" daemon prio=10 tid=0x083b9400 nid=0x881 waiting on condition [0x8f145000..0x8f145fc0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 0 on 60020" daemon prio=10 tid=0x083b8400 nid=0x880 waiting on condition [0x8f196000..0x8f197040]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server listener on 60020" daemon prio=10 tid=0x083bfc00 nid=0x87f runnable [0x8f1e7000..0x8f1e7ec0]
>    java.lang.Thread.State: RUNNABLE                                                                    
>         at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)                                        
>         at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)                                
>         at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)                            
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)                                
>         - locked <0x93d0c090> (a sun.nio.ch.Util$1)                                                    
>         - locked <0x93d0c120> (a java.util.Collections$UnmodifiableSet)                                
>         - locked <0x93d0c0b0> (a sun.nio.ch.EPollSelectorImpl)                                          
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)                                        
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:84)                                        
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Listener.run(HBaseServer.java:299)                  
> "IPC Server Responder" daemon prio=10 tid=0x083bec00 nid=0x87e runnable [0x8f238000..0x8f238f40]
>    java.lang.Thread.State: RUNNABLE                                                            
>         at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)                                
>         at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)                        
>         at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)                    
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)                        
>         - locked <0x93d0c0f0> (a sun.nio.ch.Util$1)                                            
>         - locked <0x93d0c100> (a java.util.Collections$UnmodifiableSet)                        
>         - locked <0x93d0be30> (a sun.nio.ch.EPollSelectorImpl)                                  
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)                                
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Responder.run(HBaseServer.java:458)          
> "SocketListener0-1" prio=10 tid=0x088d5000 nid=0x87c in Object.wait() [0x8f2da000..0x8f2dae40]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                  
>         at java.lang.Object.wait(Native Method)                                              
>         at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:522)                    
>         - locked <0x93e39bc8> (a org.mortbay.util.ThreadPool$PoolThread)                      
> "SocketListener0-0" prio=10 tid=0x08ab5000 nid=0x87b in Object.wait() [0x8f32b000..0x8f32c0c0]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                  
>         at java.lang.Object.wait(Native Method)                                              
>         at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:522)                    
>         - locked <0x93e39920> (a org.mortbay.util.ThreadPool$PoolThread)                      
> "Acceptor ServerSocket[addr=0.0.0.0/0.0.0.0,port=0,localport=60030]" prio=10 tid=0x084f6800 nid=0x87a 
>    runnable [0x8f37c000..0x8f37d140]
>    java.lang.Thread.State: RUNNABLE                                                                                                    
>         at java.net.PlainSocketImpl.socketAccept(Native Method)                                                                        
>         at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:384)                                                                  
>         - locked <0x93e382a8> (a java.net.SocksSocketImpl)                                                                            
>         at java.net.ServerSocket.implAccept(ServerSocket.java:453)                                                                    
>         at java.net.ServerSocket.accept(ServerSocket.java:421)                                                                        
>         at org.mortbay.util.ThreadedServer.acceptSocket(ThreadedServer.java:432)                                                      
>         at org.mortbay.util.ThreadedServer$Acceptor.run(ThreadedServer.java:631)                                                      
> "SessionScavenger" daemon prio=10 tid=0x088f1400 nid=0x879 waiting on condition [0x8f3cd000..0x8f3cdfc0]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                    
>         at java.lang.Thread.sleep(Native Method)                                                        
>         at org.mortbay.jetty.servlet.AbstractSessionManager$SessionScavenger.run(AbstractSessionManager.java:587)
> "SessionScavenger" daemon prio=10 tid=0x08ab5c00 nid=0x878 waiting on condition [0x8f41e000..0x8f41f040]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                    
>         at java.lang.Thread.sleep(Native Method)                                                        
>         at org.mortbay.jetty.servlet.AbstractSessionManager$SessionScavenger.run(AbstractSessionManager.java:587)
> "regionserver/0.0.0.0:60020.leaseChecker" prio=10 tid=0x086d5800 nid=0x877 waiting on condition [0x8f476000..0x8f476ec0]
>    java.lang.Thread.State: TIMED_WAITING (parking)                                                                      
>         at sun.misc.Unsafe.park(Native Method)                                                                          
>         - parking to wait for  <0x93d0e980> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)  
>         at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:198)                                      
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:1963)
>         at java.util.concurrent.DelayQueue.poll(DelayQueue.java:201)                                                            
>         at org.apache.hadoop.hbase.Leases.run(Leases.java:78)                                                                    
> "regionserver/0.0.0.0:60020.majorCompactionChecker" daemon prio=10 tid=0x086d4000 nid=0x876 waiting on 
>    condition [0x8f4c7000..0x8f4c7f40]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                                                      
>         at java.lang.Thread.sleep(Native Method)                                                                                        
>         at org.apache.hadoop.hbase.util.Sleeper.sleep(Sleeper.java:74)                                                                  
>         at org.apache.hadoop.hbase.Chore.run(Chore.java:72)                                                                              
> "LeaseChecker" daemon prio=10 tid=0x0837cc00 nid=0x870 waiting on condition [0x8f6ad000..0x8f6ae040]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                
>         at java.lang.Thread.sleep(Native Method)                                                    
>         at org.apache.hadoop.hdfs.DFSClient$LeaseChecker.run(DFSClient.java:979)                    
>         at java.lang.Thread.run(Thread.java:619)                                                    
> "DataStreamer for file /hbase/log_192.168.1.13_1235129194745_60020/hlog.dat.1235129195649 block blk_7545556036225037274_1820952" 
>    daemon prio=10 tid=0x08375400 nid=0x86f in Object.wait() [0x8f6fe000..0x8f6feec0]                                                                                                      
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                                
>         at java.lang.Object.wait(Native Method)                                                                                                            
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2166)                                                          
>         - locked <0x93d0ea10> (a java.util.LinkedList)                                                                                                      
> "DestroyJavaVM" prio=10 tid=0x0805a000 nid=0x855 waiting on condition [0x00000000..0xb7dd2090]
>    java.lang.Thread.State: RUNNABLE                                                          
> "regionserver/0.0.0.0:60020" prio=10 tid=0x085b2400 nid=0x863 in Object.wait() [0x8f7f7000..0x8f7f7f40]
>    java.lang.Thread.State: WAITING (on object monitor)                                                
>         at java.lang.Object.wait(Native Method)                                                        
>         at java.lang.Object.wait(Object.java:485)                                                      
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.flushInternal(DFSClient.java:3015)        
>         - locked <0x93d0ea10> (a java.util.LinkedList)                                                
>         - locked <0x93d0ec58> (a org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)                    
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.closeInternal(DFSClient.java:3104)        
>         - locked <0x93d0ec58> (a org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)                    
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.close(DFSClient.java:3053)
>         at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:59)
>         at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:79)
>         at org.apache.hadoop.io.SequenceFile$Writer.close(SequenceFile.java:959)
>         - locked <0x93d50d20> (a org.apache.hadoop.io.SequenceFile$Writer)
>         at org.apache.hadoop.hbase.regionserver.HLog.close(HLog.java:421)
>         - locked <0x93d4ce00> (a java.lang.Integer)
>         at org.apache.hadoop.hbase.regionserver.HLog.closeAndDelete(HLog.java:404)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:373)
>         at java.lang.Thread.run(Thread.java:619)
> "Low Memory Detector" daemon prio=10 tid=0x080dd800 nid=0x85e runnable [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread1" daemon prio=10 tid=0x080dc400 nid=0x85d waiting on condition [0x00000000..0x8fbc1588]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread0" daemon prio=10 tid=0x080d9c00 nid=0x85c waiting on condition [0x00000000..0x8fc42608]
>    java.lang.Thread.State: RUNNABLE
> "Signal Dispatcher" daemon prio=10 tid=0x080d8800 nid=0x85b runnable [0x00000000..0x8fc93e80]
>    java.lang.Thread.State: RUNNABLE
> "Surrogate Locker Thread (CMS)" daemon prio=10 tid=0x080d7800 nid=0x85a waiting on condition [0x00000000..0x8fce522c]
>    java.lang.Thread.State: RUNNABLE
> "Finalizer" daemon prio=10 tid=0x080bbc00 nid=0x859 in Object.wait() [0x8fd7b000..0x8fd7bec0]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116)
>         - locked <0x93d0bdc0> (a java.lang.ref.ReferenceQueue$Lock)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:132)
>         at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159)
> "Reference Handler" daemon prio=10 tid=0x080ba800 nid=0x858 in Object.wait() [0x8fdcc000..0x8fdccf40]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.Object.wait(Object.java:485)
>         at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)
>         - locked <0x93d0c110> (a java.lang.ref.Reference$Lock)
> "VM Thread" prio=10 tid=0x080b7800 nid=0x857 runnable
> "Concurrent Mark-Sweep GC Thread" prio=10 tid=0x0806c800 nid=0x856 runnable
> "VM Periodic Task Thread" prio=10 tid=0x080df000 nid=0x85f waiting on condition
> JNI global references: 922
> {noformat}
> Notice the DataStreamer thread that seems to work on the concerned block
> This force me to kill (-KILL) the region server, and restart the hdfs (if I don't restart hdfs some regions are not available because the block is still not available).
> Note that the Hbase master does not end as well, and I have to kill (-KILL). Here is the thread dump of the HMaster process:
> h5. 6. Master thread dump:
> {noformat}
> Full thread dump Java HotSpot(TM) Server VM (10.0-b23 mixed mode):
> "Attach Listener" daemon prio=10 tid=0x0805a000 nid=0x2326 waiting on condition [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE                                                                    
> "SIGTERM handler" daemon prio=10 tid=0x089f0c00 nid=0x2303 waiting for monitor entry [0x8a6a5000..0x8a6a5ec0]
>    java.lang.Thread.State: BLOCKED (on object monitor)                                                      
>         at java.lang.Shutdown.exit(Shutdown.java:178)                                                        
>         - waiting to lock <0xb0beed60> (a java.lang.Class for java.lang.Shutdown)                            
>         at java.lang.Terminator$1.handle(Terminator.java:35)                                                
>         at sun.misc.Signal$1.run(Signal.java:195)                                                            
>         at java.lang.Thread.run(Thread.java:619)                                                            
> "Thread-1" prio=10 tid=0x8f666000 nid=0x653c waiting on condition [0x8f1fe000..0x8f1fef40]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                      
>         at java.lang.Thread.sleep(Native Method)                                          
>         at org.apache.hadoop.ipc.Client.stop(Client.java:667)                            
>         at org.apache.hadoop.ipc.RPC$ClientCache.stopClient(RPC.java:189)                
>         at org.apache.hadoop.ipc.RPC$ClientCache.access$400(RPC.java:138)                
>         at org.apache.hadoop.ipc.RPC$Invoker.close(RPC.java:229)                          
>         - locked <0x938a1b18> (a org.apache.hadoop.ipc.RPC$Invoker)                      
>         at org.apache.hadoop.ipc.RPC$Invoker.access$500(RPC.java:196)                    
>         at org.apache.hadoop.ipc.RPC.stopProxy(RPC.java:353)                              
>         at org.apache.hadoop.hdfs.DFSClient.close(DFSClient.java:213)                    
>         - locked <0x93878470> (a org.apache.hadoop.hdfs.DFSClient)                        
>         at org.apache.hadoop.hdfs.DistributedFileSystem.close(DistributedFileSystem.java:243)
>         at org.apache.hadoop.fs.FileSystem$Cache.closeAll(FileSystem.java:1413)              
>         - locked <0x937d3c00> (a org.apache.hadoop.fs.FileSystem$Cache)                      
>         at org.apache.hadoop.fs.FileSystem.closeAll(FileSystem.java:236)                    
>         at org.apache.hadoop.fs.FileSystem$ClientFinalizer.run(FileSystem.java:221)          
>         - locked <0x937b8a78> (a org.apache.hadoop.fs.FileSystem$ClientFinalizer)            
> "DestroyJavaVM" prio=10 tid=0x8f66c800 nid=0x36e9 in Object.wait() [0xb7d7d000..0xb7d7e0e0]
>    java.lang.Thread.State: WAITING (on object monitor)                                    
>         at java.lang.Object.wait(Native Method)                                            
>         at java.lang.Thread.join(Thread.java:1143)                                        
>         - locked <0x937b8a78> (a org.apache.hadoop.fs.FileSystem$ClientFinalizer)          
>         at java.lang.Thread.join(Thread.java:1196)
>         at java.lang.ApplicationShutdownHooks.run(ApplicationShutdownHooks.java:79)
>         at java.lang.Shutdown.runHooks(Shutdown.java:89)
>         at java.lang.Shutdown.sequence(Shutdown.java:133)
>         at java.lang.Shutdown.shutdown(Shutdown.java:200)
>         - locked <0xb0beed60> (a java.lang.Class for java.lang.Shutdown)
> "Low Memory Detector" daemon prio=10 tid=0x8fe03800 nid=0x36f4 runnable [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread1" daemon prio=10 tid=0x8fe01c00 nid=0x36f3 waiting on condition [0x00000000..0x8f9ee488]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread0" daemon prio=10 tid=0x8fe00800 nid=0x36f2 waiting on condition [0x00000000..0x8fa6f508]
>    java.lang.Thread.State: RUNNABLE
> "Signal Dispatcher" daemon prio=10 tid=0x0813e800 nid=0x36f1 runnable [0x00000000..0x8fac0d80]
>    java.lang.Thread.State: RUNNABLE
> "Surrogate Locker Thread (CMS)" daemon prio=10 tid=0x0813d800 nid=0x36f0 waiting on condition [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE
> "Finalizer" daemon prio=10 tid=0x08125000 nid=0x36ef in Object.wait() [0x8fba1000..0x8fba1dc0]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116)
>         - locked <0x937bafd8> (a java.lang.ref.ReferenceQueue$Lock)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:132)
>         at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159)
> "Reference Handler" daemon prio=10 tid=0x08120c00 nid=0x36ee in Object.wait() [0x8fbf2000..0x8fbf2e40]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.Object.wait(Object.java:485)
>         at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)
>         - locked <0x937baf18> (a java.lang.ref.Reference$Lock)
> "VM Thread" prio=10 tid=0x0811d800 nid=0x36ed runnable
> "Gang worker#0 (Parallel GC Threads)" prio=10 tid=0x0805e000 nid=0x36ea runnable
> "Gang worker#1 (Parallel GC Threads)" prio=10 tid=0x0805f000 nid=0x36eb runnable
> "Concurrent Mark-Sweep GC Thread" prio=10 tid=0x080cd000 nid=0x36ec runnable
> "VM Periodic Task Thread" prio=10 tid=0x8fe05000 nid=0x36f5 waiting on condition
> JNI global references: 1044
> {noformat}
> When I restart dfs, the log shows me that the concerned block is marked as invalidate, and it is therefore deleted:
> h5. 7. Namenode log
> {noformat}
> 2009-02-23 10:33:49,245 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.processReport: block 
>    blk_7545556036225037274_1820952 on 192.168.1.10:50010 size 27895296 does not
>  belong to any file.
> 2009-02-23 10:33:49,245 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addToInvalidates: 
>    blk_7545556036225037274 is added to invalidSet of 192.168.1.10:50010
> {noformat}
> Finally, when I restart HBase, I can see some of these messages in region server log, and I have some data loss. (this log entry may not be linked with the  block traced before)
> h5. 8. Region server log:
> {noformat}
> 2009-02-23 10:38:55,831 WARN org.apache.hadoop.hbase.regionserver.HStore: Exception processing reconstruction log 
>    hdfs://lab5-2:9000/hbase/test-D-0.3/510178748/oldlogfile.log opening [B@ff65bf -- continuing.  Probably lack-of-HADOOP-1700 
>    causing DATA LOSS!
> java.io.EOFException
>         at java.io.DataInputStream.readFully(DataInputStream.java:180)
>         at org.apache.hadoop.hbase.io.DataOutputBuffer$Buffer.write(DataOutputBuffer.java:70)
>         at org.apache.hadoop.hbase.io.DataOutputBuffer.write(DataOutputBuffer.java:116)
>         at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1944)
>         at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1844)
>         at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1890)
>         at org.apache.hadoop.hbase.regionserver.HStore.doReconstructionLog(HStore.java:352)
>         at org.apache.hadoop.hbase.regionserver.HStore.runReconstructionLog(HStore.java:297)
>         at org.apache.hadoop.hbase.regionserver.HStore.<init>(HStore.java:237)
>         at org.apache.hadoop.hbase.regionserver.HRegion.instantiateHStore(HRegion.java:1764)
>         at org.apache.hadoop.hbase.regionserver.HRegion.initialize(HRegion.java:276)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.instantiateRegion(HRegionServer.java:1367)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.openRegion(HRegionServer.java:1338)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer$Worker.run(HRegionServer.java:1253)
>         at java.lang.Thread.run(Thread.java:619)
> {noformat}
> In my sense, since my hardware is in cause for RegionServer lease miss, I have to deal with the fact that I have to stop/restart hbase + hadoop in such a case. The problem in my sense is that a stop request should be able to perform a clean shutdown, and to avoid dataloss.
> Hope this description helps.

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


[jira] Updated: (HBASE-1214) HRegionServer fails to stop if hdfs client it tries to recover a block

Posted by "stack (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-1214?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

stack updated HBASE-1214:
-------------------------

    Priority: Critical  (was: Major)

Adding the related issue J-D cited: https://issues.apache.org/jira/browse/HADOOP-4866.

In 4866, it says dfsclient throws an IOE but J-A, did you see such an IOE bubble up to the regionserver?  Your notes cite DFSClient emissions in regionserver logs.  I've seen them myself but these are loggings of events that are usually below the regionservers' purview, IIRC.  If an IOE came out of HDFS, I wonder what it was? We may be being overly tenacious in some regards when it comes to hdfs mostly because our only alternative usually is shutting ourselves down to minimize data-loss.

I'm pretty sure I've seen this before myself -- there are other issues where we are stuck on hdfs waiting on a working datastreamer -- but in those cases absent is the doggedness demonstrated above by J-A tracing back a root cause.  Thanks for digging in J-A.

I've made this critical since HRS won't go down and J-A's work gives us some hope we can skip this issue going forward.

> HRegionServer fails to stop if hdfs client it tries to recover a block 
> -----------------------------------------------------------------------
>
>                 Key: HBASE-1214
>                 URL: https://issues.apache.org/jira/browse/HBASE-1214
>             Project: Hadoop HBase
>          Issue Type: Bug
>          Components: master, regionserver
>    Affects Versions: 0.19.0
>         Environment: 4 Node cluster with poor hardware: (DN+RS / DN+RS / RS / MA+NN)
> 1 Gb memory each 
> Ubuntu linux
> Java 6
>            Reporter: Jean-Adrien
>            Priority: Critical
>
> One of my region server falls in a long GC time that get it unresponsive during about 10 minutes.
> As I can see in the log, it seems that its DFSClient component was sending a file to hdfs, the sending times out when it recovers from GC:
> h5. Region server log after recovering from GC
> {noformat}
> 2009-02-21 01:22:26,454 WARN org.apache.hadoop.hdfs.DFSClient: DFSOutputStream ResponseProcessor exception  for block blk_7545556036225037274_1820952java.io.IOException: Bad response 1 for block blk_7545556036225037274_1820952 from datanode 192.168.1.10:50010
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:2342)
> {noformat}
> h5. corresponding error in receiving datanode
> {noformat}
> 2009-02-21 01:23:56,608 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Exception in receiveBlock for block blk_7545556036225037274_1820952 java.io.EOFException: while trying to read 65557 bytes
> [...]
> 2009-02-21 01:23:59,076 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 0 for block blk_7545556036225037274_1820952 Interrupted.
> 2009-02-21 01:24:01,484 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 0 for block blk_7545556036225037274_1820952 terminating
> 2009-02-21 01:24:01,485 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: writeBlock blk_7545556036225037274_1820952 received exception java.io.EOFException: while trying to read 65557 bytes
> {noformat}
> Since region server misses its lease to report to the master, it has to close all its regions before recover, and reopens them when the master asks for.
> From this time, it tries to _recover_ this block, as seen in the regionserver log:
> h5. Region server log in an endless loop to recover
> {noformat}
> 009-02-21 01:22:29,327 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_7545556036225037274_1820952 bad datanode[1] 192.168.1.10:50010
> 2009-02-21 01:22:29,327 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_7545556036225037274_1820952 in pipeline 192.168.1.13:50010, 192.
> 168.1.10:50010: bad datanode 192.168.1.10:50010
> 2009-02-21 01:22:29,689 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_7545556036225037274_1820952 failed  because recovery from primary datanode 192.168.1.13:50010 failed 2 times. Will retry...
> {noformat}
> To this _recover_ request, all datanodes fail with this Exception
> {noformat}
> 2009-02-21 01:24:18,650 INFO org.apache.hadoop.ipc.Server: IPC Server handler 1 on 50020, call recoverBlock(blk_7545556036225037274_1820952, false, [Lorg.apache.hadoop.hdfs.protocol.DatanodeInfo;@311c4f) from 192.168.1.13:56968: error: org.apache.hadoop.ipc.RemoteException: java.io.IOException: blk_7545556036225037274_1820952 is already commited, storedBlock == null.
>         at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.nextGenerationStampForBlock(FSNamesystem.java:4536)
>         at org.apache.hadoop.hdfs.server.namenode.NameNode.nextGenerationStamp(NameNode.java:402)
>         at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
> org.apache.hadoop.ipc.RemoteException: java.io.IOException: blk_7545556036225037274_1820952 is already commited, storedBlock == null.
>         at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.nextGenerationStampForBlock(FSNamesystem.java:4536)
>         at org.apache.hadoop.hdfs.server.namenode.NameNode.nextGenerationStamp(NameNode.java:402)
>         at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
>         at org.apache.hadoop.ipc.Client.call(Client.java:696)
>         at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:216)
>         at $Proxy4.nextGenerationStamp(Unknown Source)
>         at org.apache.hadoop.hdfs.server.datanode.DataNode.syncBlock(DataNode.java:1466)
>         at org.apache.hadoop.hdfs.server.datanode.DataNode.recoverBlock(DataNode.java:1440)
>         at org.apache.hadoop.hdfs.server.datanode.DataNode.recoverBlock(DataNode.java:1506)
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:892)
> {noformat}
> The problem is that, in this case, the RegionServer and therefore the master fails to stop when I launch the stop-hbase script.
> A kill (-TERM) on the region server launches the SIGTERM handler thread, but the loop which try to reach the block does not stop.
> here is a thread dump of the region server at this time:
> h5. Thread dump of the region server
> {noformat}
> Full thread dump Java HotSpot(TM) Server VM (10.0-b23 mixed mode):
> "Thread-16" prio=10 tid=0x081c3c00 nid=0x6e81 in Object.wait() [0x8f74f000..0x8f74fec0]
>    java.lang.Thread.State: WAITING (on object monitor)                                
>         at java.lang.Object.wait(Native Method)                                        
>         at java.lang.Thread.join(Thread.java:1143)                                    
>         - locked <0x93d0d060> (a java.lang.Thread)                                    
>         at java.lang.Thread.join(Thread.java:1196)                                    
>         at org.apache.hadoop.hbase.util.Threads.shutdown(Threads.java:78)              
>         at org.apache.hadoop.hbase.util.Threads.shutdown(Threads.java:66)              
>         at org.apache.hadoop.hbase.regionserver.HRegionServer$ShutdownThread.run(HRegionServer.java:814)
> "SIGTERM handler" daemon prio=10 tid=0x08dbcc00 nid=0x6e80 in Object.wait() [0x8edaf000..0x8edaff40]
>    java.lang.Thread.State: WAITING (on object monitor)                                              
>         at java.lang.Object.wait(Native Method)                                                    
>         at java.lang.Thread.join(Thread.java:1143)                                                  
>         - locked <0x93d0f3c0> (a org.apache.hadoop.hbase.regionserver.HRegionServer$ShutdownThread)
>         at java.lang.Thread.join(Thread.java:1196)                                                  
>         at java.lang.ApplicationShutdownHooks.run(ApplicationShutdownHooks.java:79)                
>         at java.lang.Shutdown.runHooks(Shutdown.java:89)                                            
>         at java.lang.Shutdown.sequence(Shutdown.java:133)                                          
>         at java.lang.Shutdown.exit(Shutdown.java:178)                                              
>         - locked <0xb0d3ed60> (a java.lang.Class for java.lang.Shutdown)                            
>         at java.lang.Terminator$1.handle(Terminator.java:35)                                        
>         at sun.misc.Signal$1.run(Signal.java:195)                                                  
>         at java.lang.Thread.run(Thread.java:619)                                                    
> "Attach Listener" daemon prio=10 tid=0x081e8800 nid=0x6e50 waiting on condition [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE                                                                    
> "IPC Client (47) connection to /192.168.1.13:50020 from an unknown user" daemon prio=10 tid=0x8e97a400 nid=0x39d1 in Object.wait() [0x8ec81000..0x8ec81fc0]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                              
>         at java.lang.Object.wait(Native Method)                                                                                                            
>         at org.apache.hadoop.ipc.Client$Connection.waitForWork(Client.java:395)                                                                            
>         - locked <0xa44a36a0> (a org.apache.hadoop.ipc.Client$Connection)                                                                                  
>         at org.apache.hadoop.ipc.Client$Connection.run(Client.java:437)                                                                                    
> "IPC Client (47) connection to /192.168.1.10:50020 from an unknown user" daemon prio=10 tid=0x8e5f1c00 nid=0x523f in Object.wait() [0x8ecd3000..0x8ecd3140]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                              
>         at java.lang.Object.wait(Native Method)                                                                                                            
>         at org.apache.hadoop.ipc.Client$Connection.waitForWork(Client.java:395)                                                                            
>         - locked <0xa448ca50> (a org.apache.hadoop.ipc.Client$Connection)                                                                                  
>         at org.apache.hadoop.ipc.Client$Connection.run(Client.java:437)                                                                                    
> "BlockFSInputStreamReferenceQueueChecker" daemon prio=10 tid=0x085a9400 nid=0x8c8 waiting on condition [0x8ee6c000..0x8ee6cdc0]
>    java.lang.Thread.State: WAITING (parking)                                                                                  
>         at sun.misc.Unsafe.park(Native Method)                                                                                
>         - parking to wait for  <0x93e76068> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)          
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                                  
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)  
>         at java.util.concurrent.DelayQueue.take(DelayQueue.java:160)                                                          
>         at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:582)        
>         at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:575)        
>         at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:946)                                        
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:906)                                    
>         at java.lang.Thread.run(Thread.java:619)                                                                              
> "IPC Server handler 9 on 60020" daemon prio=10 tid=0x0882e000 nid=0x889 waiting on condition [0x8eebd000..0x8eebdfc0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 8 on 60020" daemon prio=10 tid=0x0882cc00 nid=0x888 waiting on condition [0x8ef0e000..0x8ef0f040]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 7 on 60020" daemon prio=10 tid=0x0882b800 nid=0x887 waiting on condition [0x8ef5f000..0x8ef5fec0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 6 on 60020" daemon prio=10 tid=0x0882a400 nid=0x886 waiting on condition [0x8efb0000..0x8efb0f40]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 5 on 60020" daemon prio=10 tid=0x08829000 nid=0x885 waiting on condition [0x8f001000..0x8f001dc0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 4 on 60020" daemon prio=10 tid=0x083bcc00 nid=0x884 waiting on condition [0x8f052000..0x8f052e40]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 3 on 60020" daemon prio=10 tid=0x083bb800 nid=0x883 waiting on condition [0x8f0a3000..0x8f0a40c0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 2 on 60020" daemon prio=10 tid=0x083ba400 nid=0x882 waiting on condition [0x8f0f4000..0x8f0f5140]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 1 on 60020" daemon prio=10 tid=0x083b9400 nid=0x881 waiting on condition [0x8f145000..0x8f145fc0]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server handler 0 on 60020" daemon prio=10 tid=0x083b8400 nid=0x880 waiting on condition [0x8f196000..0x8f197040]
>    java.lang.Thread.State: WAITING (parking)                                                                        
>         at sun.misc.Unsafe.park(Native Method)                                                                      
>         - parking to wait for  <0x93d0e9c8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)                                        
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>         at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)                                      
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:881)                                        
> "IPC Server listener on 60020" daemon prio=10 tid=0x083bfc00 nid=0x87f runnable [0x8f1e7000..0x8f1e7ec0]
>    java.lang.Thread.State: RUNNABLE                                                                    
>         at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)                                        
>         at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)                                
>         at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)                            
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)                                
>         - locked <0x93d0c090> (a sun.nio.ch.Util$1)                                                    
>         - locked <0x93d0c120> (a java.util.Collections$UnmodifiableSet)                                
>         - locked <0x93d0c0b0> (a sun.nio.ch.EPollSelectorImpl)                                          
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)                                        
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:84)                                        
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Listener.run(HBaseServer.java:299)                  
> "IPC Server Responder" daemon prio=10 tid=0x083bec00 nid=0x87e runnable [0x8f238000..0x8f238f40]
>    java.lang.Thread.State: RUNNABLE                                                            
>         at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)                                
>         at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)                        
>         at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)                    
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)                        
>         - locked <0x93d0c0f0> (a sun.nio.ch.Util$1)                                            
>         - locked <0x93d0c100> (a java.util.Collections$UnmodifiableSet)                        
>         - locked <0x93d0be30> (a sun.nio.ch.EPollSelectorImpl)                                  
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)                                
>         at org.apache.hadoop.hbase.ipc.HBaseServer$Responder.run(HBaseServer.java:458)          
> "SocketListener0-1" prio=10 tid=0x088d5000 nid=0x87c in Object.wait() [0x8f2da000..0x8f2dae40]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                  
>         at java.lang.Object.wait(Native Method)                                              
>         at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:522)                    
>         - locked <0x93e39bc8> (a org.mortbay.util.ThreadPool$PoolThread)                      
> "SocketListener0-0" prio=10 tid=0x08ab5000 nid=0x87b in Object.wait() [0x8f32b000..0x8f32c0c0]
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                  
>         at java.lang.Object.wait(Native Method)                                              
>         at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:522)                    
>         - locked <0x93e39920> (a org.mortbay.util.ThreadPool$PoolThread)                      
> "Acceptor ServerSocket[addr=0.0.0.0/0.0.0.0,port=0,localport=60030]" prio=10 tid=0x084f6800 nid=0x87a runnable [0x8f37c000..0x8f37d140]
>    java.lang.Thread.State: RUNNABLE                                                                                                    
>         at java.net.PlainSocketImpl.socketAccept(Native Method)                                                                        
>         at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:384)                                                                  
>         - locked <0x93e382a8> (a java.net.SocksSocketImpl)                                                                            
>         at java.net.ServerSocket.implAccept(ServerSocket.java:453)                                                                    
>         at java.net.ServerSocket.accept(ServerSocket.java:421)                                                                        
>         at org.mortbay.util.ThreadedServer.acceptSocket(ThreadedServer.java:432)                                                      
>         at org.mortbay.util.ThreadedServer$Acceptor.run(ThreadedServer.java:631)                                                      
> "SessionScavenger" daemon prio=10 tid=0x088f1400 nid=0x879 waiting on condition [0x8f3cd000..0x8f3cdfc0]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                    
>         at java.lang.Thread.sleep(Native Method)                                                        
>         at org.mortbay.jetty.servlet.AbstractSessionManager$SessionScavenger.run(AbstractSessionManager.java:587)
> "SessionScavenger" daemon prio=10 tid=0x08ab5c00 nid=0x878 waiting on condition [0x8f41e000..0x8f41f040]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                    
>         at java.lang.Thread.sleep(Native Method)                                                        
>         at org.mortbay.jetty.servlet.AbstractSessionManager$SessionScavenger.run(AbstractSessionManager.java:587)
> "regionserver/0.0.0.0:60020.leaseChecker" prio=10 tid=0x086d5800 nid=0x877 waiting on condition [0x8f476000..0x8f476ec0]
>    java.lang.Thread.State: TIMED_WAITING (parking)                                                                      
>         at sun.misc.Unsafe.park(Native Method)                                                                          
>         - parking to wait for  <0x93d0e980> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)  
>         at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:198)                                      
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:1963)
>         at java.util.concurrent.DelayQueue.poll(DelayQueue.java:201)                                                            
>         at org.apache.hadoop.hbase.Leases.run(Leases.java:78)                                                                    
> "regionserver/0.0.0.0:60020.majorCompactionChecker" daemon prio=10 tid=0x086d4000 nid=0x876 waiting on condition [0x8f4c7000..0x8f4c7f40]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                                                      
>         at java.lang.Thread.sleep(Native Method)                                                                                        
>         at org.apache.hadoop.hbase.util.Sleeper.sleep(Sleeper.java:74)                                                                  
>         at org.apache.hadoop.hbase.Chore.run(Chore.java:72)                                                                              
> "LeaseChecker" daemon prio=10 tid=0x0837cc00 nid=0x870 waiting on condition [0x8f6ad000..0x8f6ae040]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                                
>         at java.lang.Thread.sleep(Native Method)                                                    
>         at org.apache.hadoop.hdfs.DFSClient$LeaseChecker.run(DFSClient.java:979)                    
>         at java.lang.Thread.run(Thread.java:619)                                                    
> "DataStreamer for file /hbase/log_192.168.1.13_1235129194745_60020/hlog.dat.1235129195649 block blk_7545556036225037274_1820952" daemon prio=10 tid=0x08375400 nid=0x86f in Object.wait() [0x8f6fe000..0x8f6feec0]                                                                                                      
>    java.lang.Thread.State: TIMED_WAITING (on object monitor)                                                                                                
>         at java.lang.Object.wait(Native Method)                                                                                                            
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2166)                                                          
>         - locked <0x93d0ea10> (a java.util.LinkedList)                                                                                                      
> "DestroyJavaVM" prio=10 tid=0x0805a000 nid=0x855 waiting on condition [0x00000000..0xb7dd2090]
>    java.lang.Thread.State: RUNNABLE                                                          
> "regionserver/0.0.0.0:60020" prio=10 tid=0x085b2400 nid=0x863 in Object.wait() [0x8f7f7000..0x8f7f7f40]
>    java.lang.Thread.State: WAITING (on object monitor)                                                
>         at java.lang.Object.wait(Native Method)                                                        
>         at java.lang.Object.wait(Object.java:485)                                                      
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.flushInternal(DFSClient.java:3015)        
>         - locked <0x93d0ea10> (a java.util.LinkedList)                                                
>         - locked <0x93d0ec58> (a org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)                    
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.closeInternal(DFSClient.java:3104)        
>         - locked <0x93d0ec58> (a org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)                    
>         at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.close(DFSClient.java:3053)
>         at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:59)
>         at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:79)
>         at org.apache.hadoop.io.SequenceFile$Writer.close(SequenceFile.java:959)
>         - locked <0x93d50d20> (a org.apache.hadoop.io.SequenceFile$Writer)
>         at org.apache.hadoop.hbase.regionserver.HLog.close(HLog.java:421)
>         - locked <0x93d4ce00> (a java.lang.Integer)
>         at org.apache.hadoop.hbase.regionserver.HLog.closeAndDelete(HLog.java:404)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:373)
>         at java.lang.Thread.run(Thread.java:619)
> "Low Memory Detector" daemon prio=10 tid=0x080dd800 nid=0x85e runnable [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread1" daemon prio=10 tid=0x080dc400 nid=0x85d waiting on condition [0x00000000..0x8fbc1588]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread0" daemon prio=10 tid=0x080d9c00 nid=0x85c waiting on condition [0x00000000..0x8fc42608]
>    java.lang.Thread.State: RUNNABLE
> "Signal Dispatcher" daemon prio=10 tid=0x080d8800 nid=0x85b runnable [0x00000000..0x8fc93e80]
>    java.lang.Thread.State: RUNNABLE
> "Surrogate Locker Thread (CMS)" daemon prio=10 tid=0x080d7800 nid=0x85a waiting on condition [0x00000000..0x8fce522c]
>    java.lang.Thread.State: RUNNABLE
> "Finalizer" daemon prio=10 tid=0x080bbc00 nid=0x859 in Object.wait() [0x8fd7b000..0x8fd7bec0]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116)
>         - locked <0x93d0bdc0> (a java.lang.ref.ReferenceQueue$Lock)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:132)
>         at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159)
> "Reference Handler" daemon prio=10 tid=0x080ba800 nid=0x858 in Object.wait() [0x8fdcc000..0x8fdccf40]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.Object.wait(Object.java:485)
>         at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)
>         - locked <0x93d0c110> (a java.lang.ref.Reference$Lock)
> "VM Thread" prio=10 tid=0x080b7800 nid=0x857 runnable
> "Concurrent Mark-Sweep GC Thread" prio=10 tid=0x0806c800 nid=0x856 runnable
> "VM Periodic Task Thread" prio=10 tid=0x080df000 nid=0x85f waiting on condition
> JNI global references: 922
> {noformat}
> Notice the DataStreamer thread that seems to work on the concerned block
> This force me to kill (-KILL) the region server, and restart the hdfs (if I don't restart hdfs some regions are not available because the block is still not available).
> Note that the Hbase master does not end as well, and I have to kill (-KILL). Here is the thread dump of the HMaster process:
> h5. Master thread dump:
> {noformat}
> Full thread dump Java HotSpot(TM) Server VM (10.0-b23 mixed mode):
> "Attach Listener" daemon prio=10 tid=0x0805a000 nid=0x2326 waiting on condition [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE                                                                    
> "SIGTERM handler" daemon prio=10 tid=0x089f0c00 nid=0x2303 waiting for monitor entry [0x8a6a5000..0x8a6a5ec0]
>    java.lang.Thread.State: BLOCKED (on object monitor)                                                      
>         at java.lang.Shutdown.exit(Shutdown.java:178)                                                        
>         - waiting to lock <0xb0beed60> (a java.lang.Class for java.lang.Shutdown)                            
>         at java.lang.Terminator$1.handle(Terminator.java:35)                                                
>         at sun.misc.Signal$1.run(Signal.java:195)                                                            
>         at java.lang.Thread.run(Thread.java:619)                                                            
> "Thread-1" prio=10 tid=0x8f666000 nid=0x653c waiting on condition [0x8f1fe000..0x8f1fef40]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)                                      
>         at java.lang.Thread.sleep(Native Method)                                          
>         at org.apache.hadoop.ipc.Client.stop(Client.java:667)                            
>         at org.apache.hadoop.ipc.RPC$ClientCache.stopClient(RPC.java:189)                
>         at org.apache.hadoop.ipc.RPC$ClientCache.access$400(RPC.java:138)                
>         at org.apache.hadoop.ipc.RPC$Invoker.close(RPC.java:229)                          
>         - locked <0x938a1b18> (a org.apache.hadoop.ipc.RPC$Invoker)                      
>         at org.apache.hadoop.ipc.RPC$Invoker.access$500(RPC.java:196)                    
>         at org.apache.hadoop.ipc.RPC.stopProxy(RPC.java:353)                              
>         at org.apache.hadoop.hdfs.DFSClient.close(DFSClient.java:213)                    
>         - locked <0x93878470> (a org.apache.hadoop.hdfs.DFSClient)                        
>         at org.apache.hadoop.hdfs.DistributedFileSystem.close(DistributedFileSystem.java:243)
>         at org.apache.hadoop.fs.FileSystem$Cache.closeAll(FileSystem.java:1413)              
>         - locked <0x937d3c00> (a org.apache.hadoop.fs.FileSystem$Cache)                      
>         at org.apache.hadoop.fs.FileSystem.closeAll(FileSystem.java:236)                    
>         at org.apache.hadoop.fs.FileSystem$ClientFinalizer.run(FileSystem.java:221)          
>         - locked <0x937b8a78> (a org.apache.hadoop.fs.FileSystem$ClientFinalizer)            
> "DestroyJavaVM" prio=10 tid=0x8f66c800 nid=0x36e9 in Object.wait() [0xb7d7d000..0xb7d7e0e0]
>    java.lang.Thread.State: WAITING (on object monitor)                                    
>         at java.lang.Object.wait(Native Method)                                            
>         at java.lang.Thread.join(Thread.java:1143)                                        
>         - locked <0x937b8a78> (a org.apache.hadoop.fs.FileSystem$ClientFinalizer)          
>         at java.lang.Thread.join(Thread.java:1196)
>         at java.lang.ApplicationShutdownHooks.run(ApplicationShutdownHooks.java:79)
>         at java.lang.Shutdown.runHooks(Shutdown.java:89)
>         at java.lang.Shutdown.sequence(Shutdown.java:133)
>         at java.lang.Shutdown.shutdown(Shutdown.java:200)
>         - locked <0xb0beed60> (a java.lang.Class for java.lang.Shutdown)
> "Low Memory Detector" daemon prio=10 tid=0x8fe03800 nid=0x36f4 runnable [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread1" daemon prio=10 tid=0x8fe01c00 nid=0x36f3 waiting on condition [0x00000000..0x8f9ee488]
>    java.lang.Thread.State: RUNNABLE
> "CompilerThread0" daemon prio=10 tid=0x8fe00800 nid=0x36f2 waiting on condition [0x00000000..0x8fa6f508]
>    java.lang.Thread.State: RUNNABLE
> "Signal Dispatcher" daemon prio=10 tid=0x0813e800 nid=0x36f1 runnable [0x00000000..0x8fac0d80]
>    java.lang.Thread.State: RUNNABLE
> "Surrogate Locker Thread (CMS)" daemon prio=10 tid=0x0813d800 nid=0x36f0 waiting on condition [0x00000000..0x00000000]
>    java.lang.Thread.State: RUNNABLE
> "Finalizer" daemon prio=10 tid=0x08125000 nid=0x36ef in Object.wait() [0x8fba1000..0x8fba1dc0]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116)
>         - locked <0x937bafd8> (a java.lang.ref.ReferenceQueue$Lock)
>         at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:132)
>         at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159)
> "Reference Handler" daemon prio=10 tid=0x08120c00 nid=0x36ee in Object.wait() [0x8fbf2000..0x8fbf2e40]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.Object.wait(Object.java:485)
>         at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)
>         - locked <0x937baf18> (a java.lang.ref.Reference$Lock)
> "VM Thread" prio=10 tid=0x0811d800 nid=0x36ed runnable
> "Gang worker#0 (Parallel GC Threads)" prio=10 tid=0x0805e000 nid=0x36ea runnable
> "Gang worker#1 (Parallel GC Threads)" prio=10 tid=0x0805f000 nid=0x36eb runnable
> "Concurrent Mark-Sweep GC Thread" prio=10 tid=0x080cd000 nid=0x36ec runnable
> "VM Periodic Task Thread" prio=10 tid=0x8fe05000 nid=0x36f5 waiting on condition
> JNI global references: 1044
> {noformat}
> When I restart dfs, the log shows me that the concerned block is marked as invalidate, and it is therefore deleted:
> h5. Namenode log
> {noformat}
> 2009-02-23 10:33:49,245 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.processReport: block blk_7545556036225037274_1820952 on 192.168.1.10:50010 size 27895296 does not
>  belong to any file.
> 2009-02-23 10:33:49,245 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addToInvalidates: blk_7545556036225037274 is added to invalidSet of 192.168.1.10:50010
> {noformat}
> Finally, when I restart HBase, I can see some of these messages in region server log, and I have some data loss. (this log entry may not be linked with the  block traced before)
> h5. Region server log:
> {noformat}
> 2009-02-23 10:38:55,831 WARN org.apache.hadoop.hbase.regionserver.HStore: Exception processing reconstruction log hdfs://lab5-2:9000/hbase/test-D-0.3/510178748/oldlogfile.log opening [B@ff65bf -- continuing.  Probably lack-of-HADOOP-1700 causing DATA LOSS!
> java.io.EOFException
>         at java.io.DataInputStream.readFully(DataInputStream.java:180)
>         at org.apache.hadoop.hbase.io.DataOutputBuffer$Buffer.write(DataOutputBuffer.java:70)
>         at org.apache.hadoop.hbase.io.DataOutputBuffer.write(DataOutputBuffer.java:116)
>         at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1944)
>         at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1844)
>         at org.apache.hadoop.hbase.io.SequenceFile$Reader.next(SequenceFile.java:1890)
>         at org.apache.hadoop.hbase.regionserver.HStore.doReconstructionLog(HStore.java:352)
>         at org.apache.hadoop.hbase.regionserver.HStore.runReconstructionLog(HStore.java:297)
>         at org.apache.hadoop.hbase.regionserver.HStore.<init>(HStore.java:237)
>         at org.apache.hadoop.hbase.regionserver.HRegion.instantiateHStore(HRegion.java:1764)
>         at org.apache.hadoop.hbase.regionserver.HRegion.initialize(HRegion.java:276)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.instantiateRegion(HRegionServer.java:1367)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.openRegion(HRegionServer.java:1338)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer$Worker.run(HRegionServer.java:1253)
>         at java.lang.Thread.run(Thread.java:619)
> {noformat}
> In my sense, since my hardware is in cause for RegionServer lease miss, I have to deal with the fact that I have to stop/restart hbase + hadoop in such a case. The problem in my sense is that a stop request should be able to perform a clean shutdown, and to avoid dataloss.
> Hope this description helps.

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