Return-Path: X-Original-To: apmail-cassandra-user-archive@www.apache.org Delivered-To: apmail-cassandra-user-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 1CD058381 for ; Thu, 15 Sep 2011 13:40:50 +0000 (UTC) Received: (qmail 14319 invoked by uid 500); 15 Sep 2011 13:40:47 -0000 Delivered-To: apmail-cassandra-user-archive@cassandra.apache.org Received: (qmail 14291 invoked by uid 500); 15 Sep 2011 13:40:47 -0000 Mailing-List: contact user-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@cassandra.apache.org Delivered-To: mailing list user@cassandra.apache.org Received: (qmail 14265 invoked by uid 99); 15 Sep 2011 13:40:47 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 15 Sep 2011 13:40:47 +0000 X-ASF-Spam-Status: No, hits=1.6 required=5.0 tests=FREEMAIL_ENVFROM_END_DIGIT,FREEMAIL_FROM,HTML_MESSAGE,NORMAL_HTTP_TO_IP,RCVD_IN_DNSWL_LOW,SPF_PASS,T_TO_NO_BRKTS_FREEMAIL X-Spam-Check-By: apache.org Received-SPF: pass (athena.apache.org: domain of ethanrowe000@gmail.com designates 209.85.210.172 as permitted sender) Received: from [209.85.210.172] (HELO mail-iy0-f172.google.com) (209.85.210.172) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 15 Sep 2011 13:40:42 +0000 Received: by iaby26 with SMTP id y26so1491011iab.31 for ; Thu, 15 Sep 2011 06:40:22 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=gamma; h=mime-version:sender:in-reply-to:references:date :x-google-sender-auth:message-id:subject:from:to:content-type; bh=bPjba6j0QqdMY7U2ssMJAolEfJvUc9hqdq3D1t9LOQU=; b=kqPPVhX74kLCNPLd75wWw9BXEL/IsinNJKAPcK2lPC8sPxjTPrSbl35h3NVEpWpvm2 K45eKkjl4be69Em/TvxmrYNLVouSmHXGokPD6zaWbvsDMiX6MKi+N1S30niz4p9Npo3e Olc2dW+uVsmCUBsVYW42sAJiO0gi4w0ODtpwQ= MIME-Version: 1.0 Received: by 10.43.46.130 with SMTP id uo2mr44106icb.136.1316094020509; Thu, 15 Sep 2011 06:40:20 -0700 (PDT) Sender: ethanrowe000@gmail.com Received: by 10.42.228.136 with HTTP; Thu, 15 Sep 2011 06:40:20 -0700 (PDT) In-Reply-To: References: Date: Thu, 15 Sep 2011 09:40:20 -0400 X-Google-Sender-Auth: SJ-fLfSUCyHjK8kPoJgF6coc3kg Message-ID: Subject: Re: New node unable to stream (0.8.5) From: Ethan Rowe To: user@cassandra.apache.org Content-Type: multipart/alternative; boundary=bcaec52994c79561b304acfb0439 --bcaec52994c79561b304acfb0439 Content-Type: text/plain; charset=ISO-8859-1 On Thu, Sep 15, 2011 at 9:21 AM, Jonathan Ellis wrote: > Where did the data loss come in? > The outcome of the analytical jobs run overnight while some of these repairs were (not) running is consistent with what I would expect if perhaps 20-30% of the source data was missing. Given the strong consistency model we're using, this is surprising to me, since the jobs did not report any read or write failures. I wonder if this is a consequence of the dead node missing and the new node being operational but having received basically none of its hinted handoff streams. Perhaps with streaming fixed the data will reappear, which would be a happy outcome, but if not, I can reimport the critical stuff from files. Scrub is safe to run in parallel. > Is it somewhat analogous to a major compaction in terms of I/O impact, with perhaps less greedy use of disk space? > On Thu, Sep 15, 2011 at 8:08 AM, Ethan Rowe wrote: > > After further review, I'm definitely going to scrub all the original > nodes > > in the cluster. > > We've lost some data as a result of this situation. It can be restored, > but > > the question is what to do with the problematic new node first. I don't > > particularly care about the data that's on it, since I'm going to > re-import > > the critical data from files anyway, and then I can recreate derivative > data > > afterwards. So it's purely a matter of getting the cluster healthy again > as > > quickly as possible so I can begin that import process. > > Any issue with running scrubs on multiple nodes at a time, provided they > > aren't replication neighbors? > > On Thu, Sep 15, 2011 at 8:18 AM, Ethan Rowe wrote: > >> > >> I just noticed the following from one of Jonathan Ellis' messages > >> yesterday: > >>> > >>> Added to NEWS: > >>> > >>> - After upgrading, run nodetool scrub against each node before > running > >>> repair, moving nodes, or adding new ones. > >> > >> > >> We did not do this, as it was not indicated as necessary in the news > when > >> we were dealing with the upgrade. > >> So perhaps I need to scrub everything before going any further, though > the > >> question is what to do with the problematic node. Additionally, it > would be > >> helpful to know if scrub will affect the hinted handoffs that have > >> accumulated, as these seem likely to be part of the set of failing > streams. > >> On Thu, Sep 15, 2011 at 8:13 AM, Ethan Rowe > wrote: > >>> > >>> Here's a typical log slice (not terribly informative, I fear): > >>>> > >>>> INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,106 > >>>> AntiEntropyService.java (l > >>>> ine 884) Performing streaming repair of 1003 ranges with /10.34.90.8for > >>>> (299 > >>>> > >>>> > 90798416657667504332586989223299634,54296681768153272037430773234349600451] > >>>> INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,427 StreamOut.java > (line > >>>> 181) > >>>> Stream context metadata > >>>> [/mnt/cassandra/data/events_production/FitsByShip-g-1 > >>>> 0-Data.db sections=88 progress=0/11707163 - 0%, > >>>> /mnt/cassandra/data/events_pr > >>>> oduction/FitsByShip-g-11-Data.db sections=169 progress=0/6133240 - 0%, > >>>> /mnt/c > >>>> assandra/data/events_production/FitsByShip-g-6-Data.db sections=1 > >>>> progress=0/ > >>>> 6918814 - 0%, > >>>> /mnt/cassandra/data/events_production/FitsByShip-g-12-Data.db s > >>>> ections=260 progress=0/9091780 - 0%], 4 sstables. > >>>> INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,428 > StreamOutSession.java > >>>> (lin > >>>> e 174) Streaming to /10.34.90.8 > >>>> ERROR [Thread-56] 2011-09-15 05:41:38,515 AbstractCassandraDaemon.java > >>>> (line > >>>> 139) Fatal exception in thread Thread[Thread-56,5,main] > >>>> java.lang.NullPointerException > >>>> at > >>>> org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpC > >>>> onnection.java:174) > >>>> at > >>>> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConn > >>>> ection.java:114) > >>> > >>> Not sure if the exception is related to the outbound streaming above; > >>> other nodes are actively trying to stream to this node, so perhaps it > comes > >>> from those and temporal adjacency to the outbound stream is just > >>> coincidental. I have other snippets that look basically identical to > the > >>> above, except if I look at the logs to which this node is trying to > stream, > >>> I see that it has concurrently opened a stream in the other direction, > which > >>> could be the one that the exception pertains to. > >>> > >>> On Thu, Sep 15, 2011 at 7:41 AM, Sylvain Lebresne < > sylvain@datastax.com> > >>> wrote: > >>>> > >>>> On Thu, Sep 15, 2011 at 1:16 PM, Ethan Rowe > wrote: > >>>> > Hi. > >>>> > > >>>> > We've been running a 7-node cluster with RF 3, QUORUM reads/writes > in > >>>> > our > >>>> > production environment for a few months. It's been consistently > >>>> > stable > >>>> > during this period, particularly once we got out maintenance > strategy > >>>> > fully > >>>> > worked out (per node, one repair a week, one major compaction a > week, > >>>> > the > >>>> > latter due to the nature of our data model and usage). While this > >>>> > cluster > >>>> > started, back in June or so, on the 0.7 series, it's been running > >>>> > 0.8.3 for > >>>> > a while now with no issues. We upgraded to 0.8.5 two days ago, > having > >>>> > tested the upgrade in our staging cluster (with an otherwise > identical > >>>> > configuration) previously and verified that our application's > various > >>>> > use > >>>> > cases appeared successful. > >>>> > > >>>> > One of our nodes suffered a disk failure yesterday. We attempted to > >>>> > replace > >>>> > the dead node by placing a new node at OldNode.initial_token - 1 > with > >>>> > auto_bootstrap on. A few things went awry from there: > >>>> > > >>>> > 1. We never saw the new node in bootstrap mode; it became available > >>>> > pretty > >>>> > much immediately upon joining the ring, and never reported a > "joining" > >>>> > state. I did verify that auto_bootstrap was on. > >>>> > > >>>> > 2. I mistakenly ran repair on the new node rather than removetoken > on > >>>> > the > >>>> > old node, due to a delightful mental error. The repair got nowhere > >>>> > fast, as > >>>> > it attempts to repair against the down node which throws an > exception. > >>>> > So I > >>>> > interrupted the repair, restarted the node to clear any pending > >>>> > validation > >>>> > compactions, and... > >>>> > > >>>> > 3. Ran removetoken for the old node. > >>>> > > >>>> > 4. We let this run for some time and saw eventually that all the > nodes > >>>> > appeared to be done various compactions and were stuck at streaming. > >>>> > Many > >>>> > streams listed as open, none making any progress. > >>>> > > >>>> > 5. I observed an Rpc-related exception on the new node (where the > >>>> > removetoken was launched) and concluded that the streams were broken > >>>> > so the > >>>> > process wouldn't ever finish. > >>>> > > >>>> > 6. Ran a "removetoken force" to get the dead node out of the mix. > No > >>>> > problems. > >>>> > > >>>> > 7. Ran a repair on the new node. > >>>> > > >>>> > 8. Validations ran, streams opened up, and again things got stuck in > >>>> > streaming, hanging for over an hour with no progress. > >>>> > > >>>> > 9. Musing that lingering tasks from the removetoken could be a > factor, > >>>> > I > >>>> > performed a rolling restart and attempted a repair again. > >>>> > > >>>> > 10. Same problem. Did another rolling restart and attempted a fresh > >>>> > repair > >>>> > on the most important column family alone. > >>>> > > >>>> > 11. Same problem. Streams included CFs not specified, so I guess > they > >>>> > must > >>>> > be for hinted handoff. > >>>> > > >>>> > In concluding that streaming is stuck, I've observed: > >>>> > - streams will be open to the new node from other nodes, but the new > >>>> > node > >>>> > doesn't list them > >>>> > - streams will be open to the other nodes from the new node, but the > >>>> > other > >>>> > nodes don't list them > >>>> > - the streams reported may make some initial progress, but then they > >>>> > hang at > >>>> > a particular point and do not move on for an hour or more. > >>>> > - The logs report repair-related activity, until NPEs on incoming > TCP > >>>> > connections show up, which appear likely to be the culprit. > >>>> > >>>> Can you send the stack trace from those NPE. > >>>> > >>>> > > >>>> > I can provide more exact details when I'm done commuting. > >>>> > > >>>> > With streaming broken on this node, I'm unable to run repairs, which > >>>> > is > >>>> > obviously problematic. The application didn't suffer any > operational > >>>> > issues > >>>> > as a consequence of this, but I need to review the overnight results > >>>> > to > >>>> > verify we're not suffering data loss (I doubt we are). > >>>> > > >>>> > At this point, I'm considering a couple options: > >>>> > 1. Remove the new node and let the adjacent node take over its range > >>>> > 2. Bring the new node down, add a new one in front of it, and > properly > >>>> > removetoken the problematic one. > >>>> > 3. Bring the new node down, remove all its data except for the > system > >>>> > keyspace, then bring it back up and repair it. > >>>> > 4. Revert to 0.8.3 and see if that helps. > >>>> > > >>>> > Recommendations? > >>>> > > >>>> > Thanks. > >>>> > - Ethan > >>>> > > >>> > >> > > > > > > > > -- > Jonathan Ellis > Project Chair, Apache Cassandra > co-founder of DataStax, the source for professional Cassandra support > http://www.datastax.com > --bcaec52994c79561b304acfb0439 Content-Type: text/html; charset=ISO-8859-1 Content-Transfer-Encoding: quoted-printable
On Thu, Sep 15, 2011 at 9:21 AM, Jonathan Ellis = <jbellis@gmail.co= m> wrote:
Where did the data loss come in?

