dfs exception and regionserver stuck during heavy write load
------------------------------------------------------------

                 Key: HBASE-751
                 URL: https://issues.apache.org/jira/browse/HBASE-751
             Project: Hadoop HBase
          Issue Type: Bug
          Components: regionserver
    Affects Versions: 0.2.0
         Environment: jdk 1.6, hadoop 0.17.2-dev (hadoop-0.17, r677779), hbase 
trunk r677517
            Reporter: Rong-En Fan


It's a 3 node setup, each runs datanode and regionserver. One runs as hbase 
master and hadoop namenode.

After some heavy write load via java client, the client is stuck. Stack trace 
on the regionserver shows:

"IPC Server handler 46 on 60020" daemon prio=10 tid=0x4dd3f000 nid=0x4eb3 
waiting for monitor entry [0x4cc82000..0x4cc83130]
   java.lang.Thread.State: BLOCKED (on object monitor)
    at 
org.apache.hadoop.hbase.regionserver.HStore.rowAtOrBeforeFromMapFile(HStore.java:1424)
    - waiting to lock <0x59d27ba0> (a 
org.apache.hadoop.hbase.regionserver.HStoreFile$BloomFilterMapFile$Reader)
    at 
org.apache.hadoop.hbase.regionserver.HStore.getRowKeyAtOrBefore(HStore.java:1399)
    at 
org.apache.hadoop.hbase.regionserver.HRegion.getClosestRowBefore(HRegion.java:1210)
    at 
org.apache.hadoop.hbase.regionserver.HRegionServer.getClosestRowBefore(HRegionServer.java:1099)
    at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source)
    at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
    at java.lang.reflect.Method.invoke(Method.java:597)
    at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:438)
    at org.apache.hadoop.ipc.Server$Handler.run(Server.java:896)

"IPC Server handler 43 on 60020" daemon prio=10 tid=0x4dd3bc00 nid=0x4eb0 
waiting for monitor entry [0x4cd75000..0x4cd75fb0]
   java.lang.Thread.State: BLOCKED (on object monitor)
    at 
org.apache.hadoop.hbase.regionserver.HStore.rowAtOrBeforeFromMapFile(HStore.java:1424)
    - waiting to lock <0x59d27ba0> (a 
org.apache.hadoop.hbase.regionserver.HStoreFile$BloomFilterMapFile$Reader)
    at 
org.apache.hadoop.hbase.regionserver.HStore.getRowKeyAtOrBefore(HStore.java:1399)
    at 
org.apache.hadoop.hbase.regionserver.HRegion.getClosestRowBefore(HRegion.java:1210)
    at 
org.apache.hadoop.hbase.regionserver.HRegionServer.getClosestRowBefore(HRegionServer.java:1099)
    at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source)
    at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
    at java.lang.reflect.Method.invoke(Method.java:597)
    at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:438)
    at org.apache.hadoop.ipc.Server$Handler.run(Server.java:896)

"IPC Server handler 40 on 60020" daemon prio=10 tid=0x4dd38400 nid=0x4ead 
runnable [0x4ce68000..0x4ce68e30]
   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 <0x6a557580> (a sun.nio.ch.Util$1)
    - locked <0x6a557570> (a java.util.Collections$UnmodifiableSet)
    - locked <0x5cdcec18> (a sun.nio.ch.EPollSelectorImpl)
    at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
    at 
org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:237)
    at 
org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:155)
    at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:149)
    at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:122)
    at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
    at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
    - locked <0x552ffb60> (a java.io.BufferedInputStream)
    at java.io.DataInputStream.readInt(DataInputStream.java:370)
    at org.apache.hadoop.dfs.DFSClient$BlockReader.readChunk(DFSClient.java:928)
    - locked <0x55300f78> (a org.apache.hadoop.dfs.DFSClient$BlockReader)
    at 
