You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@hbase.apache.org by Asaf Mesika <as...@gmail.com> on 2014/04/29 21:46:17 UTC

Re: RegionServer stuck in internalObtainRowLock forever - HBase 0.94.7

We had this issue again in production. We had to shutdown the region
server. Restart didn't help since this RS was bombarded with write requests
and execute coprocessors requests, which made it open regions in the rate
of 1 region in 2 minutes or so.

Do you think its related to this jira:
https://issues.apache.org/jira/browse/HBASE-7711 ?

This time we checked, and no failed disks on the machine, but same problem
with failing writing to local data node and then it was excluded.



On Wed, Feb 19, 2014 at 1:02 AM, Stack <st...@duboce.net> wrote:

> On Mon, Feb 17, 2014 at 1:59 AM, Asaf Mesika <as...@gmail.com>
> wrote:
>
> > Hi,
> >
> > Apparently this just happened on Staging machine as well. The common
> ground
> > between is a failed disk (1 out of 8).
> >
> > It seems like a bug if HBase can't recover from a failed disk. Could it
> be
> > that short circuit is causing it?
> >
> >
> You could down the timeouts so that the bad disk didn't hold up hbase
> reads/writes for so long.
>
> Your disks are just dying?  They are not degrading first?
>
>
>
> > Couple of interesting exceptions:
> >
> > 1. The following repeated several time
> >
> > 2014-02-16 21:20:02,850 WARN org.apache.hadoop.hdfs.DFSClient:
> > DFSOutputStream ResponseProcessor exception  for block
> >
> BP-1188452996-10.223.226.16-1391605188016:blk_-2678974508557130432_1358017
> > java.io.IOException: Bad response ERROR for block
> >
> BP-1188452996-10.223.226.16-1391605188016:blk_-2678974508557130432_1358017
> > from datanode 10.223.226.66:50010
> > at
> >
> >
> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer$ResponseProcessor.run(DFSOutputStream.java:705)
> > 2014-02-16 21:21:09,640 INFO org.apache.hadoop.hdfs.DFSClient: Exception
> in
> > createBlockOutputStream
> > java.net.SocketTimeoutException: 75000 millis timeout while waiting for
> > channel to be ready for read. ch :
> > java.nio.channels.SocketChannel[connected local=/10.223.226.91:39217
> > remote=/
> > 10.223.226.91:50010]
> > at
> >
> >
> org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:165)
> > at
> org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:156)
> > at
> org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:129)
> > at
> org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:117)
> > at java.io.FilterInputStream.read(FilterInputStream.java:66)
> > at java.io.FilterInputStream.read(FilterInputStream.java:66)
> > at
> >
> >
> org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed(HdfsProtoUtil.java:169)
> > at
> >
> >
> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.createBlockOutputStream(DFSOutputStream.java:1105)
> > at
> >
> >
> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.nextBlockOutputStream(DFSOutputStream.java:1039)
> > at
> >
> >
> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.run(DFSOutputStream.java:487)
> > 2014-02-16 21:21:09,641 INFO org.apache.hadoop.hdfs.DFSClient: Abandoning
> >
> BP-1188452996-10.223.226.16-1391605188016:blk_-4136849541182435905_1359225
> > 2014-02-16 21:21:09,658 INFO org.apache.hadoop.hdfs.DFSClient: Excluding
> > datanode 10.223.226.91:50010
> >
> >
> That was local DN?  You think we moved on to the other replica so
> read/writes progressed afterward?  You might want to tinker w/ some of your
> timeouts to make them fail over replicas faster.
>
>
>
>
> > 2. Then we started gettings tons of the following exception interwined
> with
> > exception listed above in 1
> >
> > 2014-02-16 21:23:45,789 WARN org.apache.hadoop.ipc.HBaseServer:
> > (responseTooSlow):
> >
> >
> {"processingtimems":75164,"call":"multi(org.apache.hadoop.hbase.client.MultiAction@2588f0a
> > ),
> > rpc version=1, client version=29,
> > methodsFingerPrint=-1368823753","client":"
> > 10.223.226.165:30412
> >
> >
> ","starttimems":1392585750594,"queuetimems":0,"class":"HRegionServer","responsesize":0,"method":"multi"}
> >
> >
> Latencies were going up before this on this node?  You monitor your disks?
>  Any increase in reported errors?  Complaints in dmesg, etc. that could
> have given you some forenotice?
>
>
>
> > 3. Then we got this:
> >
> > 2014-02-16 22:03:47,626 WARN org.apache.hadoop.hdfs.DFSClient: failed to
> > connect to DomainSocket(fd=1463,path=/var/run/hdfs-sockets/dn)
> > java.net.SocketTimeoutException: read(2) error: Resource temporarily
> > unavailable
> > at org.apache.hadoop.net.unix.DomainSocket.readArray0(Native Method)
> > at
> org.apache.hadoop.net.unix.DomainSocket.access$200(DomainSocket.java:47)
> > at
> >
> >
> org.apache.hadoop.net.unix.DomainSocket$DomainInputStream.read(DomainSocket.java:530)
> > at java.io.FilterInputStream.read(FilterInputStream.java:66)
> > at
> >
> >
> org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed(HdfsProtoUtil.java:169)
> > at
> >
> >
> org.apache.hadoop.hdfs.BlockReaderFactory.newShortCircuitBlockReader(BlockReaderFactory.java:187)
> > at
> >
> >
> org.apache.hadoop.hdfs.BlockReaderFactory.newBlockReader(BlockReaderFactory.java:104)
> > at
> >
> >
> org.apache.hadoop.hdfs.DFSInputStream.getBlockReader(DFSInputStream.java:1060)
> > at
> >
> org.apache.hadoop.hdfs.DFSInputStream.blockSeekTo(DFSInputStream.java:538)
> > at
> >
> >
> org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:750)
> > at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:794)
> > at java.io.DataInputStream.read(DataInputStream.java:132)
> > at
> >
> >
> org.apache.hadoop.hbase.io.hfile.HFileBlock.readWithExtra(HFileBlock.java:584)
> > at
> >
> >
> org.apache.hadoop.hbase.io.hfile.HFileBlock$AbstractFSReader.readAtOffset(HFileBlock.java:1414)
> > at
> >
> >
> org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderV2.readBlockDataInternal(HFileBlock.java:1839)
> > at
> >
> >
> org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderV2.readBlockData(HFileBlock.java:1703)
> > at
> >
> >
> org.apache.hadoop.hbase.io.hfile.HFileReaderV2.readBlock(HFileReaderV2.java:338)
> > at
> >
> >
> org.apache.hadoop.hbase.io.hfile.HFileReaderV2$AbstractScannerV2.readNextDataBlock(HFileReaderV2.java:593)
> > at
> >
> >
> org.apache.hadoop.hbase.io.hfile.HFileReaderV2$ScannerV2.next(HFileReaderV2.java:691)
> > at
> >
> >
> org.apache.hadoop.hbase.regionserver.StoreFileScanner.next(StoreFileScanner.java:130)
> > at
> >
> >
> org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:95)
> > at
> >
> >
> org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:383)
> > at
> >
> >
> org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:311)
> > at
> >
> org.apache.hadoop.hbase.regionserver.Compactor.compact(Compactor.java:184)
> > at org.apache.hadoop.hbase.regionserver.Store.compact(Store.java:1081)
> > at
> org.apache.hadoop.hbase.regionserver.HRegion.compact(HRegion.java:1335)
> > at
> >
> >
> org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest.run(CompactionRequest.java:303)
> > at
> >
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> > at
> >
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> > at java.lang.Thread.run(Thread.java:662)
> >
> > 4. Then tons of this (region key and table changed for privacy reasons)
> >
> > 2014-02-16 22:04:32,681 INFO
> org.apache.hadoop.hbase.regionserver.HRegion:
> > Blocking updates for 'IPC Server handler 942 on 60020' on region
> > ABC,\x0A\A,1392584906041.3ee371c1d20deaa9fc945b77d8c75f9a.: memstore size
> > 512.0 M is >= than blocking 512 M size
> >
> >
> This is because handlers got backed up unable to write out their load to
> hdfs?
>
>
>
> > 5. Couple of those:
> >
> > 2014-02-16 22:04:47,687 WARN org.apache.hadoop.hdfs.DFSClient: Failed to
> > connect to /10.223.226.91:50010 for block, add to deadNodes and
> continue.
> > java.net.SocketTimeoutException: 60000 millis timeout while waiting for
> > channel to be ready for read. ch :
> > java.nio.channels.SocketChannel[connected local=/10.223.226.91:41109
> > remote=/
> > 10.223.226.91:50010]
> > java.net.SocketTimeoutException: 60000 millis timeout while waiting for
> > channel to be ready for read. ch :
> > java.nio.channels.SocketChannel[connected local=/10.223.226.91:41109
> > remote=/
> > 10.223.226.91:50010]
> > at
> >
> >
> org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:165)
> > at
> org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:156)
> > at
> org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:129)
> > at
> org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:117)
> > at java.io.FilterInputStream.read(FilterInputStream.java:66)
> > at
> >
> >
> org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed(HdfsProtoUtil.java:169)
> > at
> >
> >
> org.apache.hadoop.hdfs.RemoteBlockReader2.newBlockReader(RemoteBlockReader2.java:392)
> > at
> >
> >
> org.apache.hadoop.hdfs.BlockReaderFactory.newBlockReader(BlockReaderFactory.java:137)
> > at
> >
> >
> org.apache.hadoop.hdfs.DFSInputStream.getBlockReader(DFSInputStream.java:1103)
> > at
> >
> org.apache.hadoop.hdfs.DFSInputStream.blockSeekTo(DFSInputStream.java:538)
> > at
> >
> >
> org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:750)
> > at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:794)
> > at java.io.DataInputStream.read(DataInputStream.java:132)
> > at
> >
> >
> org.apache.hadoop.hbase.io.hfile.HFileBlock.readWithExtra(HFileBlock.java:584)
> > at
> >
> >
> org.apache.hadoop.hbase.io.hfile.HFileBlock$AbstractFSReader.readAtOffset(HFileBlock.java:1414)
> > at
> >
> >
> org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderV2.readBlockDataInternal(HFileBlock.java:1839)
> > at
> >
> >
> org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderV2.readBlockData(HFileBlock.java:1703)
> > at
> >
> >
> org.apache.hadoop.hbase.io.hfile.HFileReaderV2.readBlock(HFileReaderV2.java:338)
> > at
> >
> >
> org.apache.hadoop.hbase.io.hfile.HFileReaderV2$AbstractScannerV2.readNextDataBlock(HFileReaderV2.java:593)
> > at
> >
> >
> org.apache.hadoop.hbase.io.hfile.HFileReaderV2$ScannerV2.next(HFileReaderV2.java:691)
> > at
> >
> >
> org.apache.hadoop.hbase.regionserver.StoreFileScanner.next(StoreFileScanner.java:130)
> > at
> >
> >
> org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:95)
> > at
> >
> >
> org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:383)
> > at
> >
> >
> org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:311)
> > at
> >
> org.apache.hadoop.hbase.regionserver.Compactor.compact(Compactor.java:184)
> > at org.apache.hadoop.hbase.regionserver.Store.compact(Store.java:1081)
> > at
> org.apache.hadoop.hbase.regionserver.HRegion.compact(HRegion.java:1335)
> > at
> >
> >
> org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest.run(CompactionRequest.java:303)
> > at
> >
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> > at
> >
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> > at java.lang.Thread.run(Thread.java:662)
> >
> >
> 60 seconds is a long time to wait on data.  Tune it down?
>
>
>
> > 6. In the mean time we successfull flushes.
> >
> > 7. And then, for-ever getting this exception until RS was killed.
> >
> > 2014-02-16 22:06:41,502 WARN
> org.apache.hadoop.hbase.regionserver.HRegion:
> > Failed getting lock in batch put,
> >
> >
> row=\x01+N\xD3\x1A\xD2\xBD\x1F\x03\xBE\xD4\x07l\xE1j\x92\x05\x00\x00\x01D<\x89\xB5\x848GxT\xD3]J\xEF
> > java.io.IOException: Timed out on getting lock for
> >
> >
> row=\x01+N\xD3\x1A\xD2\xBD\x1F\x03\xBE\xD4\x07l\xE1j\x92\x05\x00\x00\x01D<\x89\xB5\x848GxT\xD3]J\xEF
> > at
> >
> >
> org.apache.hadoop.hbase.regionserver.HRegion.internalObtainRowLock(HRegion.java:3391)
> > at
> org.apache.hadoop.hbase.regionserver.HRegion.getLock(HRegion.java:3468)
> > at
> >
> >
> org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchMutation(HRegion.java:2239)
> > at
> >
> org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:2110)
> > at
> >
> >
> org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java:3661)
> > at sun.reflect.GeneratedMethodAccessor34.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.WritableRpcEngine$Server.call(WritableRpcEngine.java:320)
> > at
> >
> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1426)
> >
> >
> >
> >
>
> A thread had the row lock and was stuck on HDFS?  Any other thread that
> came in would timeout trying to get to the row?
>
>
> St.Ack
>
>
>
> >
> >
> > On Mon, Feb 10, 2014 at 10:25 AM, Asaf Mesika <as...@gmail.com>
> > wrote:
> >
> > > Hi,
> > >
> > > We have HBase 0.94.7 deployed in production with 54 Region Servers
> > (Hadoop
> > > 1).
> > > Couple of days ago, we had an incident which made our system unusable
> for
> > > several hours.
> > > HBase started emitting WARN exceptions indefinitely, thus failing any
> > > writes to it. Until stopped this RS, the issue wasn't resolved.
> > >
> > >
> > > 2014-02-07 02:10:14,415 WARN
> > org.apache.hadoop.hbase.regionserver.HRegion:
> > > Failed getting lock in batch put,
> > >
> >
> row=E\x09F\xD4\xD4\xE8\xF4\x8E\x10\x18UD\x0E\xE7\x11\x1B\x05\x00\x00\x01D\x0A\x18i\xA5\x11\x8C\xEC7\x87a`\x00
> > > java.io.IOException: Timed out on getting lock for
> > >
> >
> row=E\x09F\xD4\xD4\xE8\xF4\x8E\x10\x18UD\x0E\xE7\x11\x1B\x05\x00\x00\x01D\x0A\x18i\xA5\x11\x8C\xEC7\x87a`\x00
> > >         at
> > >
> >
> org.apache.hadoop.hbase.regionserver.HRegion.internalObtainRowLock(HRegion.java:3441)
> > >         at
> > > org.apache.hadoop.hbase.regionserver.HRegion.getLock(HRegion.java:3518)
> > >         at
> > >
> >
> org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchMutation(HRegion.java:2282)
> > >         at
> > >
> >
> org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:2153)
> > >         at
> > >
> >
> org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java:3755)
> > >         at sun.reflect.GeneratedMethodAccessor168.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.WritableRpcEngine$Server.call(WritableRpcEngine.java:320)
> > >         at
> > >
> >
> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1426)
> > >
> > >
> > > *So what happened?*
> > >
> > >    - At 01:52, from some reason, the local DataNode stopped responding
> to
> > >    the RS, although its logs seems alive until 02:10 and then nothing
> > until
> > >    shut down manually.
> > >    - HBase gets time outs for writing to HDFS, understands there is a
> > >    problem with the local data node and excludes it.
> > >    - HDFS write for 1-2 minutes (Flush throughput) drops to 0.9 mb/sec,
> > >    and then is regained back to 56 mb/sec. all write are done to a
> > remote data
> > >    node.
> > >    - And then suddenly the exception which is quoted above.
> > >
> > > Any idea what's this issue about?
> > >
> > >
> >
>