Hi Ted, Stack, not sure if we could add some guide for information that need to be provided when describing a problem...? many times I see Ted asked the hbase version etc...:-P maybe it could help save some time..
On Thu, Jun 12, 2014 at 4:30 PM, Qiang Tian <[email protected]> wrote: > another occurrence of this problem: > > http://mail-archives.apache.org/mod_mbox/hbase-dev/201402.mbox/%3ccahkeahxaajfyc6xnv+cbwupqakotmkj349cpquk02fgs8qc...@mail.gmail.com%3E > > > According to the java doc, the read call should returned immediately after > reading the available data in socket buffer. there is only 1 case that it > can be blocked: > If another thread has already initiated a read operation upon this > channel, however, then an invocation of this method will block until the > first operation is complete. > > from the regionserver jstack output, only 1 reader is in read, so not the > case. > > > there is a slight code path difference for the 2 stacks: > > firstly: > org.apache.hadoop.hbase.ipc.RpcServer.channelIO(RpcServer.java:2402) > org.apache.hadoop.hbase.ipc.RpcServer.channelRead(RpcServer.java:2368) > > > and later: > sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:243) > org.apache.hadoop.hbase.ipc.RpcServer.channelRead(RpcServer.java:2368) > > i.e. firslty in channelIO call, then read directly. there is a while loop > in channelIO. perhaps it could take time when there is much data to read? > > questions for WeiWei: > 1)what is the hbase version? > 2)How the problem happened? what operation you did? > > thanks. > > > On Sat, Jun 7, 2014 at 9:47 PM, Qiang Tian <[email protected]> wrote: > >> it looks not a busy system behavior: >> 1)From regionserver stacktraces, only reader 4 is in reading, all other >> readers are in waiting.. >> 2)Weiwei mentioned "ipc.RpcServer: RpcServer.listener,port=60020: count >> of bytes read: 0 java.nio.channels.ClosedByInterruptException", --the >> reader read nothing before it is interrupted >> >> >> we set the socket to be non-blocking, and the code looks fine to do the >> read: >> if (key.isValid()) { >> if (key.isReadable()) { >> doRead(key); >> } >> } >> >> googled the issue, only find cases that non-blocking socket blocks in >> read at client side(due to finishConnect not called). >> >> for our case, maybe there is something wrong with the corresponding >> client connection? >> need to dig more. >> >> >> >> >> On Sat, Jun 7, 2014 at 7:38 AM, sunweiwei <[email protected]> >> wrote: >> >>> Hi >>> >>> the regionserver jstack log is http://paste2.org/2wJc5CJx >>> the master jstack log is http://paste2.org/t4KLpbXw. Maybe the >>> regionserver is hbase:namespace server. >>> the master dump log (from the 60010 web) is http://paste2.org/vB2UVPyO >>> >>> >>> I saw the regionserver's Requests Per Second is 0 at that time , >>> other regionserver's Requests Per Second is about 10000~50000, and some >>> clients failed. >>> I'm not sure about loading at that time. regionserver's loading usually >>> is not high. >>> >>> I restart the regionserver and master then hbase back to normal. >>> >>> Thanks >>> -----邮件原件----- >>> 发件人: [email protected] [mailto:[email protected]] 代表 Stack >>> 发送时间: 2014年6月7日 0:16 >>> 收件人: Hbase-User >>> 主题: Re: regionserver 60020 timeout >>> >>> On Thu, Jun 5, 2014 at 11:35 PM, sunweiwei <[email protected]> >>> wrote: >>> >>> > Hi >>> > >>> > I find some 60000 millis timeout in master log, when master connect >>> > regionserver's 60020 port. Log like this: >>> > >>> > >>> Could we see the full stack trace? It would help confirm Qiang's >>> prognosis. Do you have monitoring of this server setup? What did it >>> show >>> for loading, handler-count, at this time? >>> >>> Thanks, >>> St.Ack >>> >>> >>> >>> > because java.net.SocketTimeoutException: 60000 millis timeout while >>> waiting >>> > for channel to be ready for read. ch : >>> > java.nio.channels.SocketChannel[connected local=/132.228.248.61:34835 >>> > remote=dsjhd074/132.228.248.84:60020] >>> > >>> > at >>> org.apache.hadoop.hbase.ipc.RpcClient.wrapException(RpcClient.java:1475) >>> > >>> > at >>> org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:1450) >>> > >>> > at >>> > >>> > >>> org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(RpcClient.java:1650 >>> > ) >>> > >>> > at >>> > >>> > >>> org.apache.hadoop.hbase.ipc.RpcClient$BlockingRpcChannelImplementation.callB >>> > lockingMethod(RpcClient.java:1708) >>> > >>> > at >>> > >>> > >>> org.apache.hadoop.hbase.protobuf.generated.AdminProtos$AdminService$Blocking >>> > Stub.openRegion(AdminProtos.java:20595) >>> > >>> > at >>> > >>> > >>> org.apache.hadoop.hbase.master.ServerManager.sendRegionOpen(ServerManager.ja >>> > va:628) >>> > >>> > at >>> > >>> > >>> org.apache.hadoop.hbase.master.AssignmentManager.assign(AssignmentManager.ja >>> > va:1901) >>> > >>> > at >>> > >>> > >>> org.apache.hadoop.hbase.master.AssignmentManager.access$300(AssignmentManage >>> > r.java:106) >>> > >>> > at >>> > >>> > >>> org.apache.hadoop.hbase.master.AssignmentManager$2.process(AssignmentManager >>> > .java:660) >>> > >>> > at >>> > >>> org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:128) >>> > >>> > at >>> > >>> > >>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.ja >>> > va:886) >>> > >>> > at >>> > >>> > >>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:9 >>> > 08) >>> > >>> > at java.lang.Thread.run(Thread.java:662) >>> > >>> > >>> > >>> > And I find listener,port 60020 is waiting to lock <0x000000041ae290d8> >>> in >>> > regionserver's jstack log. Is that ok? Thanks >>> > >>> > "RpcServer.reader=4,port=60020" daemon prio=10 tid=0x00007fade53c0000 >>> > nid=0x22d8c runnable [0x00007fadb8b32000] >>> > >>> > java.lang.Thread.State: RUNNABLE >>> > >>> > at sun.nio.ch.FileDispatcher.read0(Native Method) >>> > >>> > at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:21) >>> > >>> > at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:198) >>> > >>> > at sun.nio.ch.IOUtil.read(IOUtil.java:171) >>> > >>> > at >>> sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:243) >>> > >>> > - locked <0x000000041b1c78c0> (a java.lang.Object) >>> > >>> > at >>> > org.apache.hadoop.hbase.ipc.RpcServer.channelIO(RpcServer.java:2402) >>> > >>> > at >>> > org.apache.hadoop.hbase.ipc.RpcServer.channelRead(RpcServer.java:2368) >>> > >>> > at >>> > >>> > >>> org.apache.hadoop.hbase.ipc.RpcServer$Connection.readAndProcess(RpcServer.ja >>> > va:1476) >>> > >>> > at >>> > >>> org.apache.hadoop.hbase.ipc.RpcServer$Listener.doRead(RpcServer.java:770) >>> > >>> > at >>> > >>> > >>> org.apache.hadoop.hbase.ipc.RpcServer$Listener$Reader.doRunLoop(RpcServer.ja >>> > va:563) >>> > >>> > - locked <0x000000041ae290d8> (a >>> > org.apache.hadoop.hbase.ipc.RpcServer$Listener$Reader) >>> > >>> > at >>> > >>> > >>> org.apache.hadoop.hbase.ipc.RpcServer$Listener$Reader.run(RpcServer.java:538 >>> > ) >>> > >>> > at >>> > >>> > >>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.ja >>> > va:886) >>> > >>> > at >>> > >>> > >>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:9 >>> > 08) >>> > >>> > at java.lang.Thread.run(Thread.java:662) >>> > >>> > >>> > >>> > "RpcServer.listener,port=60020" daemon prio=10 tid=0x00007fadd4426800 >>> > nid=0x22db3 waiting for monitor entry [0x00007fadb658a000] >>> > >>> > java.lang.Thread.State: BLOCKED (on object monitor) >>> > >>> > at >>> > >>> > >>> org.apache.hadoop.hbase.ipc.RpcServer$Listener$Reader.registerChannel(RpcSer >>> > ver.java:593) >>> > >>> > - waiting to lock <0x000000041ae290d8> (a >>> > org.apache.hadoop.hbase.ipc.RpcServer$Listener$Reader) >>> > >>> > at >>> > >>> org.apache.hadoop.hbase.ipc.RpcServer$Listener.doAccept(RpcServer.java:745) >>> > >>> > at >>> > org.apache.hadoop.hbase.ipc.RpcServer$Listener.run(RpcServer.java:668) >>> > >>> > >>> >>> >> >