org.apache.hadoop.fs.FSInputChecker.readChecksumChunk(FSInputChecker.java:236)
    at org.apache.hadoop.fs.FSInputChecker.fill(FSInputChecker.java:178)
    at org.apache.hadoop.fs.FSInputChecker.read1(FSInputChecker.java:195)
    at org.apache.hadoop.fs.FSInputChecker.read(FSInputChecker.java:159)
    - locked <0x55300f78> (a org.apache.hadoop.dfs.DFSClient$BlockReader)
    at org.apache.hadoop.dfs.DFSClient$BlockReader.read(DFSClient.java:823)
    - locked <0x55300f78> (a org.apache.hadoop.dfs.DFSClient$BlockReader)
    at 
org.apache.hadoop.dfs.DFSClient$DFSInputStream.readBuffer(DFSClient.java:1352)
    - locked <0x59a70e40> (a org.apache.hadoop.dfs.DFSClient$DFSInputStream)
    at org.apache.hadoop.dfs.DFSClient$DFSInputStream.read(DFSClient.java:1388)
    - locked <0x59a70e40> (a org.apache.hadoop.dfs.DFSClient$DFSInputStream)
    at org.apache.hadoop.dfs.DFSClient$DFSInputStream.read(DFSClient.java:1337)
    - locked <0x59a70e40> (a org.apache.hadoop.dfs.DFSClient$DFSInputStream)
    at java.io.DataInputStream.readInt(DataInputStream.java:370)
    at 
org.apache.hadoop.io.SequenceFile$Reader.readRecordLength(SequenceFile.java:1847)
    - locked <0x651f77b0> (a org.apache.hadoop.io.SequenceFile$Reader)
    at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:1877)
    - locked <0x651f77b0> (a org.apache.hadoop.io.SequenceFile$Reader)
    at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:1782)
    - locked <0x651f77b0> (a org.apache.hadoop.io.SequenceFile$Reader)
    at org.apache.hadoop.io.MapFile$Reader.seekInternal(MapFile.java:476)
    - locked <0x59d27ba0> (a 
org.apache.hadoop.hbase.regionserver.HStoreFile$BloomFilterMapFile$Reader)
    at org.apache.hadoop.io.MapFile$Reader.getClosest(MapFile.java:558)
    - locked <0x59d27ba0> (a 
org.apache.hadoop.hbase.regionserver.HStoreFile$BloomFilterMapFile$Reader)
    at 
org.apache.hadoop.hbase.regionserver.HStore.rowKeyFromMapFileEmptyKeys(HStore.java:1463)
    at 
org.apache.hadoop.hbase.regionserver.HStore.rowAtOrBeforeFromMapFile(HStore.java:1434)
    - locked <0x59d27ba0> (a 
org.apache.hadoop.hbase.regionserver.HStoreFile$BloomFilterMapFile$Reader)
    at 
org.apache.hadoop.hbase.regionserver.HStore.getRowKeyAtOrBefore(HStore.java:1399)
    at 
org.apache.hadoop.hbase.regionserver.HRegion.getClosestRowBefore(HRegion.java:1210)
    at 
org.apache.hadoop.hbase.regionserver.HRegionServer.getClosestRowBefore(HRegionServer.java:1099)
    at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source)
    at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
    at java.lang.reflect.Method.invoke(Method.java:597)
    at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:438)
    at org.apache.hadoop.ipc.Server$Handler.run(Server.java:896)

"IPC Server handler 38 on 60020" daemon prio=10 tid=0x4dd36000 nid=0x4eab 
waiting for monitor entry [0x4cf0a000..0x4cf0b130]
   java.lang.Thread.State: BLOCKED (on object monitor)
    at 
org.apache.hadoop.hbase.regionserver.HStore.rowAtOrBeforeFromMapFile(HStore.java:1424)
    - waiting to lock <0x59d27ba0> (a 
org.apache.hadoop.hbase.regionserver.HStoreFile$BloomFilterMapFile$Reader)
    at 
org.apache.hadoop.hbase.regionserver.HStore.getRowKeyAtOrBefore(HStore.java:1399)
    at 
org.apache.hadoop.hbase.regionserver.HRegion.getClosestRowBefore(HRegion.java:1210)
    at 
