Author: jbellis
Date: Tue Mar  2 21:23:23 2010
New Revision: 918187

URL: http://svn.apache.org/viewvc?rev=918187&view=rev
Log:
fix consistencylevel during bootstrap, partially, by hinting "down" 
bootstrapping nodes and just expanding target set to include them in blockFor 
calculation.  patch by jbellis; reviewed by Ryan King for CASSANDRA-822

Modified:
    
incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
    
incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/StorageProxy.java

Modified: 
incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java?rev=918187&r1=918186&r2=918187&view=diff
==============================================================================
--- 
incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
 (original)
+++ 
incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
 Tue Mar  2 21:23:23 2010
@@ -89,10 +89,6 @@
             }
             else
             {
-                // Ignore targets that have died when bootstrapping
-                if (!tokenMetadata_.isMember(ep))
-                    continue;
-
                 // find another endpoint to store a hint on.  prefer endpoints 
that aren't already in use
                 InetAddress hintLocation = null;
                 List<InetAddress> preferred = 
endPointSnitch.getSortedListByProximity(ep, liveNodes);

Modified: 
incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/StorageProxy.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/StorageProxy.java?rev=918187&r1=918186&r2=918187&view=diff
==============================================================================
--- 
incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/StorageProxy.java
 (original)
+++ 
incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/StorageProxy.java
 Tue Mar  2 21:23:23 2010
@@ -183,7 +183,7 @@
                 mostRecentRowMutation = rm;
                 List<InetAddress> naturalEndpoints = 
StorageService.instance.getNaturalEndpoints(rm.getTable(), rm.key());
                 Map<InetAddress, InetAddress> endpointMap = 
StorageService.instance.getHintedEndpointMap(rm.getTable(), rm.key(), 
naturalEndpoints);
-                int blockFor = determineBlockFor(naturalEndpoints.size(), 
endpointMap.size(), consistency_level);
+                int blockFor = determineBlockFor(endpointMap.size(), 
consistency_level);
                 
                 // avoid starting a write we know can't achieve the required 
consistency
                 assureSufficientLiveNodes(endpointMap, blockFor, 
consistency_level);
@@ -292,40 +292,25 @@
         StageManager.getStage(StageManager.MUTATION_STAGE).execute(runnable);
     }
 
-    private static int determineBlockFor(int naturalTargets, int 
hintedTargets, ConsistencyLevel consistency_level)
+    private static int determineBlockFor(int expandedTargets, ConsistencyLevel 
consistency_level)
     {
-        assert naturalTargets >= 1;
-        assert hintedTargets >= naturalTargets;
-
-        int bootstrapTargets = hintedTargets - naturalTargets;
-        int blockFor;
-        if (consistency_level == ConsistencyLevel.ONE)
-        {
-            blockFor = 1 + bootstrapTargets;
-        }
-        else if (consistency_level == ConsistencyLevel.QUORUM)
-        {
-            blockFor = (naturalTargets / 2) + 1 + bootstrapTargets;
-        }
-        else if (consistency_level == ConsistencyLevel.DCQUORUM || 
consistency_level == ConsistencyLevel.DCQUORUMSYNC)
-        {
-            // TODO this is broken
-            blockFor = naturalTargets;
-        }
-        else if (consistency_level == ConsistencyLevel.ALL)
-        {
-            blockFor = naturalTargets + bootstrapTargets;
-        }
-        else if (consistency_level == ConsistencyLevel.ANY)
+        switch (consistency_level)
         {
-            blockFor = 1;
+            case ONE:
+            case ANY:
+                return 1;
+            case QUORUM:
+                return (expandedTargets / 2) + 1;
+            case DCQUORUM:
+            case DCQUORUMSYNC:
+                // TODO this is broken
+                return expandedTargets;
+            case ALL:
+                return expandedTargets;
+            default:
+                throw new UnsupportedOperationException("invalid consistency 
level " + consistency_level);
         }
-        else
-        {
-            throw new UnsupportedOperationException("invalid consistency level 
" + consistency_level);
-        }
-        return blockFor;
-    }    
+    }
 
     /**
      * Read the data from one replica.  When we get
@@ -443,7 +428,7 @@
             InetAddress dataPoint = 
StorageService.instance.findSuitableEndPoint(command.table, command.key);
             List<InetAddress> endpointList = 
StorageService.instance.getLiveNaturalEndpoints(command.table, command.key);
             final String table = command.table;
-            int responseCount = 
determineBlockFor(DatabaseDescriptor.getReplicationFactor(table), 
DatabaseDescriptor.getReplicationFactor(table), consistency_level);
+            int responseCount = 
determineBlockFor(DatabaseDescriptor.getReplicationFactor(table), 
consistency_level);
             if (endpointList.size() < responseCount)
                 throw new UnavailableException();
 
@@ -547,7 +532,7 @@
         long startTime = System.nanoTime();
 
         final String table = command.keyspace;
-        int responseCount = 
determineBlockFor(DatabaseDescriptor.getReplicationFactor(table), 
DatabaseDescriptor.getReplicationFactor(table), consistency_level);
+        int responseCount = 
determineBlockFor(DatabaseDescriptor.getReplicationFactor(table), 
consistency_level);
 
         List<Pair<AbstractBounds, List<InetAddress>>> ranges = 
getRestrictedRanges(command.range, command.keyspace, responseCount);
 


Reply via email to