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 3483A510A for ; Thu, 12 May 2011 02:26:58 +0000 (UTC) Received: (qmail 65316 invoked by uid 500); 12 May 2011 02:26:55 -0000 Delivered-To: apmail-cassandra-user-archive@cassandra.apache.org Received: (qmail 65291 invoked by uid 500); 12 May 2011 02:26:55 -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 65283 invoked by uid 99); 12 May 2011 02:26:55 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 12 May 2011 02:26:55 +0000 X-ASF-Spam-Status: No, hits=2.9 required=5.0 tests=HTML_MESSAGE,RCVD_IN_DNSWL_NONE,SPF_NEUTRAL X-Spam-Check-By: apache.org Received-SPF: neutral (nike.apache.org: local policy) Received: from [66.33.216.122] (HELO hapkido.dreamhost.com) (66.33.216.122) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 12 May 2011 02:26:48 +0000 Received: from homiemail-a45.g.dreamhost.com (caibbdcaaaaf.dreamhost.com [208.113.200.5]) by hapkido.dreamhost.com (Postfix) with ESMTP id B326E1844C7 for ; Wed, 11 May 2011 19:26:26 -0700 (PDT) Received: from homiemail-a45.g.dreamhost.com (localhost [127.0.0.1]) by homiemail-a45.g.dreamhost.com (Postfix) with ESMTP id 2C490480A0 for ; Wed, 11 May 2011 19:26:19 -0700 (PDT) DomainKey-Signature: a=rsa-sha1; c=nofws; d=thelastpickle.com; h=from :mime-version:content-type:subject:date:in-reply-to:to :references:message-id; q=dns; s=thelastpickle.com; b=dUolkr3ei/ +wYQUh1YIspWG68gdTqNfeuCf7l0QBzvfXmOyN6AynTeay57Z9E+N5J7O5HoNHj7 XKjuApz7fNh2pFWlFFYOPzx/cY+Vmn2EZW6Iw9AZNCUesesxGz7do982mxadYFx9 MhjP6a9Vyhv+Ag9ww3OTCpw9aa1qBYpJs= DKIM-Signature: v=1; a=rsa-sha1; c=relaxed; d=thelastpickle.com; h=from :mime-version:content-type:subject:date:in-reply-to:to :references:message-id; s=thelastpickle.com; bh=H4uukqX10WaJ6PtF I3Z6/o1Uiss=; b=b9PdXRhXDuSW7dXMmT4/3mtRLlsz74XOOcEY822so93RbtJT FcUJxZ4J9hrqi1ut1OLzlrpO8qN00GqUYPw7dkvfiG92DIGZ1Zj2dUvyxqAmVV6c DfanW1OUBV4ED7s2FxjaJTbftVSHEemhO59kr5l9IdyjYuvskMdy55yj2tc= Received: from [10.0.1.151] (121-73-157-230.cable.telstraclear.net [121.73.157.230]) (using TLSv1 with cipher AES128-SHA (128/128 bits)) (No client certificate requested) (Authenticated sender: aaron@thelastpickle.com) by homiemail-a45.g.dreamhost.com (Postfix) with ESMTPSA id 6015848021 for ; Wed, 11 May 2011 19:26:18 -0700 (PDT) From: aaron morton Mime-Version: 1.0 (Apple Message framework v1084) Content-Type: multipart/alternative; boundary=Apple-Mail-57-442461583 Subject: Re: network topology issue Date: Thu, 12 May 2011 14:26:15 +1200 In-Reply-To: To: user@cassandra.apache.org References: Message-Id: X-Mailer: Apple Mail (2.1084) X-Virus-Checked: Checked by ClamAV on apache.org --Apple-Mail-57-442461583 Content-Transfer-Encoding: quoted-printable Content-Type: text/plain; charset=windows-1252 When creating a multi DC deployment tokens should be evenly distributed = in *each* dc, see this recent discussion for an example http://www.mail-archive.com/user@cassandra.apache.org/msg12975.html = (I'll also update the wiki when I get time, making a note now) But no = two nodes in the global ring can have the same token, hence the error. When using the NTS the RF must be set per DC using the = strategy_options clause in the `create keyspace CL i statement. The = global RF is just the sum of the per DC values.=20 Hope that helps.=20 ----------------- Aaron Morton Freelance Cassandra Developer @aaronmorton http://www.thelastpickle.com On 12 May 2011, at 12:59, Sameer Farooqui wrote: > Yeah, Narendra is correct. >=20 > If you have 2 nodes, one in each data center, use RF=3D2 and do reads = and writes with either level ONE or QUORUM (which means 2 in this case). >=20 > However, if you had 2 nodes in DC1 and 1 node in DC2, then you could = use RF=3D3 and use LOCAL_QUORUM for reads and writes. >=20 > For writes, LOCAL_QUORUM means: Ensure that the write has been written = to / 2 + 1 nodes, within the local datacenter = (requires NetworkTopologyStrategy) >=20 > For reads, LOCAL_QUORUM means: Returns the record with the most recent = timestamp once a majority of replicas within the local datacenter have = replied. >=20 > - Sameer >=20 > On Wed, May 11, 2011 at 5:49 PM, Narendra Sharma = wrote: > My understanding is that the replication factor is for the entire = ring. Even if you have 2 DCs the nodes are part of the same ring. What = you get additionally from NTS is that you can specify how many replicas = to place in each DC. >=20 > So RF =3D 1 and DC1:1, DC2:1 looks incorrect to me. >=20 > What is possible with NTS is following: > RF=3D3, DC1=3D1, DC2=3D2 >=20 > Would wait for others comments to see if my understand is correct. >=20 > -Naren >=20 >=20 > On Wed, May 11, 2011 at 5:41 PM, Anurag Gujral = wrote: > Thanks Sameer for your answer.=20 > I am using two DCs DC1 , DC2 with both having one node each, my = straegy_options values are DC1:1,DC2:1 I am not sure what my RF should = be , should it be 1 or 2? > Please Advise > Thanks > Anurag >=20 >=20 > On Wed, May 11, 2011 at 5:27 PM, Sameer Farooqui = wrote: > Anurag, >=20 > The Cassandra ring spans datacenters, so you can't use token 0 on both = nodes. Cassandra=92s ring is from 0 to 2**127 in size. >=20 > Try assigning one node the token of 0 and the second node 8.50705917 =D7= 10^37 (input this as a single long number). >=20 > To add a new keyspace in 0.8, run this from the CLI: > create keyspace KEYSPACENAME with placement_strategy =3D = org.apache.Cassandra.locator.NetworkTopologyStrategy' and = strategy_options =3D [{replication_factor:2}]; >=20 > If using 0.7, run "help create keyspace;" from the CLI and it'll show = you the correct syntax. >=20 >=20 > More info on tokens: > = http://journal.paul.querna.org/articles/2010/09/24/cassandra-token-selecti= on/ > http://wiki.apache.org/cassandra/Operations#Token_selection >=20 >=20 > On Wed, May 11, 2011 at 4:58 PM, Anurag Gujral = wrote: > Hi All, > I am testing network topology strategy in cassandra I am = using two nodes , one node each in different data center. > Since the nodes are in different dc I assigned token 0 to both the = nodes. > I added both the nodes as seeds in the cassandra.yaml and I am using = properyfilesnitch as endpoint snitch where I have specified the colo = details. >=20 > I started first node then I when I restarted second node I got an = error that token "0" is already being used.Why am I getting this error. >=20 > Second Question: I already have cassandra running in two different = data centers I want to add a new keyspace which uses networkTopology = strategy > in the light of above errors how can I accomplish this. >=20 >=20 > Thanks > Anurag >=20 >=20 >=20 >=20 >=20 > --=20 > Narendra Sharma > Solution Architect > http://www.persistentsys.com > http://narendrasharma.blogspot.com/ >=20 >=20 >=20 --Apple-Mail-57-442461583 Content-Transfer-Encoding: quoted-printable Content-Type: text/html; charset=windows-1252 When = creating a multi DC deployment tokens should be evenly distributed in = *each* dc, see this recent discussion for an example
http://www.mail-archive.com/user@cassandra.apache.org/msg12975.html=  (I'll also update the wiki when I get time, making a note now) But = no two nodes in the global ring can have the same token, hence the = error.

 When using the NTS the RF must be = set per DC using the  strategy_options clause in the `create = keyspace  CL i statement. The global RF is just the sum of the per = DC values. 

 Hope that = helps. 

http://www.thelastpickle.com

On 12 May 2011, at 12:59, Sameer Farooqui wrote:

Yeah, = Narendra is correct.

If you have 2 nodes, one in each = data center, use RF=3D2 and do reads and writes with either level ONE = or QUORUM (which means 2 in this = case).

However, if you had 2 nodes in DC1 and 1 = node in DC2, then you could use RF=3D3 and use LOCAL_QUORUM for reads = and writes.

For writes, LOCAL_QUORUM means: Ensure = that the write has been written to <ReplicationFactor> / = 2 + 1 nodes, within the local datacenter = (requires NetworkTopologyStrategy)

For reads, LOCAL_QUORUM means: Returns the record with the = most recent timestamp once a majority of replicas within the local = datacenter have replied.

- Sameer

On Wed, May 11, 2011 at 5:49 PM, Narendra Sharma = <narendra.sharma@gmail.com>= ; wrote:
My understanding is = that the replication factor is for the entire ring. Even if you have 2 = DCs the nodes are part of the same ring. What you get additionally from = NTS is that you can specify how many replicas to place in each DC.

So RF =3D 1 and DC1:1, DC2:1 looks incorrect to = me.

What is possible with NTS is = following:
RF=3D3, DC1=3D1, = DC2=3D2

Would wait for others comments to see = if my understand is correct.

-Naren


On Wed, May 11, 2011 at 5:41 PM, Anurag Gujral = <anurag.gujral@gmail.com> wrote:
Thanks Sameer for your answer.
I am using two DCs DC1 , DC2 with = both having one node each, my straegy_options values are = DC1:1,DC2:1  I am not sure what my RF should be , should it be 1 or = 2?
Please Advise
Thanks
Anurag


On = Wed, May 11, 2011 at 5:27 PM, Sameer Farooqui <cassandralabs@gmail.com> wrote:
Anurag,

The Cassandra ring spans datacenters, so you = can't use token 0 on both nodes. Cassandra=92s ring is from 0 to = 2**127 in size.

Try assigning one node the = token of 0 and the second node 8.50705917 =D7 10^37 (input this as = a single long number).

To add a new keyspace in 0.8, run this from the = CLI:
create keyspace KEYSPACENAME with placement_strategy =3D = org.apache.Cassandra.locator.NetworkTopologyStrategy' and = strategy_options =3D [{replication_factor:2}];

If using 0.7, run "help create keyspace;" from the = CLI and it'll show you the correct = syntax.


More info on = tokens:
http://wiki.apache.org/cassandra/Operations#Token_select= ion


On Wed, May 11, 2011 at 4:58 PM, Anurag = Gujral <anurag.gujral@gmail.com> = wrote:
Hi = All,
           =   I am testing network topology strategy in cassandra I am using = two nodes , one node each in different data center.
Since the nodes = are in different dc I assigned token 0 to both the nodes.
I added = both the nodes as seeds in the cassandra.yaml and  I am  using = properyfilesnitch as endpoint snitch where I have specified the colo = details.

I started first node then I when I restarted second node I got an = error that token "0" is already being used.Why am I getting this = error.

Second Question: I already have cassandra running in two = different data centers I want to add a new keyspace which uses = networkTopology strategy
in the light of above errors how can I accomplish = this.


Thanks
Anurag





-- =
Narendra Sharma


= --Apple-Mail-57-442461583--