org.apache.hadoop.hbase.regionserver.HRegionServer.getClosestRowBefore(HRegionServer.java:1099)
    at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source)
    at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
    at java.lang.reflect.Method.invoke(Method.java:597)
    at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:438)
    at org.apache.hadoop.ipc.Server$Handler.run(Server.java:896)

"IPC Server handler 37 on 60020" daemon prio=10 tid=0x4dd35000 nid=0x4eaa 
waiting for monitor entry [0x4cf5b000..0x4cf5c0b0]
   java.lang.Thread.State: BLOCKED (on object monitor)
    at 
org.apache.hadoop.hbase.regionserver.HStore.rowAtOrBeforeFromMapFile(HStore.java:1424)
    - waiting to lock <0x59d27ba0> (a 
org.apache.hadoop.hbase.regionserver.HStoreFile$BloomFilterMapFile$Reader)
    at 
org.apache.hadoop.hbase.regionserver.HStore.getRowKeyAtOrBefore(HStore.java:1399)
    at 
org.apache.hadoop.hbase.regionserver.HRegion.getClosestRowBefore(HRegion.java:1210)
    at 
org.apache.hadoop.hbase.regionserver.HRegionServer.getClosestRowBefore(HRegionServer.java:1099)
    at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source)
    at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
    at java.lang.reflect.Method.invoke(Method.java:597)
    at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:438)
    at org.apache.hadoop.ipc.Server$Handler.run(Server.java:896)

"IPC Server handler 35 on 60020" daemon prio=10 tid=0x4dd32c00 nid=0x4ea8 
waiting for monitor entry [0x4cffd000..0x4cffdfb0]
   java.lang.Thread.State: BLOCKED (on object monitor)
    at 
org.apache.hadoop.hbase.regionserver.HStore.rowAtOrBeforeFromMapFile(HStore.java:1424)
    - waiting to lock <0x59d27ba0> (a 
org.apache.hadoop.hbase.regionserver.HStoreFile$BloomFilterMapFile$Reader)
    at 
org.apache.hadoop.hbase.regionserver.HStore.getRowKeyAtOrBefore(HStore.java:1399)
    at 
org.apache.hadoop.hbase.regionserver.HRegion.getClosestRowBefore(HRegion.java:1210)
    at 
org.apache.hadoop.hbase.regionserver.HRegionServer.getClosestRowBefore(HRegionServer.java:1099)
    at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source)
    at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
    at java.lang.reflect.Method.invoke(Method.java:597)
    at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:438)
    at org.apache.hadoop.ipc.Server$Handler.run(Server.java:896)

"IPC Server handler 30 on 60020" daemon prio=10 tid=0x4dd2d400 nid=0x4ea3 
waiting for monitor entry [0x4d192000..0x4d193130]
   java.lang.Thread.State: BLOCKED (on object monitor)
    at 
org.apache.hadoop.hbase.regionserver.HStore.rowAtOrBeforeFromMapFile(HStore.java:1424)
    - waiting to lock <0x59d27ba0> (a 
org.apache.hadoop.hbase.regionserver.HStoreFile$BloomFilterMapFile$Reader)
    at 
org.apache.hadoop.hbase.regionserver.HStore.getRowKeyAtOrBefore(HStore.java:1399)
    at 
org.apache.hadoop.hbase.regionserver.HRegion.getClosestRowBefore(HRegion.java:1210)
    at 
org.apache.hadoop.hbase.regionserver.HRegionServer.getClosestRowBefore(HRegionServer.java:1099)
    at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source)
    at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
    at java.lang.reflect.Method.invoke(Method.java:597)
    at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:438)
    at org.apache.hadoop.ipc.Server$Handler.run(Server.java:896)

"IPC Server handler 29 on 60020" daemon prio=10 tid=0x4dd2c000 nid=0x4ea2 
waiting for monitor entry [0x4d1e3000..0x4d1e40b0]
   java.lang.Thread.State: BLOCKED (on object monitor)
    at 
org.apache.hadoop.hbase.regionserver.HStore.rowAtOrBeforeFromMapFile(HStore.java:1424)
    - waiting to lock <0x59d27ba0> (a 
org.apache.hadoop.hbase.regionserver.HStoreFile$BloomFilterMapFile$Reader)
    at 
