[ 
https://issues.apache.org/jira/browse/CASSANDRA-14303?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Joseph Lynch updated CASSANDRA-14303:
-------------------------------------
    Description: 
Right now when creating a keyspace with {{NetworkTopologyStrategy}} the user 
has to manually specify the datacenters they want their data replicated to with 
parameters, e.g.:
{noformat}
 CREATE KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy', 
'dc1': 3, 'dc2': 3}{noformat}
This is a poor user interface because it requires the creator of the keyspace 
(typically a developer) to know the layout of the Cassandra cluster (which may 
or may not be controlled by them). Also, at least in my experience, folks typo 
the datacenters _all_ the time. To work around this I see a number of users 
creating automation around this where the automation describes the Cassandra 
cluster and automatically expands out to all the dcs that Cassandra knows 
about. Why can't Cassandra just do this for us, re-using the previously 
forbidden {{replication_factor}} option (for backwards compatibility):
{noformat}
 CREATE KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy', 
'replication_factor': 3}{noformat}
This would automatically replicate this Keyspace to all datacenters that are 
present in the cluster. If you need to _override_ the default you could supply 
a datacenter name, e.g.:
{noformat}
CREATE KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy', 
'replication_factor': 3, 'dc1': 0}
{noformat}
On the implementation side I think this may be reasonably straightforward to do 
an auto-expansion at the time of keyspace creation (or alter), where the above 
would automatically expand to list out the datacenters. We could allow this to 
be recomputed whenever an AlterKeyspaceStatement runs so that to add 
datacenters you would just run:
{noformat}
ALTER KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy', 
'replication_factor': 3}{noformat}
and this would check that if the dc's in the current schema are different you 
add in the new ones (for safety reasons we'd probably never remove none zero rf 
dcs when auto-generating dcs). Removing a datacenter becomes a two step 
process, e.g. if we wanted to remove {{dc1}} we would do:
{noformat}
// First tell it not to replicate to dc1
ALTER KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy', 
'replication_factor': 3, 'dc1': 0}
// Remove all nodes from dc1
ALTER KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy', 
'replication_factor': 3}{noformat}
I think the only issue with this would be that I think {{EACH_QUORUM}} doesn't 
handle DCs with 0 replicas very well, but I think that is tractable.

  was:
Right now when creating a keyspace with {{NetworkTopologyStrategy}} the user 
has to manually specify the datacenters they want their data replicated to with 
parameters, e.g.:
{noformat}
 CREATE KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy', 
'dc1': 3, 'dc2': 3}{noformat}
This is a poor user interface because it requires the creator of the keyspace 
(typically a developer) to know the layout of the Cassandra cluster (which may 
or may not be controlled by them). Also, at least in my experience, folks typo 
the datacenters _all_ the time. To work around this I see a number of users 
creating automation around this where the automation describes the Cassandra 
cluster and automatically expands out to all the dcs that Cassandra knows 
about. Why can't Cassandra just do this for us, re-using the previously 
forbidden {{replication_factor}} option (for backwards compatibility):
{noformat}
 CREATE KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy', 
'replication_factor': 3}{noformat}
This would automatically replicate this Keyspace to all datacenters that are 
present in the cluster. If you need to _override_ the default you could supply 
a datacenter name, e.g.:
{noformat}
CREATE KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy', 
'replication_factor': 3, 'dc1': 0}
{noformat}
On the implementation side I think this may be reasonably straightforward to do 
an auto-expansion at the time of keyspace creation (or alter), where the above 
would automatically expand to list out the datacenters. We could allow this to 
be recomputed whenever an AlterKeyspaceStatement runs so that to add 
datacenters you would just run:
{noformat}
ALTER KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy', 
'replication_factor': 3}{noformat}
and this would check that if the dc's in the current schema are different you 
add in the new ones (for safety reasons we'd probably never remove dcs when 
auto-generating dcs). Removing a datacenter becomes a two step process, e.g. if 
we wanted to remove {{dc1}} we would do:
{noformat}
// First tell it not to replicate to dc1
ALTER KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy', 
'replication_factor': 3, 'dc1': 0}
// Remove all nodes from dc1
ALTER KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy', 
'replication_factor': 3}{noformat}
I think the only issue with this would be that I think {{EACH_QUORUM}} doesn't 
handle DCs with 0 replicas very well, but I think that is tractable.


> NetworkTopologyStrategy could have a "default replication" option
> -----------------------------------------------------------------
>
>                 Key: CASSANDRA-14303
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-14303
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Configuration
>            Reporter: Joseph Lynch
>            Priority: Minor
>
> Right now when creating a keyspace with {{NetworkTopologyStrategy}} the user 
> has to manually specify the datacenters they want their data replicated to 
> with parameters, e.g.:
> {noformat}
>  CREATE KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy', 
> 'dc1': 3, 'dc2': 3}{noformat}
> This is a poor user interface because it requires the creator of the keyspace 
> (typically a developer) to know the layout of the Cassandra cluster (which 
> may or may not be controlled by them). Also, at least in my experience, folks 
> typo the datacenters _all_ the time. To work around this I see a number of 
> users creating automation around this where the automation describes the 
> Cassandra cluster and automatically expands out to all the dcs that Cassandra 
> knows about. Why can't Cassandra just do this for us, re-using the previously 
> forbidden {{replication_factor}} option (for backwards compatibility):
> {noformat}
>  CREATE KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy', 
> 'replication_factor': 3}{noformat}
> This would automatically replicate this Keyspace to all datacenters that are 
> present in the cluster. If you need to _override_ the default you could 
> supply a datacenter name, e.g.:
> {noformat}
> CREATE KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy', 
> 'replication_factor': 3, 'dc1': 0}
> {noformat}
> On the implementation side I think this may be reasonably straightforward to 
> do an auto-expansion at the time of keyspace creation (or alter), where the 
> above would automatically expand to list out the datacenters. We could allow 
> this to be recomputed whenever an AlterKeyspaceStatement runs so that to add 
> datacenters you would just run:
> {noformat}
> ALTER KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy', 
> 'replication_factor': 3}{noformat}
> and this would check that if the dc's in the current schema are different you 
> add in the new ones (for safety reasons we'd probably never remove none zero 
> rf dcs when auto-generating dcs). Removing a datacenter becomes a two step 
> process, e.g. if we wanted to remove {{dc1}} we would do:
> {noformat}
> // First tell it not to replicate to dc1
> ALTER KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy', 
> 'replication_factor': 3, 'dc1': 0}
> // Remove all nodes from dc1
> ALTER KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy', 
> 'replication_factor': 3}{noformat}
> I think the only issue with this would be that I think {{EACH_QUORUM}} 
> doesn't handle DCs with 0 replicas very well, but I think that is tractable.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to