Author: jbellis
Date: Tue Sep 20 11:59:45 2011
New Revision: 1173099

URL: http://svn.apache.org/viewvc?rev=1173099&view=rev
Log:
update comments

Modified:
    
cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/locator/TokenMetadata.java

Modified: 
cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/locator/TokenMetadata.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/locator/TokenMetadata.java?rev=1173099&r1=1173098&r2=1173099&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/locator/TokenMetadata.java
 (original)
+++ 
cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/locator/TokenMetadata.java
 Tue Sep 20 11:59:45 2011
@@ -43,22 +43,34 @@ public class TokenMetadata
     /* Maintains token to endpoint map of every node in the cluster. */
     private BiMap<Token, InetAddress> tokenToEndpointMap;
 
-    // Suppose that there is a ring of nodes A, C and E, with replication 
factor 3.
+    // Prior to CASSANDRA-603, we just had <tt>Map<Range, InetAddress> 
pendingRanges<tt>,
+    // which was added to when a node began bootstrap and removed from when it 
finished.
+    //
+    // This is inadequate when multiple changes are allowed simultaneously.  
For example,
+    // suppose that there is a ring of nodes A, C and E, with replication 
factor 3.
     // Node D bootstraps between C and E, so its pending ranges will be E-A, 
A-C and C-D.
-    // Now suppose node B bootstraps between A and C at the same time. Its 
pending ranges would be C-E, E-A and A-B.
-    // Now both nodes have pending range E-A in their list, which will cause 
pending range collision
-    // even though we're only talking about replica range, not even primary 
range. The same thing happens
-    // for any nodes that boot simultaneously between same two nodes. For this 
we cannot simply make pending ranges a <tt>Multimap</tt>,
-    // since that would make us unable to notice the real problem of two nodes 
trying to boot using the same token.
-    // In order to do this properly, we need to know what tokens are booting 
at any time.
+    // Now suppose node B bootstraps between A and C at the same time. Its 
pending ranges
+    // would be C-E, E-A and A-B. Now both nodes need to be assigned pending 
range E-A,
+    // which we would be unable to represent with the old Map.  The same thing 
happens
+    // even more obviously for any nodes that boot simultaneously between same 
two nodes.
+    //
+    // So, we made two changes:
+    //
+    // First, we changed pendingRanges to a <tt>Multimap<Range, 
InetAddress></tt> (now
+    // <tt>Map<String, Multimap<Range, InetAddress>></tt>, because replication 
strategy
+    // and options are per-KeySpace).
+    //
+    // Second, we added the bootstrapTokens and leavingEndpoints collections, 
so we can
+    // rebuild pendingRanges from the complete information of what is going 
on, when
+    // additional changes are made mid-operation.
+    //
+    // Finally, note that recording the tokens of joining nodes in 
bootstrapTokens also
+    // means we can detect and reject the addition of multiple nodes at the 
same token
+    // before one becomes part of the ring.
     private BiMap<Token, InetAddress> bootstrapTokens = HashBiMap.create();
-
-    // we will need to know at all times what nodes are leaving and calculate 
ranges accordingly.
-    // An anonymous pending ranges list is not enough, as that does not tell 
which node is leaving
-    // and/or if the ranges are there because of bootstrap or leave operation.
-    // (See CASSANDRA-603 for more detail + examples).
+    // (don't need to record Token here since it's still part of 
tokenToEndpointMap until it's done leaving)
     private Set<InetAddress> leavingEndpoints = new HashSet<InetAddress>();
-
+    // this is a cache of the calculation from {tokenToEndpointMap, 
bootstrapTokens, leavingEndpoints}
     private ConcurrentMap<String, Multimap<Range, InetAddress>> pendingRanges 
= new ConcurrentHashMap<String, Multimap<Range, InetAddress>>();
 
     // nodes which are migrating to the new tokens in the ring


Reply via email to