Return-Path: X-Original-To: apmail-hbase-issues-archive@www.apache.org Delivered-To: apmail-hbase-issues-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 30AF71194A for ; Mon, 2 Jun 2014 17:16:05 +0000 (UTC) Received: (qmail 79622 invoked by uid 500); 2 Jun 2014 17:16:04 -0000 Delivered-To: apmail-hbase-issues-archive@hbase.apache.org Received: (qmail 79574 invoked by uid 500); 2 Jun 2014 17:16:04 -0000 Mailing-List: contact issues-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list issues@hbase.apache.org Received: (qmail 79563 invoked by uid 99); 2 Jun 2014 17:16:04 -0000 Received: from arcas.apache.org (HELO arcas.apache.org) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 02 Jun 2014 17:16:04 +0000 Date: Mon, 2 Jun 2014 17:16:04 +0000 (UTC) From: "Andrew Purtell (JIRA)" To: issues@hbase.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (HBASE-11277) RPCServer threads can wedge under high load MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/HBASE-11277?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14015586#comment-14015586 ] Andrew Purtell commented on HBASE-11277: ---------------------------------------- bq. This said, I wonder if data/connectionPreambleRead should not be volatile? What are our guarantees here? As I read the code only one thread is handling a connection at a time. Readers are assigned channel selectors in round robin fashion by an acceptor thread servicing the listener socket (that thread is the 'Listener'). There is only one of these running per RpcServer. Readers service their selectable channels one at a time. It looks to me like readAndProcess is called serially for each connection. > 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 > Assignee: Andrew Purtell > Priority: Critical > Fix For: 0.99.0, 0.96.3, 0.98.3 > > Attachments: HBASE-11277.patch > > > 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)