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 BF1003E38 for ; Thu, 28 Apr 2011 08:58:18 +0000 (UTC) Received: (qmail 74682 invoked by uid 500); 28 Apr 2011 08:58:16 -0000 Delivered-To: apmail-cassandra-user-archive@cassandra.apache.org Received: (qmail 74629 invoked by uid 500); 28 Apr 2011 08:58:15 -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 74621 invoked by uid 99); 28 Apr 2011 08:58:15 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 28 Apr 2011 08:58:15 +0000 X-ASF-Spam-Status: No, hits=4.4 required=5.0 tests=FREEMAIL_ENVFROM_END_DIGIT,FREEMAIL_FROM,HTML_MESSAGE,NORMAL_HTTP_TO_IP,RCVD_IN_DNSWL_NONE,RFC_ABUSE_POST,SPF_PASS,T_TO_NO_BRKTS_FREEMAIL X-Spam-Check-By: apache.org Received-SPF: pass (athena.apache.org: domain of chensheng2010@gmail.com designates 209.85.220.172 as permitted sender) Received: from [209.85.220.172] (HELO mail-vx0-f172.google.com) (209.85.220.172) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 28 Apr 2011 08:58:10 +0000 Received: by vxg33 with SMTP id 33so2195718vxg.31 for ; Thu, 28 Apr 2011 01:57:49 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=gamma; h=domainkey-signature:mime-version:in-reply-to:references:date :message-id:subject:from:to:content-type; bh=FPWvEO75aKe7muVuoRIBUUtZ/I8A2m7x/QrqwWgdlCg=; b=XIs9NQRhirqF2nUAYoIe0NDK8ttR2T7g8JCv2EFPzdX/egYBJXCufw9UF/HzOkXTs2 2e8RYjIfG74drq4YOiSIhl18NZ445bFWlvItHz4KvoJUmXFSAmLIkx2d9AmQaH3sNZTS 4WSv5AnYjymnl5zChBQZBF0oEocekXvQA30bo= DomainKey-Signature: a=rsa-sha1; c=nofws; d=gmail.com; s=gamma; h=mime-version:in-reply-to:references:date:message-id:subject:from:to :content-type; b=b95b0Q21Y8NFzTUOpj3ZC7FjopYV1bQROOLmgs5QwsmWLE0wyKMDCYVe1EAD9D+4sv sge3797jsxZRsAAhMqqlMbfaxkMFBAOZYna6wdNr399OATYPSD688HhF5fwTvkdM9dOG fRL48/jAib3ivZVrkrwlhPWVafIRmNEd2YVbQ= MIME-Version: 1.0 Received: by 10.52.76.201 with SMTP id m9mr4543637vdw.154.1303981068984; Thu, 28 Apr 2011 01:57:48 -0700 (PDT) Received: by 10.52.107.2 with HTTP; Thu, 28 Apr 2011 01:57:48 -0700 (PDT) In-Reply-To: References: Date: Thu, 28 Apr 2011 16:57:48 +0800 Message-ID: Subject: Re: Heavy writes ok for single node, but failed for cluster From: Sheng Chen To: user@cassandra.apache.org Content-Type: multipart/alternative; boundary=bcaec501c62a69247c04a1f6c0cf --bcaec501c62a69247c04a1f6c0cf Content-Type: text/plain; charset=ISO-8859-1 Thank you for your advice. Rf>=2 is a good work around. I was using 0.7.4 and have updated to the latest 0.7 branch, which includes 2554 patch. But it doesn't help. I still get lots of UnavailableException after the following logs, INFO [GossipTasks:1] 2011-04-28 16:12:17,661 Gossiper.java (line 228) InetAddress /192.168.125.49 is now dead. INFO [GossipStage:1] 2011-04-28 16:12:19,627 Gossiper.java (line 609) InetAddress /192.168.125.49 is now UP INFO [HintedHandoff:1] 2011-04-28 16:13:11,452 HintedHandOffManager.java (line 304) Started hinted handoff for endpoint /192.168.125.49 INFO [HintedHandoff:1] 2011-04-28 16:13:11,453 HintedHandOffManager.java (line 360) Finished hinted handoff of 0 rows to endpoint /192.168.125.49 It seems that the gossip failure detection is too sensitive. Is there any configuration? 2011/4/27 Sylvain Lebresne > On Wed, Apr 27, 2011 at 10:32 AM, Sheng Chen > wrote: > > I succeeded to insert 1 billion records into a single node cassandra, > >>> bin/stress -d cas01 -o insert -n 1000000000 -c 5 -S 34 -C5 -t 20 > > Inserts finished in about 14 hours at a speed of 20k/sec. > > But when I added another node, tests always failed with > UnavailableException > > in an hour. > >>> bin/stress -d cas01,cas02 -o insert -n 1000000000 -c 5 -S 34 -C5 -t 20 > > Writes speed is also 20k/sec because of the bottleneck in the client, so > the > > pressure on each server node should be 50% of the single node test. > > Why couldn't they handle? > > By default, rf=1, consistency=ONE > > Some information that may be helpful, > > 1. no warn/error in log file, the cluster is still alive after those > > exception > > 2. the last logs on both nodes happen to be a compaction complete info > > 3. gossip log shows one node is dead and then up again in 3 seconds > > That's your problem. Once marked down (and since rf=1), when an update for > cas02 reach cas01 and cas01 has marked cas02 down, it will throw the > UnavailableException. > > Now, it shouldn't have been marked down and I suspect this is due to > https://issues.apache.org/jira/browse/CASSANDRA-2554 > (even though you didn't tell which version you're using, I suppose > this is a 0.7.*). > > If you apply this patch or use the svn current 0.7 branch, that should > hopefully > not happen again. > > Note that if you had rf >= 2, the node would still have been marked down > wrongly > for 3 seconds, but that would have been transparent to the stress test. > > > 4. I set hinted_handoff_enabled: false, but still see lots of handoff > logs > > What are those saying ? > > -- > Sylvain > --bcaec501c62a69247c04a1f6c0cf Content-Type: text/html; charset=ISO-8859-1 Content-Transfer-Encoding: quoted-printable Thank you for your advice.=A0Rf>=3D2 is a good work around.
I was us= ing 0.7.4 and have updated to the latest 0.7 branch, which includes 2554 pa= tch.
But it doesn't help. I still get lots of=A0UnavailableException after the following = logs,