org.apache.hadoop.hbase.regionserver.HStore.getRowKeyAtOrBefore(HStore.java:1399)
    at 
org.apache.hadoop.hbase.regionserver.HRegion.getClosestRowBefore(HRegion.java:1210)
    at 
org.apache.hadoop.hbase.regionserver.HRegionServer.getClosestRowBefore(HRegionServer.java:1099)
    at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source)
    at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
    at java.lang.reflect.Method.invoke(Method.java:597)
    at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:438)
    at org.apache.hadoop.ipc.Server$Handler.run(Server.java:896)

"IPC Server handler 26 on 60020" daemon prio=10 tid=0x4dd29800 nid=0x4e9f 
waiting for monitor entry [0x4d2d6000..0x4d2d6f30]
   java.lang.Thread.State: BLOCKED (on object monitor)
    at 
org.apache.hadoop.hbase.regionserver.HStore.rowAtOrBeforeFromMapFile(HStore.java:1424)
    - waiting to lock <0x59d27ba0> (a 
org.apache.hadoop.hbase.regionserver.HStoreFile$BloomFilterMapFile$Reader)
    at 
org.apache.hadoop.hbase.regionserver.HStore.getRowKeyAtOrBefore(HStore.java:1399)
    at 
org.apache.hadoop.hbase.regionserver.HRegion.getClosestRowBefore(HRegion.java:1210)
    at 
org.apache.hadoop.hbase.regionserver.HRegionServer.getClosestRowBefore(HRegionServer.java:1099)
    at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source)
    at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
    at java.lang.reflect.Method.invoke(Method.java:597)
    at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:438)
    at org.apache.hadoop.ipc.Server$Handler.run(Server.java:896)

"IPC Server handler 17 on 60020" daemon prio=10 tid=0x4dd1f800 nid=0x4e96 
waiting for monitor entry [0x4d5af000..0x4d5afeb0]
   java.lang.Thread.State: BLOCKED (on object monitor)
    at 
org.apache.hadoop.hbase.regionserver.HStore.rowAtOrBeforeFromMapFile(HStore.java:1424)
    - waiting to lock <0x59d27ba0> (a 
org.apache.hadoop.hbase.regionserver.HStoreFile$BloomFilterMapFile$Reader)
    at 
org.apache.hadoop.hbase.regionserver.HStore.getRowKeyAtOrBefore(HStore.java:1399)
    at 
org.apache.hadoop.hbase.regionserver.HRegion.getClosestRowBefore(HRegion.java:1210)
    at 
org.apache.hadoop.hbase.regionserver.HRegionServer.getClosestRowBefore(HRegionServer.java:1099)
    at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source)
    at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
    at java.lang.reflect.Method.invoke(Method.java:597)
    at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:438)
    at org.apache.hadoop.ipc.Server$Handler.run(Server.java:896)

"IPC Server handler 14 on 60020" daemon prio=10 tid=0x4dd1c400 nid=0x4e93 
waiting for monitor entry [0x4d6a2000..0x4d6a3130]
   java.lang.Thread.State: BLOCKED (on object monitor)
    at 
org.apache.hadoop.hbase.regionserver.HStore.rowAtOrBeforeFromMapFile(HStore.java:1424)
    - waiting to lock <0x59d27ba0> (a 
org.apache.hadoop.hbase.regionserver.HStoreFile$BloomFilterMapFile$Reader)
    at 
org.apache.hadoop.hbase.regionserver.HStore.getRowKeyAtOrBefore(HStore.java:1399)
    at 
org.apache.hadoop.hbase.regionserver.HRegion.getClosestRowBefore(HRegion.java:1210)
    at 
org.apache.hadoop.hbase.regionserver.HRegionServer.getClosestRowBefore(HRegionServer.java:1099)
    at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source)
    at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
    at java.lang.reflect.Method.invoke(Method.java:597)
    at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:438)
    at org.apache.hadoop.ipc.Server$Handler.run(Server.java:896)

