[ https://issues.apache.org/jira/browse/HBASE-11277?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14014315#comment-14014315 ]
Andrew Purtell commented on HBASE-11277: ---------------------------------------- I'm running YCSB's workload A on a well endowed EC2 testbed with 10GE networking against a table preloaded with 100M entries, running 5 instances of the YCSB client with 10 threads on 5 separate client servers. The clients are attempting 10M ops at a target rate of 100,000 ops/second in aggregate. (I also tried an aggregate target of 250,000 ops/sec but this might have been triggering EC2 / kernel / networking issues.) I added a tiny bit of code to RpcServer.Reader#readAndProcess that throws an IOE if we go 100,000 times around the readAndProcess loop and it was triggered maybe 2-3 times on every server during a run. I then changed the limit to 10,000 with better results but many more exceptions observed, maybe ~20 per server per test run. They look like: {noformat} 2014-05-30 20:58:40,637 WARN [RpcServer.reader=3,port=8120] ipc.RpcServer: RpcServer.listener,port=8120: count of bytes read: 0 java.io.IOException: readAndProcess waited too long for message data: client=10.220.3.99 authMethod=SIMPLE connectionPreambleRead=true connectionHeaderRead=true data.remaining=624118 data.limit=644612 at org.apache.hadoop.hbase.ipc.RpcServer$Connection.readAndProcess(RpcServer.java:1411) at org.apache.hadoop.hbase.ipc.RpcServer$Listener.doRead(RpcServer.java:790) at org.apache.hadoop.hbase.ipc.RpcServer$Listener$Reader.doRunLoop(RpcServer.java:581) at org.apache.hadoop.hbase.ipc.RpcServer$Listener$Reader.run(RpcServer.java:556) 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) {noformat} In truth readAndProcess isn't waiting very long. Readers are pulled from a pool to handle channels that become readable. In the message processing connection phase, they read 4 bytes of length and then want to read exactly those many additional bytes in a blocking manner. During this process if data is not temporarily available on the channel we will spin hard on channelRead. The IOE I added short circuits this by closing the connection and forcing a retry. The spinning is bad. We might want to yield the thread or introduce a small sleep. Better yet would be a fully async server side. Dropping the connection almost seems like a good idea. Without doing that one of the client (multiplexed) connections stops making progress for periods of 20-40 seconds. If we do drop the connection, the test sustains an average 100,000 ops/sec. > RPCServer threads can wedge under high load > ------------------------------------------- > > Key: HBASE-11277 > URL: https://issues.apache.org/jira/browse/HBASE-11277 > Project: HBase > Issue Type: Bug > Affects Versions: 0.96.2, 0.98.3 > Reporter: Andrew Purtell > Priority: Critical > > This is with 0.98.0 in an insecure setup with 7u55 and 7u60. Under high load, > RPCServer threads can wedge, fail to make progess, and consume 100% CPU time > on a core indefinitely. > Dumping threads, all threads are in BLOCKED or IN_NATIVE state. The IN_NATIVE > threads are mostly in EPollArrayWrapper.epollWait or > FileDispatcherImpl.read0. The number of threads found in > FileDispatcherImpl.read0 correspond to the number of runaway threads expected > based on looking at 'top' output. These look like: > {noformat} > Thread 64758: (state = IN_NATIVE) > - sun.nio.ch.FileDispatcherImpl.read0(java.io.FileDescriptor, long, int) > @bci=0 (Compiled frame; information may be imprecise) > - sun.nio.ch.SocketDispatcher.read(java.io.FileDescriptor, long, int) > @bci=4, line=39 (Compiled frame) > - sun.nio.ch.IOUtil.readIntoNativeBuffer(java.io.FileDescriptor, > java.nio.ByteBuffer, long, sun.nio.ch.NativeDispatcher) @bci=114, line=223 > (Compil > ed frame) > - sun.nio.ch.IOUtil.read(java.io.FileDescriptor, java.nio.ByteBuffer, long, > sun.nio.ch.NativeDispatcher) @bci=48, line=197 (Compiled frame) > - sun.nio.ch.SocketChannelImpl.read(java.nio.ByteBuffer) @bci=234, line=379 > (Compiled frame) > - > org.apache.hadoop.hbase.ipc.RpcServer.channelRead(java.nio.channels.ReadableByteChannel, > java.nio.ByteBuffer) @bci=12, line=2224 (Compiled frame) > - org.apache.hadoop.hbase.ipc.RpcServer$Connection.readAndProcess() > @bci=509, line=1488 (Compiled frame) > - > org.apache.hadoop.hbase.ipc.RpcServer$Listener.doRead(java.nio.channels.SelectionKey) > @bci=23, line=790 (Compiled frame) > - org.apache.hadoop.hbase.ipc.RpcServer$Listener$Reader.doRunLoop() @bci=97, > line=581 (Compiled frame) > - org.apache.hadoop.hbase.ipc.RpcServer$Listener$Reader.run() @bci=1, > line=556 (Interpreted frame) > - > java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker) > @bci=95, line=1145 (Interpreted frame) > - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=615 > (Interpreted frame) > - java.lang.Thread.run() @bci=11, line=745 (Interpreted frame) > {noformat} -- This message was sent by Atlassian JIRA (v6.2#6252)