The ou= tcome of the analytical jobs run overnight while some of these repairs were= (not) running is consistent with what I would expect if perhaps 20-30% of = the source data was missing. =A0Given the strong consistency model we'r= e using, this is surprising to me, since the jobs did not report any read o= r write failures. =A0I wonder if this is a consequence of the dead node mis= sing and the new node being operational but having received basically none = of its hinted handoff streams. =A0Perhaps with streaming fixed the data wil= l reappear, which would be a happy outcome, but if not, I can reimport the = critical stuff from files.

Scrub is safe to run in parallel.

Is it= somewhat analogous to a major compaction in terms of I/O impact, with perh= aps less greedy use of disk space?
=A0
On Thu, Sep 15, 2011 at 8:08 AM, Ethan Rowe <ethan@the-rowes.com> wrote:
> After further review, I'm definitely going to scrub all the origin= al nodes
> in the cluster.
> We've lost some data as a result of this situation. =A0It can be r= estored, but
> the question is what to do with the problematic new node first. =A0I d= on't
> particularly care about the data that's on it, since I'm going= to re-import
> the critical data from files anyway, and then I can recreate derivativ= e data
> afterwards. =A0So it's purely a matter of getting the cluster heal= thy again as
> quickly as possible so I can begin that import process.
> Any issue with running scrubs on multiple nodes at a time, provided th= ey
> aren't replication neighbors?
> On Thu, Sep 15, 2011 at 8:18 AM, Ethan Rowe <ethan@the-rowes.com> wrote:
>>
>> I just noticed the following from one of Jonathan Ellis' messa= ges
>> yesterday:
>>>
>>> Added to NEWS:
>>>
>>> =A0 =A0- After upgrading, run nodetool scrub against each node= before running
>>> =A0 =A0 =A0repair, moving nodes, or adding new ones.
>>
>>
>> We did not do this, as it was not indicated as necessary in the ne= ws when
>> we were dealing with the upgrade.
>> So perhaps I need to scrub everything before going any further, th= ough the
>> question is what to do with the problematic node. =A0Additionally,= it would be
>> helpful to know if scrub will affect the hinted handoffs that have=
>> accumulated, as these seem likely to be part of the set of failing= streams.
>> On Thu, Sep 15, 2011 at 8:13 AM, Ethan Rowe <ethan@the-rowes.com> wrote:
>>>
>>> Here's a typical log slice (not terribly informative, I fe= ar):
>>>>
>>>> =A0INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,106
>>>> AntiEntropyService.java (l
>>>> ine 884) Performing streaming repair of 1003 ranges with /= 10.34.90.8 for
>>>> (299
>>>>
>>>> 90798416657667504332586989223299634,5429668176815327203743= 0773234349600451]
>>>> =A0INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,427 Strea= mOut.java (line
>>>> 181)
>>>> Stream context metadata
>>>> [/mnt/cassandra/data/events_production/FitsByShip-g-1
>>>> 0-Data.db sections=3D88 progress=3D0/11707163 - 0%,
>>>> /mnt/cassandra/data/events_pr
>>>> oduction/FitsByShip-g-11-Data.db sections=3D169 progress= =3D0/6133240 - 0%,
>>>> /mnt/c
>>>> assandra/data/events_production/FitsByShip-g-6-Data.db sec= tions=3D1
>>>> progress=3D0/
>>>> 6918814 - 0%,
>>>> /mnt/cassandra/data/events_production/FitsByShip-g-12-Data= .db s
>>>> ections=3D260 progress=3D0/9091780 - 0%], 4 sstables.
>>>> =A0INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,428 Strea= mOutSession.java
>>>> (lin
>>>> e 174) Streaming to /10.34.90.8
>>>> ERROR [Thread-56] 2011-09-15 05:41:38,515 AbstractCassandr= aDaemon.java
>>>> (line
>>>> 139) Fatal exception in thread Thread[Thread-56,5,main] >>>> java.lang.NullPointerException
>>>> =A0 =A0 =A0 =A0 at
>>>> org.apache.cassandra.net.IncomingTcpConnection.stream(Inco= mingTcpC
>>>> onnection.java:174)
>>>> =A0 =A0 =A0 =A0 at
>>>> org.apache.cassandra.net.IncomingTcpConnection.run(Incomin= gTcpConn
>>>> ection.java:114)
>>>
>>> Not sure if the exception is related to the outbound streaming= above;
>>> other nodes are actively trying to stream to this node, so per= haps it comes
>>> from those and temporal adjacency to the outbound stream is ju= st
>>> coincidental. =A0I have other snippets that look basically ide= ntical to the
>>> above, except if I look at the logs to which this node is tryi= ng to stream,
>>> I see that it has concurrently opened a stream in the other di= rection, which
>>> could be the one that the exception pertains to.
>>>
>>> On Thu, Sep 15, 2011 at 7:41 AM, Sylvain Lebresne <sylvain@datastax.com>
>>> wrote:
>>>>
>>>> On Thu, Sep 15, 2011 at 1:16 PM, Ethan Rowe <ethan@the-rowes.com> wrote:
>>>> > Hi.
>>>> >
>>>> > We've been running a 7-node cluster with RF 3, QU= ORUM reads/writes in
>>>> > our
>>>> > production environment for a few months. =A0It's = been consistently
>>>> > stable
>>>> > during this period, particularly once we got out main= tenance strategy
>>>> > fully
>>>> > worked out (per node, one repair a week, one major co= mpaction a week,
>>>> > the
>>>> > latter due to the nature of our data model and usage)= . =A0While this
>>>> > cluster
>>>> > started, back in June or so, on the 0.7 series, it= 9;s been running
>>>> > 0.8.3 for
>>>> > a while now with no issues. =A0We upgraded to 0.8.5 t= wo days ago, having
>>>> > tested the upgrade in our staging cluster (with an ot= herwise identical
>>>> > configuration) previously and verified that our appli= cation's various
>>>> > use
>>>> > cases appeared successful.
>>>> >
>>>> > One of our nodes suffered a disk failure yesterday. = =A0We attempted to
>>>> > replace
>>>> > the dead node by placing a new node at OldNode.initia= l_token - 1 with
>>>> > auto_bootstrap on. =A0A few things went awry from the= re:
>>>> >
>>>> > 1. We never saw the new node in bootstrap mode; it be= came available
>>>> > pretty
>>>> > much immediately upon joining the ring, and never rep= orted a "joining"
>>>> > state. =A0I did verify that auto_bootstrap was on. >>>> >
>>>> > 2. I mistakenly ran repair on the new node rather tha= n removetoken on
>>>> > the
>>>> > old node, due to a delightful mental error. =A0The re= pair got nowhere
>>>> > fast, as
>>>> > it attempts to repair against the down node which thr= ows an exception.
>>>> > =A0So I
>>>> > interrupted the repair, restarted the node to clear a= ny pending
>>>> > validation
>>>> > compactions, and...
>>>> >
>>>> > 3. Ran removetoken for the old node.
>>>> >
>>>> > 4. We let this run for some time and saw eventually t= hat all the nodes
>>>> > appeared to be done various compactions and were stuc= k at streaming.
>>>> > Many
>>>> > streams listed as open, none making any progress.
>>>> >
>>>> > 5.=A0 I observed an Rpc-related exception on the new = node (where the
>>>> > removetoken was launched) and concluded that the stre= ams were broken
>>>> > so the
>>>> > process wouldn't ever finish.
>>>> >
>>>> > 6. Ran a "removetoken force" to get the dea= d node out of the mix.=A0 No
>>>> > problems.
>>>> >
>>>> > 7. Ran a repair on the new node.
>>>> >
>>>> > 8. Validations ran, streams opened up, and again thin= gs got stuck in
>>>> > streaming, hanging for over an hour with no progress.=
>>>> >
>>>> > 9. Musing that lingering tasks from the removetoken c= ould be a factor,
>>>> > I
>>>> > performed a rolling restart and attempted a repair ag= ain.
>>>> >
>>>> > 10. Same problem.=A0 Did another rolling restart and = attempted a fresh
>>>> > repair
>>>> > on the most important column family alone.
>>>> >
>>>> > 11. Same problem.=A0 Streams included CFs not specifi= ed, so I guess they
>>>> > must
>>>> > be for hinted handoff.
>>>> >
>>>> > In concluding that streaming is stuck, I've obser= ved:
>>>> > - streams will be open to the new node from other nod= es, but the new
>>>> > node
>>>> > doesn't list them
>>>> > - streams will be open to the other nodes from the ne= w node, but the
>>>> > other
>>>> > nodes don't list them
>>>> > - the streams reported may make some initial progress= , but then they
>>>> > hang at
>>>> > a particular point and do not move on for an hour or = more.
>>>> > - The logs report repair-related activity, until NPEs= on incoming TCP
>>>> > connections show up, which appear likely to be the cu= lprit.
>>>>
>>>> Can you send the stack trace from those NPE.
>>>>
>>>> >
>>>> > I can provide more exact details when I'm done co= mmuting.
>>>> >
>>>> > With streaming broken on this node, I'm unable to= run repairs, which
>>>> > is
>>>> > obviously problematic.=A0 The application didn't = suffer any operational
>>>> > issues
>>>> > as a consequence of this, but I need to review the ov= ernight results
>>>> > to
>>>> > verify we're not suffering data loss (I doubt we = are).
>>>> >
>>>> > At this point, I'm considering a couple options:<= br> >>>> > 1. Remove the new node and let the adjacent node take= over its range
>>>> > 2. Bring the new node down, add a new one in front of= it, and properly
>>>> > removetoken the problematic one.
>>>> > 3. Bring the new node down, remove all its data excep= t for the system
>>>> > keyspace, then bring it back up and repair it.
>>>> > 4. Revert to 0.8.3 and see if that helps.
>>>> >
>>>> > Recommendations?
>>>> >
>>>> > Thanks.
>>>> > - Ethan
>>>> >
>>>
>>
>
>



--
Jonathan Ellis
Project Chair, Apache Cassandra
co-founder of DataStax, the source for professional Cassandra support
http://www.datastax.c= om

--bcaec52994c79561b304acfb0439--