You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by br...@apache.org on 2011/07/14 23:24:12 UTC

svn commit: r1146900 - in /cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra: locator/ service/

Author: brandonwilliams
Date: Thu Jul 14 21:24:11 2011
New Revision: 1146900

URL: http://svn.apache.org/viewvc?rev=1146900&view=rev
Log:
Allow RF to exceed the number of nodes (but disallow writes)
Patch by brandonwilliams, reviewed by Pavel Yaskevich for CASSANDRA-2129

Modified:
    cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
    cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
    cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java
    cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/SimpleStrategy.java
    cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/service/WriteResponseHandler.java

Modified: cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java?rev=1146900&r1=1146899&r2=1146900&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java (original)
+++ cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java Thu Jul 14 21:24:11 2011
@@ -87,9 +87,8 @@ public abstract class AbstractReplicatio
      * we return a List to avoid an extra allocation when sorting by proximity later
      * @param searchToken the token the natural endpoints are requested for
      * @return a copy of the natural endpoints for the given token
-     * @throws IllegalStateException if the number of requested replicas is greater than the number of known endpoints
      */
-    public ArrayList<InetAddress> getNaturalEndpoints(Token searchToken) throws IllegalStateException
+    public ArrayList<InetAddress> getNaturalEndpoints(Token searchToken)
     {
         Token keyToken = TokenMetadata.firstToken(tokenMetadata.sortedTokens(), searchToken);
         ArrayList<InetAddress> endpoints = getCachedEndpoints(keyToken);
@@ -99,10 +98,6 @@ public abstract class AbstractReplicatio
             keyToken = TokenMetadata.firstToken(tokenMetadataClone.sortedTokens(), searchToken);
             endpoints = new ArrayList<InetAddress>(calculateNaturalEndpoints(searchToken, tokenMetadataClone));
             cacheEndpoint(keyToken, endpoints);
-            // calculateNaturalEndpoints should have checked this already, this is a safety
-            assert getReplicationFactor() <= endpoints.size() : String.format("endpoints %s generated for RF of %s",
-                                                                              Arrays.toString(endpoints.toArray()),
-                                                                              getReplicationFactor());
         }
 
         return new ArrayList<InetAddress>(endpoints);
@@ -115,9 +110,8 @@ public abstract class AbstractReplicatio
      *
      * @param searchToken the token the natural endpoints are requested for
      * @return a copy of the natural endpoints for the given token
-     * @throws IllegalStateException if the number of requested replicas is greater than the number of known endpoints
      */
-    public abstract List<InetAddress> calculateNaturalEndpoints(Token searchToken, TokenMetadata tokenMetadata) throws IllegalStateException;
+    public abstract List<InetAddress> calculateNaturalEndpoints(Token searchToken, TokenMetadata tokenMetadata);
 
     public IWriteResponseHandler getWriteResponseHandler(Collection<InetAddress> writeEndpoints,
                                                          Multimap<InetAddress, InetAddress> hintedEndpoints,

Modified: cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java?rev=1146900&r1=1146899&r2=1146900&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java (original)
+++ cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java Thu Jul 14 21:24:11 2011
@@ -120,9 +120,6 @@ public class NetworkTopologyStrategy ext
                     dcEndpoints.add(endpoint);
             }
 
-            if (dcEndpoints.size() < dcReplicas)
-                throw new IllegalStateException(String.format("datacenter (%s) has no more endpoints, (%s) replicas still needed",
-                                                              dcName, dcReplicas - dcEndpoints.size()));
             if (logger.isDebugEnabled())
                 logger.debug("{} endpoints in datacenter {} for token {} ",
                              new Object[] { StringUtils.join(dcEndpoints, ","), dcName, searchToken});

Modified: cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java?rev=1146900&r1=1146899&r2=1146900&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java (original)
+++ cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java Thu Jul 14 21:24:11 2011
@@ -96,9 +96,6 @@ public class OldNetworkTopologyStrategy 
                 if (!endpoints.contains(metadata.getEndpoint(t)))
                     endpoints.add(metadata.getEndpoint(t));
             }
-
-            if (endpoints.size() < replicas)
-                throw new IllegalStateException(String.format("replication factor (%s) exceeds number of endpoints (%s)", replicas, endpoints.size()));
         }
 
         return endpoints;

Modified: cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/SimpleStrategy.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/SimpleStrategy.java?rev=1146900&r1=1146899&r2=1146900&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/SimpleStrategy.java (original)
+++ cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/SimpleStrategy.java Thu Jul 14 21:24:11 2011
@@ -56,10 +56,6 @@ public class SimpleStrategy extends Abst
         {
             endpoints.add(metadata.getEndpoint(iter.next()));
         }
-
-        if (endpoints.size() < replicas)
-            throw new IllegalStateException(String.format("replication factor (%s) exceeds number of endpoints (%s)", replicas, endpoints.size()));
-        
         return endpoints;
     }
 

Modified: cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/service/WriteResponseHandler.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/service/WriteResponseHandler.java?rev=1146900&r1=1146899&r2=1146900&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/service/WriteResponseHandler.java (original)
+++ cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/service/WriteResponseHandler.java Thu Jul 14 21:24:11 2011
@@ -85,9 +85,9 @@ public class WriteResponseHandler extend
             case THREE:
                 return 3;
             case QUORUM:
-                return (writeEndpoints.size() / 2) + 1;
+                return (Table.open(table).getReplicationStrategy().getReplicationFactor() / 2) + 1;
             case ALL:
-                return writeEndpoints.size();
+                return Table.open(table).getReplicationStrategy().getReplicationFactor();
             default:
                 throw new UnsupportedOperationException("invalid consistency level: " + consistencyLevel.toString());
         }