Author: jbellis
Date: Tue Sep 21 15:40:23 2010
New Revision: 999469

URL: http://svn.apache.org/viewvc?rev=999469&view=rev
Log:
use total cluster membership rather than live node count when validating new 
keyspace replication factor, and set cli default RF to 1.  patch by jbellis

Modified:
    cassandra/trunk/src/java/org/apache/cassandra/avro/CassandraServer.java
    cassandra/trunk/src/java/org/apache/cassandra/cli/CliClient.java
    cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
    cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/avro/CassandraServer.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/avro/CassandraServer.java?rev=999469&r1=999468&r2=999469&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/avro/CassandraServer.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/avro/CassandraServer.java Tue 
Sep 21 15:40:23 2010
@@ -46,6 +46,7 @@ import org.apache.cassandra.db.migration
 import org.apache.cassandra.db.migration.UpdateColumnFamily;
 import org.apache.cassandra.db.migration.UpdateKeyspace;
 import org.apache.cassandra.dht.*;
+import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.utils.FBUtilities;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -625,10 +626,12 @@ public class CassandraServer implements 
     {
         if (!(DatabaseDescriptor.getAuthenticator() instanceof 
AllowAllAuthenticator))
             throw newInvalidRequestException("Unable to create new keyspace 
while authentication is enabled.");
-        
-        if (StorageService.instance.getLiveNodes().size() < 
ksDef.replication_factor)
-            throw newInvalidRequestException("Not enough live nodes to support 
this keyspace");
-        
+
+        int totalNodes = Gossiper.instance.getLiveMembers().size() + 
Gossiper.instance.getUnreachableMembers().size();
+        if (totalNodes < ksDef.replication_factor)
+            throw newInvalidRequestException(String.format("%s live nodes are 
not enough to support replication factor %s",
+                                                           totalNodes, 
ksDef.replication_factor));
+
         //generate a meaningful error if the user setup keyspace and/or column 
definition incorrectly
         for (CfDef cf : ksDef.cf_defs) 
         {
@@ -737,8 +740,10 @@ public class CassandraServer implements 
         if (ks_def.cf_defs != null && ks_def.cf_defs.size() > 0)
             throw newInvalidRequestException("Keyspace update must not contain 
any column family definitions.");
         
-        if (StorageService.instance.getLiveNodes().size() < 
ks_def.replication_factor)
-            throw newInvalidRequestException("Not enough live nodes to support 
this keyspace");
+        int totalNodes = Gossiper.instance.getLiveMembers().size() + 
Gossiper.instance.getUnreachableMembers().size();
+        if (totalNodes < ks_def.replication_factor)
+            throw newInvalidRequestException(String.format("%s live nodes are 
not enough to support replication factor %s",
+                                                           totalNodes, 
ks_def.replication_factor));
         if (DatabaseDescriptor.getTableDefinition(ks_def.name.toString()) == 
null)
             throw newInvalidRequestException("Keyspace does not exist.");
         

Modified: cassandra/trunk/src/java/org/apache/cassandra/cli/CliClient.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/cli/CliClient.java?rev=999469&r1=999468&r2=999469&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/cli/CliClient.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/cli/CliClient.java Tue Sep 21 
15:40:23 2010
@@ -718,7 +718,7 @@ public class CliClient 
 
         //defaults
         String replicaPlacementStrategy = 
"org.apache.cassandra.locator.SimpleStrategy";
-        int replicationFactor = 2;
+        int replicationFactor = 1;
 
         /*
          * first value is the keyspace name, after that it is all key=value

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java?rev=999469&r1=999468&r2=999469&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java 
Tue Sep 21 15:40:23 2010
@@ -1705,10 +1705,11 @@ public class StorageService implements I
                     throw new ConfigurationException("Cannot load from XML on 
top of pre-existing schemas.");
              
                 // cycle through first to make sure we can satisfy live nodes 
constraint.
-                int liveNodeCount = getLiveNodes().size();
+                int totalNodes = Gossiper.instance.getLiveMembers().size() + 
Gossiper.instance.getUnreachableMembers().size();
                 for (KSMetaData table : tables)
-                    if (liveNodeCount < table.replicationFactor)
-                        throw new ConfigurationException("Not enough live 
nodes to support this keyspace: " + table.name);
+                    if (totalNodes < table.replicationFactor)
+                        throw new ConfigurationException(String.format("%s 
live nodes are not enough to support replication factor %s",
+                                                                       
totalNodes, table.replicationFactor));
                 
                 Migration migration = null;
                 for (KSMetaData table : tables)

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java?rev=999469&r1=999468&r2=999469&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java 
Tue Sep 21 15:40:23 2010
@@ -29,6 +29,7 @@ import org.apache.cassandra.concurrent.S
 import org.apache.cassandra.db.migration.Migration;
 import org.apache.cassandra.db.migration.UpdateColumnFamily;
 import org.apache.cassandra.db.migration.UpdateKeyspace;
+import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.utils.FBUtilities;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -748,9 +749,11 @@ public class CassandraServer implements 
         if (!(DatabaseDescriptor.getAuthenticator() instanceof 
AllowAllAuthenticator))
             throw new InvalidRequestException("Unable to create new keyspace 
while authentication is enabled.");
 
-        if (StorageService.instance.getLiveNodes().size() < 
ks_def.replication_factor)
-            throw new InvalidRequestException("Not enough live nodes to 
support this keyspace");
-        
+        int totalNodes = Gossiper.instance.getLiveMembers().size() + 
Gossiper.instance.getUnreachableMembers().size();
+        if (totalNodes < ks_def.replication_factor)
+            throw new InvalidRequestException(String.format("%s live nodes are 
not enough to support replication factor %s",
+                                                            totalNodes, 
ks_def.replication_factor));
+
         //generate a meaningful error if the user setup keyspace and/or column 
definition incorrectly
         for (CfDef cf : ks_def.cf_defs) 
         {
@@ -855,8 +858,10 @@ public class CassandraServer implements 
         if (ks_def.getCf_defs() != null && ks_def.getCf_defs().size() > 0)
             throw new InvalidRequestException("Keyspace update must not 
contain any column family definitions.");
         
-        if (StorageService.instance.getLiveNodes().size() < 
ks_def.replication_factor)
-            throw new InvalidRequestException("Not enough live nodes to 
support this keyspace");
+        int totalNodes = Gossiper.instance.getLiveMembers().size() + 
Gossiper.instance.getUnreachableMembers().size();
+        if (totalNodes < ks_def.replication_factor)
+            throw new InvalidRequestException(String.format("%s live nodes are 
not enough to support replication factor %s",
+                                                            totalNodes, 
ks_def.replication_factor));
         if (DatabaseDescriptor.getTableDefinition(ks_def.name) == null)
             throw new InvalidRequestException("Keyspace does not exist.");
         


Reply via email to