You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by gd...@apache.org on 2010/07/14 14:10:50 UTC

svn commit: r964020 - in /cassandra/trunk: src/java/org/apache/cassandra/dht/BootStrapper.java test/unit/org/apache/cassandra/dht/BootStrapperTest.java

Author: gdusbabek
Date: Wed Jul 14 12:10:50 2010
New Revision: 964020

URL: http://svn.apache.org/viewvc?rev=964020&view=rev
Log:
fail bootstrap if all nodes are bootstrapping. patch by gdusbabek, reviewed by jbellis. CASSANDRA-1011

Modified:
    cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java
    cassandra/trunk/test/unit/org/apache/cassandra/dht/BootStrapperTest.java

Modified: cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java?rev=964020&r1=964019&r2=964020&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java Wed Jul 14 12:10:50 2010
@@ -138,6 +138,9 @@ public class BootStrapper
 
         InetAddress maxEndpoint = endpoints.get(endpoints.size() - 1);
         assert !maxEndpoint.equals(FBUtilities.getLocalAddress());
+        if (metadata.pendingRangeChanges(maxEndpoint) > 0)
+            throw new RuntimeException("Every node is a bootstrap source! Please specify an initial token manually or wait for an existing bootstrap operation to finish.");
+        
         return maxEndpoint;
     }
 

Modified: cassandra/trunk/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/dht/BootStrapperTest.java?rev=964020&r1=964019&r2=964020&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/dht/BootStrapperTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/dht/BootStrapperTest.java Wed Jul 14 12:10:50 2010
@@ -48,6 +48,63 @@ public class BootStrapperTest extends Cl
         // fetch a bootstrap token from the local node
         assert BootStrapper.getBootstrapTokenFrom(FBUtilities.getLocalAddress()) != null;
     }
+    
+    @Test
+    public void testMulitipleAutomaticBootstraps() throws IOException
+    {
+        StorageService ss = StorageService.instance;
+        generateFakeEndpoints(5);
+        InetAddress[] addrs = new InetAddress[] 
+        {
+            InetAddress.getByName("127.0.0.2"),  
+            InetAddress.getByName("127.0.0.3"),  
+            InetAddress.getByName("127.0.0.4"),  
+            InetAddress.getByName("127.0.0.5"),  
+        };
+        InetAddress[] bootstrapAddrs = new InetAddress[]
+        {
+            InetAddress.getByName("127.0.0.12"),  
+            InetAddress.getByName("127.0.0.13"),  
+            InetAddress.getByName("127.0.0.14"),  
+            InetAddress.getByName("127.0.0.15"),  
+        };
+        Map<InetAddress, Double> load = new HashMap<InetAddress, Double>();
+        for (int i = 0; i < addrs.length; i++)
+            load.put(addrs[i], (double)i+2);
+        
+        // give every node a bootstrap source.
+        for (int i = 3; i >=0; i--)
+        {
+            InetAddress bootstrapSource = BootStrapper.getBootstrapSource(ss.getTokenMetadata(), load);
+            assert bootstrapSource != null;
+            assert bootstrapSource.equals(addrs[i]) : String.format("expected %s but got %s for %d", addrs[i], bootstrapSource, i);
+            assert !ss.getTokenMetadata().getBootstrapTokens().containsValue(bootstrapSource);
+            
+            Range range = ss.getPrimaryRangeForEndpoint(bootstrapSource);
+            Token token = StorageService.getPartitioner().midpoint(range.left, range.right);
+            assert range.contains(token);
+            ss.onChange(bootstrapAddrs[i], StorageService.MOVE_STATE, new ApplicationState(StorageService.STATE_BOOTSTRAPPING + StorageService.Delimiter + StorageService.getPartitioner().getTokenFactory().toString(token)));
+        }
+        
+        // any further attempt to bootsrtap should fail since every node in the cluster is splitting.
+        try
+        {
+            BootStrapper.getBootstrapSource(ss.getTokenMetadata(), load);
+            throw new AssertionError("This bootstrap should have failed.");
+        }
+        catch (RuntimeException ex) 
+        {
+            // success!
+        }
+        
+        // indicate that one of the nodes is done. see if the node it was bootstrapping from is still available.
+        Range range = ss.getPrimaryRangeForEndpoint(addrs[2]);
+        Token token = StorageService.getPartitioner().midpoint(range.left, range.right);
+        ss.onChange(bootstrapAddrs[2], StorageService.MOVE_STATE, new ApplicationState(StorageService.STATE_NORMAL + StorageService.Delimiter + StorageService.getPartitioner().getTokenFactory().toString(token)));
+        load.put(bootstrapAddrs[2], 0d);
+        InetAddress addr = BootStrapper.getBootstrapSource(ss.getTokenMetadata(), load);
+        assert addr != null && addr.equals(addrs[2]);
+    }
 
     @Test
     public void testGuessToken() throws IOException