=A0INFO [GossipTasks:1] 2011-04-28 16:12:17,661 Gossiper.java (line 228) = InetAddress /192.168.125.49 is now de= ad.
=A0INFO [GossipStage:1] 2011-04-28 16:12:19,627 Gossiper.java (line 60= 9) InetAddress /192.168.125.49 is now= UP

=A0INFO [HintedHandoff:1] 2011-04-28 16:1= 3:11,452 HintedHandOffManager.java (line 304) Started hinted handoff for en= dpoint /192.168.125.49
=A0INFO [HintedHandoff:1] 2011-04-28 16:13:11,453 HintedHandOffManager= .java (line 360) Finished hinted handoff of 0 rows to endpoint /192.168.125.49

It= seems that the gossip failure detection is too sensitive. Is there any con= figuration?





=
2011/4/27 Sylvain Lebresne <sylvain@datastax.com>
On Wed, A= pr 27, 2011 at 10:32 AM, Sheng Chen <chensheng2010@gmail.com> wrote:
> I succeeded to insert 1 billion records into a single node cassandra,<= br> >>> bin/stress -d cas01 -o insert -n 1000000000 -c 5 -S 34 -C5 -t = 20
> Inserts finished in about 14 hours at a speed of 20k/sec.
> But when I added another node, tests always failed with UnavailableExc= eption
> in an hour.
>>>=A0bin/stress -d cas01,cas02 -o insert -n 1000000000 -c 5 -S 34= -C5 -t 20
> Writes speed is also 20k/sec because of the bottleneck in the client, = so the
> pressure on each server node should be 50% of the single node test. > Why couldn't they handle?
> By default, rf=3D1, consistency=3DONE
> Some information that may be helpful,
> 1. no warn/error in log file, the cluster is still alive after those > exception
> 2. the last logs on both nodes happen to be a compaction complete info=
> 3. gossip log shows one node is dead and then up again in 3 seconds
That's your problem. Once marked down (and since rf=3D1), w= hen an update for
cas02 reach cas01 and cas01 has marked cas02 down, it will throw the
UnavailableException.

Now, it shouldn't have been marked down and I suspect this is due to https://issues.apache.org/jira/browse/CASSANDRA-2554
(even though you didn't tell which version you're using, I suppose<= br> this is a 0.7.*).

If you apply this patch or use the svn current 0.7 branch, that should hope= fully
not happen again.

Note that if you had rf >=3D 2, the node would still have been marked do= wn wrongly
for 3 seconds, but that would have been transparent to the stress test.

> 4. I set hinted_handoff_enabled: false, but still see lots of handoff = logs

What are those saying ?

--
Sylvain

--bcaec501c62a69247c04a1f6c0cf--