incubator-cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From aaron morton <aa...@thelastpickle.com>
Subject Re: Ec2Snitch to Ec2MultiRegionSnitch
Date Fri, 26 Apr 2013 01:24:43 GMT
> So you mean this part doesn't need more testing ? This will work for sure ? Did you already
did it yourself ?
Always test. 
But if you only had one AZ then all nodes will be in one Rack, so the NTS will not behave
differently. 

>  C* will be able to reach the LOCAL_QUORUM everywhere, won't it ? So why should I use
6 + 6 servers ?
LOCAL_QUOURM in structs the coordinator to wait for a QUOURM in the local DC. So if the coordinator
is in us-west then it will wait for nodes in there only.

You do not *have* to use 6+6 but if you have 300GB on each of the 6 nodes in us-west when
you add us-east with only 3 nodes and use the same RF they have to handle 600GB each. It's
just easier to have the same number of nodes in each DC. 

Cheers
  
-----------------
Aaron Morton
Freelance Cassandra Consultant
New Zealand

@aaronmorton
http://www.thelastpickle.com

On 24/04/2013, at 6:48 PM, Alain RODRIGUEZ <arodrime@gmail.com> wrote:

> "If you are only using one Available Zone per region then you have only one rack per
DC and the NetworkTopologyStrategy will do the right thing."
> 
> So you mean this part doesn't need more testing ? This will work for sure ? Did you already
did it yourself ?
> 
> "Because you are going to replicate your data 3 times in each DC so that each DC can
operate with a LOCAL_QUOURM"
> Yet I don't get it. Tell me where I am wrong. LOCAL_QUORUM need to read/write 2 nodes
(since RF = 3) per region. So if I use 6 eu-west and 3 us-east, C* will be able to reach the
LOCAL_QUORUM everywhere, won't it ? So why should I use 6 + 6 servers ?
> 
> "nodetool rebuild is designed to handle pulling data from another dc, so you can use
it when the local DC does not contain data. i.e. you do not want a node in the new DC bootstrapping
from other nodes in the new DC, they have no data"
> Good to know, thanks about it, as about all your pointers to the doc.
> 
> "Cause it's easier to understand than interleaving the nodes and works with 2+ DC's."
> Good point.
> 
> If your are interested, I'll let you know how all the things go when we'll add the second
DC.
> 
> 
> 2013/4/24 aaron morton <aaron@thelastpickle.com>
> > You are advising me to test it, what would be a good way of testing it (I can use
AWS EC2 instances if needed) ?
> If you are only using one Available Zone per region then you have only one rack per DC
and the NetworkTopologyStrategy will do the right thing.
> 
> > Why ? I mean we have maybe only 5% of our customers on the us-east zone, what in
C* require to have the same number of node on each DC ?
> Because you are going to replicate your data 3 times in each DC so that each DC can operate
with a LOCAL_QUOURM.
> 
> > What is better on adding nodes with no data and then rebuild them compared to using
the auto_bootstrap ?
> 
> nodetool rebuild is designed to handle pulling data from another dc, so you can use it
when the local DC does not contain data. i.e. you do not want a node in the new DC bootstrapping
from other nodes in the new DC, they have no data.
> 
> > Any doc on this ? I am not aware of all the possibilities. Why is this the best
method according to you ?
> http://wiki.apache.org/cassandra/Operations?highlight=%28token%29#Token_selection
> http://www.datastax.com/docs/1.2/initialize/token_generation
> 
> Cause it's easier to understand than interleaving the nodes and works with 2+ DC's.
> 
> > What is the point of this ?
> http://wiki.apache.org/cassandra/FAQ#seed
> 
> > I didn't thought this change would be that tricky, thank you guys for these warnings
and your help ;)
> Yup, this is a lot of work.
> 
> Cheers
> -----------------
> Aaron Morton
> Freelance Cassandra Consultant
> New Zealand
> 
> @aaronmorton
> http://www.thelastpickle.com
> 
> On 23/04/2013, at 7:26 PM, Alain RODRIGUEZ <arodrime@gmail.com> wrote:
> 
> > Hi,these advice are very welcome.
> >
> > @Dane, about the rack awareness, we use only one rack per DC, so I guess using EC2MultiRegionSnitch
will do just fine and it doesn't need any configuration. Does it seem right to you. If we
are someday interested on multi racks I will make sure to use them properly. Thank you for
this insight anyway. You are advising me to test it, what would be a good way of testing it
(I can use AWS EC2 instances if needed) ?
> >
> > @Aaron
> >
> > "I recommend using the same number of nodes in both DC's."
> >
> > Why ? I mean we have maybe only 5% of our customers on the us-east zone, what in
C* require to have the same number of node on each DC ?
> >
> > "Add the nodes (I recommend 6) with auto_bootstrap: false added to the yaml.
> > update the keyspace replication strategy to add rf:3 for the new DC.
> > Use nodetool rebuild on the new nodes to rebuild them from the us-west DC. "
> >
> > What is better on adding nodes with no data and then rebuild them compared to using
the auto_bootstrap ?
> >
> > "I prefer to use the offset method. Take the 6 tokens from your us-west DC and add
100 to them for the new dc. "
> >
> > Any doc on this ? I am not aware of all the possibilities. Why is this the best
method according to you ?
> >
> > About seeds => "Yes. Have 3 from each."
> >
> > What is the point of this ?
> >
> > I didn't thought this change would be that tricky, thank you guys for these warnings
and your help ;)
> >
> > Alain
> >
> >
> > 2013/4/23 Dane Miller <dane@optimalsocial.com>
> > On Thu, Apr 18, 2013 at 7:41 AM, Alain RODRIGUEZ <arodrime@gmail.com> wrote:
> > > I am wondering about the process to grow from one data center to a few of
> > > them. First thing is we use EC2Snitch for now. So I guess we have to switch
> > > to Ec2MultiRegionSnitch.
> > >
> > > c/ I am using the SimpleStrategy. Is it worth it/mandatory to change this
> > > strategy when using multiple DC ?
> >
> > I suggest you thoroughly read the datastax documentation on cassandra
> > replication.  The change you are planning is big - make sure to try it
> > in a test environment first.  Also, you might find you don't really
> > need Cassandra's rack aware feature, and can operate using
> > (Gossiping)PropertyFileSnitch.  The rack feature is listed as an
> > "anti-pattern" here:
> > http://www.datastax.com/docs/1.2/cluster_architecture/anti_patterns
> >
> > Here are some recent discussions on this list:
> > http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/migrating-from-SimpleStrategy-to-NetworkTopologyStrategy-tp7586272.html
> > http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/migrating-from-SimpleStrategy-to-NetworkTopologyStrategy-tp7481090.html
> >
> > Dane
> >
> 
> 


Mime
View raw message