[
https://issues.apache.org/jira/browse/HBASE-13238?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14362130#comment-14362130
]
Andrew Purtell edited comment on HBASE-13238 at 3/15/15 1:11 AM:
-----------------------------------------------------------------
bq. Do you still have the two stack traces Andrew Purtell?
Here's the stacktrace of the hung thread as posted on an internal issue tracker
but unfortunately I didn't save the complete stack dumps before I had to force
reboot my laptop the other day:
{noformat}
Thread 102 (B.DefaultRpcServer.handler=19,queue=4,port=60020):
sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:87)
sun.nio.ch.SelectorImpl.select(SelectorImpl.java:98)
org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:335)
org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:157)
org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:161)
org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.readChannelFully(PacketReceiver.java:258)
org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.doReadFully(PacketReceiver.java:209)
org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.doRead(PacketReceiver.java:171)
org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.receiveNextPacket(PacketReceiver.java:102)
org.apache.hadoop.hdfs.RemoteBlockReader2.readNextPacket(RemoteBlockReader2.java:173)
org.apache.hadoop.hdfs.RemoteBlockReader2.read(RemoteBlockReader2.java:138)
org.apache.hadoop.hdfs.RemoteBlockReader2.skip(RemoteBlockReader2.java:240)
org.apache.hadoop.hdfs.DFSInputStream.seek(DFSInputStream.java:1399)
org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:63)
org.apache.hadoop.hbase.io.hfile.HFileBlock$AbstractFSReader.readAtOffset(HFileBlock.java:1302)
org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderV2.readBlockDataInternal(HFileBlock.java:1531)
org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderV2.readBlockData(HFileBlock.java:1413)
{noformat}
A user wrote on on user@hbase with a closely related problem I think. I've seen
us get wedged in HDFS when closing regions through a variety of code paths. Let
me see if I can dig up another stack trace from one of the SplitTransaction
investigations.
See
http://mail-archives.apache.org/mod_mbox/hbase-user/201503.mbox/%3CCADSBNAh5v9sxMBHPsqKt%3DyoP7Q-hjpHkbLaWL5_rBTot9wBg9w%40mail.gmail.com%3E
{quote}
I think I found the thread that is stuck. Is restarting the server harmless in
this state?
"RS_CLOSE_REGION-hdfs-ix03.se-ix.delta.prod,60020,1424687995350-1" prio=10
tid=0x00007f75a0008000 nid=0x23ee in Object.wait() [0x00007f757d30b000]
java.lang.Thread.State: WAITING (on object monitor)
at java.lang.Object.wait(Native Method)
at java.lang.Object.wait(Object.java:503)
at
org.apache.hadoop.hdfs.DFSOutputStream.waitAndQueueCurrentPacket(DFSOutputStream.java:1411)
- locked <0x00000007544573e8> (a java.util.LinkedList)
at org.apache.hadoop.hdfs.DFSOutputStream.writeChunk(DFSOutputStream.java:1479)
- locked <0x0000000756780218> (a org.apache.hadoop.hdfs.DFSOutputStream)
at
org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunk(FSOutputSummer.java:173)
at org.apache.hadoop.fs.FSOutputSummer.write1(FSOutputSummer.java:116)
at org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:102)
- locked <0x0000000756780218> (a org.apache.hadoop.hdfs.DFSOutputStream)
at
org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:54)
at java.io.DataOutputStream.write(DataOutputStream.java:107)
- locked <0x00000007543ef268> (a
org.apache.hadoop.hdfs.client.HdfsDataOutputStream)
at java.io.FilterOutputStream.write(FilterOutputStream.java:97)
at
org.apache.hadoop.hbase.io.hfile.HFileBlock$Writer.writeHeaderAndData(HFileBlock.java:1061)
at
org.apache.hadoop.hbase.io.hfile.HFileBlock$Writer.writeHeaderAndData(HFileBlock.java:1047)
at
org.apache.hadoop.hbase.io.hfile.HFileBlockIndex$BlockIndexWriter.writeIntermediateBlock(HFileBlockIndex.java:952)
at
org.apache.hadoop.hbase.io.hfile.HFileBlockIndex$BlockIndexWriter.writeIntermediateLevel(HFileBlockIndex.java:935)
at
org.apache.hadoop.hbase.io.hfile.HFileBlockIndex$BlockIndexWriter.writeIndexBlocks(HFileBlockIndex.java:844)
at org.apache.hadoop.hbase.io.hfile.HFileWriterV2.close(HFileWriterV2.java:403)
at
org.apache.hadoop.hbase.regionserver.StoreFile$Writer.close(StoreFile.java:1272)
at org.apache.hadoop.hbase.regionserver.Store.internalFlushCache(Store.java:835)
- locked <0x000000075d8b2110> (a java.lang.Object)
at org.apache.hadoop.hbase.regionserver.Store.flushCache(Store.java:746)
at
org.apache.hadoop.hbase.regionserver.Store$StoreFlusherImpl.flushCache(Store.java:2348)
at
org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:1580)
at
org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:1479)
at org.apache.hadoop.hbase.regionserver.HRegion.doClose(HRegion.java:992)
at org.apache.hadoop.hbase.regionserver.HRegion.close(HRegion.java:956)
- locked <0x000000075d97b628> (a java.lang.Object)
at
org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler.process(CloseRegionHandler.java:119)
at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:175)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
{quote}
was (Author: apurtell):
bq. Do you still have the two stack traces Andrew Purtell?
Here's the stacktrace of the hung thread as posted on an internal issue tracker
but unfortunately I didn't save the complete stack dumps before I had to force
reboot my laptop the other day:
{noformat}
Thread 102 (B.DefaultRpcServer.handler=19,queue=4,port=60020):
sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:87)
sun.nio.ch.SelectorImpl.select(SelectorImpl.java:98)
org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:335)
org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:157)
org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:161)
org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.readChannelFully(PacketReceiver.java:258)
org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.doReadFully(PacketReceiver.java:209)
org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.doRead(PacketReceiver.java:171)
org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.receiveNextPacket(PacketReceiver.java:102)
org.apache.hadoop.hdfs.RemoteBlockReader2.readNextPacket(RemoteBlockReader2.java:173)
org.apache.hadoop.hdfs.RemoteBlockReader2.read(RemoteBlockReader2.java:138)
org.apache.hadoop.hdfs.RemoteBlockReader2.skip(RemoteBlockReader2.java:240)
org.apache.hadoop.hdfs.DFSInputStream.seek(DFSInputStream.java:1399)
org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:63)
org.apache.hadoop.hbase.io.hfile.HFileBlock$AbstractFSReader.readAtOffset(HFileBlock.java:1302)
org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderV2.readBlockDataInternal(HFileBlock.java:1531)
org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderV2.readBlockData(HFileBlock.java:1413)
{noformat}
A user wrote on on user@hbase with a closely related problem I think, see
http://mail-archives.apache.org/mod_mbox/hbase-user/201503.mbox/%3CCADSBNAh5v9sxMBHPsqKt%3DyoP7Q-hjpHkbLaWL5_rBTot9wBg9w%40mail.gmail.com%3E
{quote}
I think I found the thread that is stuck. Is restarting the server harmless in
this state?
"RS_CLOSE_REGION-hdfs-ix03.se-ix.delta.prod,60020,1424687995350-1" prio=10
tid=0x00007f75a0008000 nid=0x23ee in Object.wait() [0x00007f757d30b000]
java.lang.Thread.State: WAITING (on object monitor)
at java.lang.Object.wait(Native Method)
at java.lang.Object.wait(Object.java:503)
at
org.apache.hadoop.hdfs.DFSOutputStream.waitAndQueueCurrentPacket(DFSOutputStream.java:1411)
- locked <0x00000007544573e8> (a java.util.LinkedList)
at org.apache.hadoop.hdfs.DFSOutputStream.writeChunk(DFSOutputStream.java:1479)
- locked <0x0000000756780218> (a org.apache.hadoop.hdfs.DFSOutputStream)
at
org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunk(FSOutputSummer.java:173)
at org.apache.hadoop.fs.FSOutputSummer.write1(FSOutputSummer.java:116)
at org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:102)
- locked <0x0000000756780218> (a org.apache.hadoop.hdfs.DFSOutputStream)
at
org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:54)
at java.io.DataOutputStream.write(DataOutputStream.java:107)
- locked <0x00000007543ef268> (a
org.apache.hadoop.hdfs.client.HdfsDataOutputStream)
at java.io.FilterOutputStream.write(FilterOutputStream.java:97)
at
org.apache.hadoop.hbase.io.hfile.HFileBlock$Writer.writeHeaderAndData(HFileBlock.java:1061)
at
org.apache.hadoop.hbase.io.hfile.HFileBlock$Writer.writeHeaderAndData(HFileBlock.java:1047)
at
org.apache.hadoop.hbase.io.hfile.HFileBlockIndex$BlockIndexWriter.writeIntermediateBlock(HFileBlockIndex.java:952)
at
org.apache.hadoop.hbase.io.hfile.HFileBlockIndex$BlockIndexWriter.writeIntermediateLevel(HFileBlockIndex.java:935)
at
org.apache.hadoop.hbase.io.hfile.HFileBlockIndex$BlockIndexWriter.writeIndexBlocks(HFileBlockIndex.java:844)
at org.apache.hadoop.hbase.io.hfile.HFileWriterV2.close(HFileWriterV2.java:403)
at
org.apache.hadoop.hbase.regionserver.StoreFile$Writer.close(StoreFile.java:1272)
at org.apache.hadoop.hbase.regionserver.Store.internalFlushCache(Store.java:835)
- locked <0x000000075d8b2110> (a java.lang.Object)
at org.apache.hadoop.hbase.regionserver.Store.flushCache(Store.java:746)
at
org.apache.hadoop.hbase.regionserver.Store$StoreFlusherImpl.flushCache(Store.java:2348)
at
org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:1580)
at
org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:1479)
at org.apache.hadoop.hbase.regionserver.HRegion.doClose(HRegion.java:992)
at org.apache.hadoop.hbase.regionserver.HRegion.close(HRegion.java:956)
- locked <0x000000075d97b628> (a java.lang.Object)
at
org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler.process(CloseRegionHandler.java:119)
at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:175)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
{quote}
> Time out locks and abort if HDFS is wedged
> ------------------------------------------
>
> Key: HBASE-13238
> URL: https://issues.apache.org/jira/browse/HBASE-13238
> Project: HBase
> Issue Type: Brainstorming
> Reporter: Andrew Purtell
>
> This is a brainstorming issue on the topic of timing out locks and aborting
> rather than waiting infinitely. Perhaps even as a rule.
> We had a minor production incident where a region was unable to close after
> trying for 24 hours. The CloseRegionHandler was waiting for a write lock on
> the ReentrantReadWriteLock we take in HRegion#doClose. There were outstanding
> read locks. Three other threads were stuck in scanning, all blocked on the
> same DFSInputStream. Two were blocked in DFSInputStream#getFileLength, the
> third was waiting in epoll from SocketIOWithTimeout$SelectorPool#select with
> apparent infinite timeout from PacketReceiver#readChannelFully.
> This is similar to other issues we have seen before, in the context of the
> region wanting to finish a compaction before closing for a split, but can't
> due to some HDFS issue causing the reader to become extremely slow if not
> wedged. This has lead to what should be quick SplitTransactions causing
> availability problems of many minutes in length.
> The Hadoop version was 2.3 (specifically 2.3 CDH 5.0.1), and we are planning
> to upgrade, but [~lhofhansl] and I were discussing the issue in general and
> wonder if we should not be timing out locks such as the
> ReentrantReadWriteLock, and if so, abort the regionserver. In this case this
> would have caused recovery and reassignment of the region in question and we
> would not have had a prolonged availability problem.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)