"IPC Server handler 10 on 60020" daemon prio=10 tid=0x4dd17c00 nid=0x4e8f 
waiting for monitor entry [0x4d7e6000..0x4d7e6f30]
   java.lang.Thread.State: BLOCKED (on object monitor)
    at 
org.apache.hadoop.hbase.regionserver.HStore.rowAtOrBeforeFromMapFile(HStore.java:1424)
    - waiting to lock <0x59d27ba0> (a 
org.apache.hadoop.hbase.regionserver.HStoreFile$BloomFilterMapFile$Reader)
    at 
org.apache.hadoop.hbase.regionserver.HStore.getRowKeyAtOrBefore(HStore.java:1399)
    at 
org.apache.hadoop.hbase.regionserver.HRegion.getClosestRowBefore(HRegion.java:1210)
    at 
org.apache.hadoop.hbase.regionserver.HRegionServer.getClosestRowBefore(HRegionServer.java:1099)
    at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source)
    at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
    at java.lang.reflect.Method.invoke(Method.java:597)
    at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:438)
    at org.apache.hadoop.ipc.Server$Handler.run(Server.java:896)

"IPC Server handler 7 on 60020" daemon prio=10 tid=0x4dd14800 nid=0x4e8c 
waiting for monitor entry [0x4d8d9000..0x4d8da1b0]
   java.lang.Thread.State: BLOCKED (on object monitor)
    at 
org.apache.hadoop.hbase.regionserver.HStore.rowAtOrBeforeFromMapFile(HStore.java:1424)
    - waiting to lock <0x59d27ba0> (a 
org.apache.hadoop.hbase.regionserver.HStoreFile$BloomFilterMapFile$Reader)
    at 
org.apache.hadoop.hbase.regionserver.HStore.getRowKeyAtOrBefore(HStore.java:1399)
    at 
org.apache.hadoop.hbase.regionserver.HRegion.getClosestRowBefore(HRegion.java:1210)
    at 
org.apache.hadoop.hbase.regionserver.HRegionServer.getClosestRowBefore(HRegionServer.java:1099)
    at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source)
    at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
    at java.lang.reflect.Method.invoke(Method.java:597)
    at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:438)
    at org.apache.hadoop.ipc.Server$Handler.run(Server.java:896)

"IPC Server handler 0 on 60020" daemon prio=10 tid=0x4e2c0c00 nid=0x4e85 
waiting for monitor entry [0x4db10000..0x4db10e30]
   java.lang.Thread.State: BLOCKED (on object monitor)
    at 
org.apache.hadoop.hbase.regionserver.HStore.rowAtOrBeforeFromMapFile(HStore.java:1424)
    - waiting to lock <0x59d27ba0> (a 
org.apache.hadoop.hbase.regionserver.HStoreFile$BloomFilterMapFile$Reader)
    at 
org.apache.hadoop.hbase.regionserver.HStore.getRowKeyAtOrBefore(HStore.java:1399)
    at 
org.apache.hadoop.hbase.regionserver.HRegion.getClosestRowBefore(HRegion.java:1210)
    at 
org.apache.hadoop.hbase.regionserver.HRegionServer.getClosestRowBefore(HRegionServer.java:1099)
    at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source)
    at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
    at java.lang.reflect.Method.invoke(Method.java:597)
    at org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:438)
    at org.apache.hadoop.ipc.Server$Handler.run(Server.java:896)

in regionserver log, I see the following right before the client stuck (there 
are few other similar logs, but the client keeps going at those time points):

2008-07-17 22:31:49,404 INFO org.apache.hadoop.hbase.regionserver.HRegion: 
region aaa,bbb,1216304670433/1145836031 available
2008-07-17 22:31:49,404 INFO org.apache.hadoop.hbase.regionserver.HRegion: 
starting compaction on region aaa,bbb,1216304670433
2008-07-17 22:32:07,653 WARN org.apache.hadoop.hbase.regionserver.HStore: 
Exception closing reader for 1145836031/ccc
java.io.IOException: Stream closed
    at org.apache.hadoop.dfs.DFSClient$DFSInputStream.close(DFSClient.java:1319)
    at java.io.FilterInputStream.close(FilterInputStream.java:155)
    at org.apache.hadoop.io.SequenceFile$Reader.close(SequenceFile.java:1581)
    at org.apache.hadoop.io.MapFile$Reader.close(MapFile.java:577)
    at 
