Return-Path: X-Original-To: apmail-cassandra-commits-archive@www.apache.org Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 67EB579B6 for ; Thu, 1 Dec 2011 11:43:01 +0000 (UTC) Received: (qmail 71773 invoked by uid 500); 1 Dec 2011 11:43:01 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 71697 invoked by uid 500); 1 Dec 2011 11:43:01 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 71689 invoked by uid 99); 1 Dec 2011 11:43:01 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 01 Dec 2011 11:43:01 +0000 X-ASF-Spam-Status: No, hits=-2001.2 required=5.0 tests=ALL_TRUSTED,RP_MATCHES_RCVD X-Spam-Check-By: apache.org Received: from [140.211.11.116] (HELO hel.zones.apache.org) (140.211.11.116) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 01 Dec 2011 11:43:00 +0000 Received: from hel.zones.apache.org (hel.zones.apache.org [140.211.11.116]) by hel.zones.apache.org (Postfix) with ESMTP id EE3E5AC3B5 for ; Thu, 1 Dec 2011 11:42:39 +0000 (UTC) Date: Thu, 1 Dec 2011 11:42:39 +0000 (UTC) From: "Sylvain Lebresne (Updated) (JIRA)" To: commits@cassandra.apache.org Message-ID: <1431488171.30574.1322739759977.JavaMail.tomcat@hel.zones.apache.org> In-Reply-To: <1234445441.30414.1322736220030.JavaMail.tomcat@hel.zones.apache.org> Subject: [jira] [Updated] (CASSANDRA-3546) Hinted handoffs isn't delivered if/when HintedHandOffManager ends up in invalid state. MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: quoted-printable X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/CASSANDRA-3546?page=3Dcom.atla= ssian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne updated CASSANDRA-3546: ---------------------------------------- Attachment: 3546.patch Patch attached, thanks Fredrik. =20 > Hinted handoffs isn't delivered if/when HintedHandOffManager ends up in i= nvalid state. > -------------------------------------------------------------------------= ------------- > > Key: CASSANDRA-3546 > URL: https://issues.apache.org/jira/browse/CASSANDRA-3546 > Project: Cassandra > Issue Type: Bug > Components: Core > Affects Versions: 1.0.3 > Reporter: Fredrik L Stigb=C3=A4ck > Attachments: 3546.patch > > > Running Cassandra 1.0.3. > I've done some testing with 2 nodes (node A, node B), replication factor = 2. > I take node A down, writing some data to node B and then take node A up. > Sometimes hints aren't delivered when node A comes up. > I've done some debugging in org.apache.cassandra.db.HintedHandOffManager = and sometimes node B ends up in a strange state in method org.apache.cassan= dra.db.HintedHandOffManager.deliverHints(final InetAddress to), where org.a= pache.cassandra.db.HintedHandOffManager.queuedDeliveries already has node A= in it's Set and therefore no hints will ever be delivered to node A. > The only reason for this that I can see is that in org.apache.cassandra.d= b.HintedHandOffManager.deliverHintsToEndpoint(InetAddress endpoint) the hin= tStore.isEmpty() check returns true and the endpoint (node A) isn't remove= d from org.apache.cassandra.db.HintedHandOffManager.queuedDeliveries. Then = no hints will ever be delivered again until node B is restarted. > During what conditions will hintStore.isEmpty() return true? > Shouldn't the hintStore.isEmpty() check be inside the try {} finally{} cl= ause, removing the endpoint from queuedDeliveries in the finally block? > {code} > public void deliverHints(final InetAddress to) > { > logger_.debug("deliverHints to {}", to); > if (!queuedDeliveries.add(to)) > return; > ....... > } > {code} > {code} > private void deliverHintsToEndpoint(InetAddress endpoint)=20 > throws IOException, DigestMismatchException, InvalidRequestException,= TimeoutException, InterruptedException > { > ColumnFamilyStore hintStore =3D Table.open(Table.SYSTEM_TABLE).getCo= lumnFamilyStore(HINTS_CF); > if (hintStore.isEmpty()) > return; // nothing to do, don't confuse users by logging a no-op= handoff > try > { > ...... > } > finally > { > queuedDeliveries.remove(endpoint); > } > } > {code}=20 -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrato= rs: https://issues.apache.org/jira/secure/ContactAdministrators!default.jsp= a For more information on JIRA, see: http://www.atlassian.com/software/jira