cassandra-pr mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From aweisberg <...@git.apache.org>
Subject [GitHub] cassandra pull request #224: 14405 replicas
Date Wed, 16 May 2018 17:05:40 GMT
Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/224#discussion_r187429139
  
    --- Diff: src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java ---
    @@ -96,7 +98,35 @@ private void warnIfIncreasingRF(KeyspaceMetadata ksm, KeyspaceParams
params)
                                                                                         
                   StorageService.instance.getTokenMetadata(),
                                                                                         
                   DatabaseDescriptor.getEndpointSnitch(),
                                                                                         
                   params.replication.options);
    -        if (newStrategy.getReplicationFactor() > oldStrategy.getReplicationFactor())
    +
    +        validateTransientReplication(oldStrategy, newStrategy);
    +        warnIfIncreasingRF(oldStrategy, newStrategy);
    +    }
    +
    +    private void validateTransientReplication(AbstractReplicationStrategy oldStrategy,
AbstractReplicationStrategy newStrategy)
    +    {
    +        if (oldStrategy.getReplicationFactor().trans == 0 && newStrategy.getReplicationFactor().trans
> 0)
    +        {
    +            Keyspace ks = Keyspace.open(keyspace());
    +            for (ColumnFamilyStore cfs: ks.getColumnFamilyStores())
    +            {
    +                if (cfs.viewManager.hasViews())
    +                {
    +                    throw new ConfigurationException("Cannot use transient replication
on keyspaces using materialized views");
    +                }
    +
    +                if (cfs.indexManager.hasIndexes())
    +                {
    +                    throw new ConfigurationException("Cannot use transient replication
on keyspaces using secondary indexes");
    +                }
    +            }
    +
    +        }
    +    }
    --- End diff --
    
    I think we can allow people to increase # of transient replicas (they make it real with
nodetool cleanup or repair), but how does decreasing number of transient replicas work safely?
I think we should disallow it until we can articulate how and why it is safe. There has to
be a temporary pending state where a node is transitioning from transient to full where it
receives writes but not reads because it can't correctly service reads as a full replcia.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


Mime
View raw message