org.apache.hadoop.hbase.regionserver.HStore.closeCompactionReaders(HStore.java:917)
    at 
org.apache.hadoop.hbase.regionserver.HStore.compactHStoreFiles(HStore.java:910)
    at org.apache.hadoop.hbase.regionserver.HStore.compact(HStore.java:787)
    at 
org.apache.hadoop.hbase.regionserver.HRegion.compactStores(HRegion.java:887)
    at 
org.apache.hadoop.hbase.regionserver.HRegion.compactStores(HRegion.java:847)
    at 
org.apache.hadoop.hbase.regionserver.CompactSplitThread.run(CompactSplitThread.java:84)
(and two of the same exception, since I have 3 HStoreFIle to compact)
2008-07-17 22:32:07,912 INFO org.apache.hadoop.hbase.regionserver.HRegion: 
compaction completed on region aaa,bbb,1216304670433 in 18sec
[after this point, I only see regionserver rotates HLog, no other activities)

At 22:32, no suspicious log in datanode, but 8mins later, I see this

2008-07-17 22:40:07,928 WARN org.apache.hadoop.dfs.DataNode: 
192.168.1.5650010:Got exception while serving blk_-38731635936101350 to 
/192.168.1.56
java.net.SocketTimeoutException: 480000 millis timeout while waiting for 
channel to be ready for write. ch : java.nio.channels.SocketChannel[connected 
local=/192.168.1.56:50010 remote=/192.168.1.56:40691]
    at 
org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:170)
    at 
org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:144)
    at 
org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:105)
    at java.io.BufferedOutputStream.write(BufferedOutputStream.java:105)
    at java.io.DataOutputStream.write(DataOutputStream.java:90)
    at org.apache.hadoop.dfs.DataNode$BlockSender.sendChunks(DataNode.java:1784)
    at org.apache.hadoop.dfs.DataNode$BlockSender.sendBlock(DataNode.java:1840)
    at org.apache.hadoop.dfs.DataNode$DataXceiver.readBlock(DataNode.java:1055)
    at org.apache.hadoop.dfs.DataNode$DataXceiver.run(DataNode.java:984)
    at java.lang.Thread.run(Thread.java:619)

for this particular block in question, I found around the region available time:

2008-07-17 22:31:49,642 INFO org.apache.hadoop.dfs.DataNode: Receiving block 
blk_-38731635936101350 src: /192.168.1.56:37878 dest: /192.168.1.56:50010
2008-07-17 22:31:56,856 INFO org.apache.hadoop.dfs.DataNode: Received block 
blk_-38731635936101350 of size 67108864 from /192.168.1.56
2008-07-17 22:31:56,857 INFO org.apache.hadoop.dfs.DataNode: PacketResponder 1 
for block blk_-38731635936101350 terminating

And after the hbase client stuck, I found one datanode keeps sending the *same* 
block to the regionserver, which is blocked as shown above.

=====

For the record, I did not see this "Stream closed" error on another small 
4-node cluster with trunk r675659 (same hadoop version with the 3-node cluster 
above).

For hbase trunk r677011, I got 

java.lang.NullPointerException
        at 
org.apache.hadoop.hbase.client.ServerCallable.getServerName(ServerCallable.java:63)
        at 
org.apache.hadoop.hbase.client.HConnectionManager$TableServers.getRegionServerWithRetries(HConnectionManager.java:886
        at org.apache.hadoop.hbase.client.HTable.commit(HTable.java:1161)

then, the region server stucks

08/07/18 05:29:29 INFO ipc.RPC: Problem connecting to server: 
/192.168.1.56:60020

stack dump shows similar as the above one, and I'm also seeing the dfs 
exception.

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

Reply via email to