Return-Path: Delivered-To: apmail-hadoop-hbase-dev-archive@minotaur.apache.org Received: (qmail 94065 invoked from network); 29 Sep 2009 01:58:38 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 29 Sep 2009 01:58:38 -0000 Received: (qmail 32652 invoked by uid 500); 29 Sep 2009 01:58:38 -0000 Delivered-To: apmail-hadoop-hbase-dev-archive@hadoop.apache.org Received: (qmail 32620 invoked by uid 500); 29 Sep 2009 01:58:38 -0000 Mailing-List: contact hbase-dev-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hbase-dev@hadoop.apache.org Delivered-To: mailing list hbase-dev@hadoop.apache.org Received: (qmail 32459 invoked by uid 99); 29 Sep 2009 01:58:38 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 29 Sep 2009 01:58:38 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.140] (HELO brutus.apache.org) (140.211.11.140) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 29 Sep 2009 01:58:36 +0000 Received: from brutus (localhost [127.0.0.1]) by brutus.apache.org (Postfix) with ESMTP id 086D0234C045 for ; Mon, 28 Sep 2009 18:58:16 -0700 (PDT) Message-ID: <1358860588.1254189496033.JavaMail.jira@brutus> Date: Mon, 28 Sep 2009 18:58:16 -0700 (PDT) From: "Eric Tschetter (JIRA)" To: hbase-dev@hadoop.apache.org Subject: [jira] Updated: (HBASE-1873) Race condition around HRegionServer -> HMaster communication In-Reply-To: <1615448772.1254189495970.JavaMail.jira@brutus> MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 X-Virus-Checked: Checked by ClamAV on apache.org [ https://issues.apache.org/jira/browse/HBASE-1873?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eric Tschetter updated HBASE-1873: ---------------------------------- Description: HRegionServer on lines 459 - 463 (part of run()) accesses outboundMsgs in a synchronized fashion, but other uses of the object are not synchronized. Specifically, the code is 459 synchronized(this.outboundMsgs) { 460 outboundArray = 461 this.outboundMsgs.toArray(new HMsg[outboundMsgs.size()]); 462 this.outboundMsgs.clear(); 463 } Whereas things are added to this list from calls like private void reportOpen(HRegionInfo region) { outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_OPEN, region)); } And void reportSplit(HRegionInfo oldRegion, HRegionInfo newRegionA, HRegionInfo newRegionB) { outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_SPLIT, oldRegion, ("Daughters; " + newRegionA.getRegionNameAsString() + ", " + newRegionB.getRegionNameAsString()).getBytes())); outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_OPEN, newRegionA)); outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_OPEN, newRegionB)); } It looks like the object is initialized as private final List outboundMsgs = Collections.synchronizedList(new ArrayList()); Which would appear to provide security, but it doesn't actually prevent an insert from happening between lines 461 and 462, which would subsequently get removed from the call to clear(). At least, from the Sun HotSpot source code, it looks like Collections.synchronizedList() does the right thing and synchronizes on an inner mutex object instead of synchronizing on the externally visible list object itself. That means, however, that the synchronized() on line 459 is largely meaningless. I'm not sure how often this race condition would occur in the wild, but every thread waiting on the mutex around the toArray() call increases the probability that the next person to get the mutex is someone who wants to add something to the List, rather than the thread calling clear(). Simple fix would be to do external synchronization around all accesses to the List. Barring that, perhaps a SynchronizedList implementation with a "emptyToArray()" method that encapsulates the toArray() and subsequent clear(). was: HRegionServer on lines 459 - 463 (part of run()) accesses outboundMsgs in a synchronized fashion, but other uses of the object are not synchronized. Specifically, the code is 459 synchronized(this.outboundMsgs) { 460 outboundArray = 461 this.outboundMsgs.toArray(new HMsg[outboundMsgs.size()]); 462 this.outboundMsgs.clear(); 463 } Whereas things are added to this list from calls like private void reportOpen(HRegionInfo region) { outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_OPEN, region)); } And void reportSplit(HRegionInfo oldRegion, HRegionInfo newRegionA, HRegionInfo newRegionB) { outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_SPLIT, oldRegion, ("Daughters; " + newRegionA.getRegionNameAsString() + ", " + newRegionB.getRegionNameAsString()).getBytes())); outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_OPEN, newRegionA)); outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_OPEN, newRegionB)); } It looks like the object is initialized as private final List outboundMsgs = Collections.synchronizedList(new ArrayList()); Which would appear to provide security, but it doesn't actually prevent an insert from happening between lines 461 and 462, which would subsequently get removed from the call to clear(). At least, from the Sun HotSpot source code, it looks like Collections.synchronizedList() does the right thing and synchronizes on an inner mutex object instead of synchronizing on the externally visible list object itself. That means, however, that the synchronized() on line 459 is largely meaningless. I'm not sure how often this race condition would occur in the wild, but every thread waiting on the mutex around the toArray() call increases the probability that the next person to get the mutex is someone who wants to add something to the List, rather than the thread calling clear(). Simple fix would be to do external synchronization around all accesses to the List. Barring that, perhaps a SynchronizedList implementation with a "emptyToArray()" method that encapsulates the toArray() and subsequent clear(). > Race condition around HRegionServer -> HMaster communication > ------------------------------------------------------------ > > Key: HBASE-1873 > URL: https://issues.apache.org/jira/browse/HBASE-1873 > Project: Hadoop HBase > Issue Type: Bug > Affects Versions: 0.20.0 > Reporter: Eric Tschetter > > HRegionServer on lines 459 - 463 (part of run()) accesses outboundMsgs in a synchronized fashion, but other uses of the object are not synchronized. > Specifically, the code is > 459 synchronized(this.outboundMsgs) { > 460 outboundArray = > 461 this.outboundMsgs.toArray(new HMsg[outboundMsgs.size()]); > 462 this.outboundMsgs.clear(); > 463 } > Whereas things are added to this list from calls like > private void reportOpen(HRegionInfo region) { > outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_OPEN, region)); > } > And > void reportSplit(HRegionInfo oldRegion, HRegionInfo newRegionA, > HRegionInfo newRegionB) { > outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_SPLIT, oldRegion, > ("Daughters; " + > newRegionA.getRegionNameAsString() + ", " + > newRegionB.getRegionNameAsString()).getBytes())); > outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_OPEN, newRegionA)); > outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_OPEN, newRegionB)); > } > It looks like the object is initialized as > private final List outboundMsgs = > Collections.synchronizedList(new ArrayList()); > Which would appear to provide security, but it doesn't actually prevent an insert from happening between lines 461 and 462, which would subsequently get removed from the call to clear(). At least, from the Sun HotSpot source code, it looks like Collections.synchronizedList() does the right thing and synchronizes on an inner mutex object instead of synchronizing on the externally visible list object itself. That means, however, that the synchronized() on line 459 is largely meaningless. > I'm not sure how often this race condition would occur in the wild, but every thread waiting on the mutex around the toArray() call increases the probability that the next person to get the mutex is someone who wants to add something to the List, rather than the thread calling clear(). > Simple fix would be to do external synchronization around all accesses to the List. Barring that, perhaps a SynchronizedList implementation with a "emptyToArray()" method that encapsulates the toArray() and subsequent clear(). -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.