cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jbel...@apache.org
Subject svn commit: r1065665 - in /cassandra/branches/cassandra-0.7: src/java/org/apache/cassandra/locator/ src/java/org/apache/cassandra/service/ test/unit/org/apache/cassandra/dht/
Date Mon, 31 Jan 2011 16:13:20 GMT
Author: jbellis
Date: Mon Jan 31 16:13:20 2011
New Revision: 1065665

URL: http://svn.apache.org/viewvc?rev=1065665&view=rev
Log:
convert SS.partitioner, valueFactory to instance fields

Modified:
    cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/locator/Ec2Snitch.java
    cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/MigrationManager.java
    cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/StorageLoadBalancer.java
    cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/StorageService.java
    cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/dht/BootStrapperTest.java

Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/locator/Ec2Snitch.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/locator/Ec2Snitch.java?rev=1065665&r1=1065664&r2=1065665&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/locator/Ec2Snitch.java
(original)
+++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/locator/Ec2Snitch.java
Mon Jan 31 16:13:20 2011
@@ -89,7 +89,7 @@ public class Ec2Snitch extends AbstractN
     {
         // Share EC2 info via gossip.  We have to wait until Gossiper is initialized though.
         logger.info("Ec2Snitch adding ApplicationState ec2region=" + ec2region + " ec2zone="
+ ec2zone);
-        Gossiper.instance.addLocalApplicationState(ApplicationState.DC, StorageService.valueFactory.datacenter(ec2region));
-        Gossiper.instance.addLocalApplicationState(ApplicationState.RACK, StorageService.valueFactory.rack(ec2zone));
+        Gossiper.instance.addLocalApplicationState(ApplicationState.DC, StorageService.instance.valueFactory.datacenter(ec2region));
+        Gossiper.instance.addLocalApplicationState(ApplicationState.RACK, StorageService.instance.valueFactory.rack(ec2zone));
     }
 }

Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/MigrationManager.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/MigrationManager.java?rev=1065665&r1=1065664&r2=1065665&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/MigrationManager.java
(original)
+++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/MigrationManager.java
Mon Jan 31 16:13:20 2011
@@ -97,7 +97,7 @@ public class MigrationManager implements
             MessagingService.instance().sendOneWay(msg, host);
         // this is for notifying nodes as they arrive in the cluster.
         if (!StorageService.instance.isClientMode())
-            Gossiper.instance.addLocalApplicationState(ApplicationState.SCHEMA, StorageService.valueFactory.migration(version));
+            Gossiper.instance.addLocalApplicationState(ApplicationState.SCHEMA, StorageService.instance.valueFactory.migration(version));
     }
 
     /**

Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/StorageLoadBalancer.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/StorageLoadBalancer.java?rev=1065665&r1=1065664&r2=1065665&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/StorageLoadBalancer.java
(original)
+++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/StorageLoadBalancer.java
Mon Jan 31 16:13:20 2011
@@ -348,7 +348,7 @@ public class StorageLoadBalancer impleme
                 if (logger_.isDebugEnabled())
                     logger_.debug("Disseminating load info ...");
                 Gossiper.instance.addLocalApplicationState(ApplicationState.LOAD,
-                                                           StorageService.valueFactory.load(StorageService.instance.getLoad()));
+                                                           StorageService.instance.valueFactory.load(StorageService.instance.getLoad()));
             }
         };
         StorageService.scheduledTasks.scheduleWithFixedDelay(runnable, 2 * Gossiper.intervalInMillis_,
BROADCAST_INTERVAL, TimeUnit.MILLISECONDS);

Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/StorageService.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/StorageService.java?rev=1065665&r1=1065664&r2=1065665&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/StorageService.java
(original)
+++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/StorageService.java
Mon Jan 31 16:13:20 2011
@@ -27,7 +27,6 @@ import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.*;
 import javax.management.MBeanServer;
-import javax.management.MalformedObjectNameException;
 import javax.management.ObjectName;
 
 import com.google.common.base.Charsets;
@@ -152,13 +151,14 @@ public class StorageService implements I
     /* This abstraction maintains the token/endpoint metadata information */
     private TokenMetadata tokenMetadata_ = new TokenMetadata();
 
-    private static IPartitioner partitioner_ = DatabaseDescriptor.getPartitioner();
-    public static VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(partitioner_);
+    private IPartitioner partitioner = DatabaseDescriptor.getPartitioner();
+    public VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(partitioner);
     
     public static final StorageService instance = new StorageService();
 
-    public static IPartitioner getPartitioner() {
-        return partitioner_;
+    public static IPartitioner getPartitioner()
+    {
+        return instance.partitioner;
     }
 
     public Collection<Range> getLocalRanges(String table)
@@ -458,12 +458,12 @@ public class StorageService implements I
                 String initialToken = DatabaseDescriptor.getInitialToken();
                 if (initialToken == null)
                 {
-                    token = partitioner_.getRandomToken();
+                    token = partitioner.getRandomToken();
                     logger_.warn("Generated random token " + token + ". Random tokens will
result in an unbalanced ring; see http://wiki.apache.org/cassandra/Operations");
                 }
                 else
                 {
-                    token = partitioner_.getTokenFactory().fromString(initialToken);
+                    token = partitioner.getTokenFactory().fromString(initialToken);
                     logger_.info("Saved token not found. Using " + token + " from configuration");
                 }
             }
@@ -1444,7 +1444,7 @@ public class StorageService implements I
      */
     public List<InetAddress> getNaturalEndpoints(String table, ByteBuffer key)
     {
-        return getNaturalEndpoints(table, partitioner_.getToken(key));
+        return getNaturalEndpoints(table, partitioner.getToken(key));
     }
 
     public List<InetAddress> getNaturalEndpoints(String table, byte[] key)
@@ -1473,7 +1473,7 @@ public class StorageService implements I
      */
     public List<InetAddress> getLiveNaturalEndpoints(String table, ByteBuffer key)
     {
-        return getLiveNaturalEndpoints(table, partitioner_.getToken(key));
+        return getLiveNaturalEndpoints(table, partitioner.getToken(key));
     }
 
     public List<InetAddress> getLiveNaturalEndpoints(String table, Token token)
@@ -1546,7 +1546,7 @@ public class StorageService implements I
         FBUtilities.sortSampledKeys(keys, range);
 
         if (keys.size() < 3)
-            return partitioner_.midpoint(range.left, range.right);
+            return partitioner.midpoint(range.left, range.right);
         else
             return keys.get(keys.size() / 2).token;
     }
@@ -1670,7 +1670,7 @@ public class StorageService implements I
 
     public void move(String newToken) throws IOException, InterruptedException
     {
-        move(partitioner_.getTokenFactory().fromString(newToken));
+        move(partitioner.getTokenFactory().fromString(newToken));
     }
 
     public void loadBalance() throws IOException, InterruptedException
@@ -1760,7 +1760,7 @@ public class StorageService implements I
     {
         InetAddress myAddress = FBUtilities.getLocalAddress();
         Token localToken = tokenMetadata_.getToken(myAddress);
-        Token token = partitioner_.getTokenFactory().fromString(tokenString);
+        Token token = partitioner.getTokenFactory().fromString(tokenString);
         InetAddress endpoint = tokenMetadata_.getEndpoint(token);
 
         if (endpoint == null)
@@ -2074,9 +2074,9 @@ public class StorageService implements I
     // Never ever do this at home. Used by tests.
     IPartitioner setPartitionerUnsafe(IPartitioner newPartitioner)
     {
-        IPartitioner oldPartitioner = partitioner_;
-        partitioner_ = newPartitioner;
-        valueFactory = new VersionedValue.VersionedValueFactory(partitioner_);
+        IPartitioner oldPartitioner = partitioner;
+        partitioner = newPartitioner;
+        valueFactory = new VersionedValue.VersionedValueFactory(partitioner);
         return oldPartitioner;
     }
 
@@ -2109,7 +2109,7 @@ public class StorageService implements I
     {
         List<Token> sortedTokens = new ArrayList<Token>(getTokenToEndpointMap().keySet());
         Collections.sort(sortedTokens);
-        return partitioner_.describeOwnership(sortedTokens);
+        return partitioner.describeOwnership(sortedTokens);
     }
 
     public List<String> getKeyspaces()

Modified: cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/dht/BootStrapperTest.java?rev=1065665&r1=1065664&r2=1065665&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
(original)
+++ cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
Mon Jan 31 16:13:20 2011
@@ -85,7 +85,7 @@ public class BootStrapperTest extends Cl
             Range range = ss.getPrimaryRangeForEndpoint(bootstrapSource);
             Token token = StorageService.getPartitioner().midpoint(range.left, range.right);
             assert range.contains(token);
-            ss.onChange(bootstrapAddrs[i], ApplicationState.STATUS, StorageService.valueFactory.bootstrapping(token));
+            ss.onChange(bootstrapAddrs[i], ApplicationState.STATUS, StorageService.instance.valueFactory.bootstrapping(token));
         }
         
         // any further attempt to bootsrtap should fail since every node in the cluster is
splitting.
@@ -102,7 +102,7 @@ public class BootStrapperTest extends Cl
         // 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], ApplicationState.STATUS, StorageService.valueFactory.normal(token));
+        ss.onChange(bootstrapAddrs[2], ApplicationState.STATUS, StorageService.instance.valueFactory.normal(token));
         load.put(bootstrapAddrs[2], 0d);
         InetAddress addr = BootStrapper.getBootstrapSource(ss.getTokenMetadata(), load);
         assert addr != null && addr.equals(addrs[2]);
@@ -134,7 +134,7 @@ public class BootStrapperTest extends Cl
         Range range5 = ss.getPrimaryRangeForEndpoint(five);
         Token fakeToken = StorageService.getPartitioner().midpoint(range5.left, range5.right);
         assert range5.contains(fakeToken);
-        ss.onChange(myEndpoint, ApplicationState.STATUS, StorageService.valueFactory.bootstrapping(fakeToken));
+        ss.onChange(myEndpoint, ApplicationState.STATUS, StorageService.instance.valueFactory.bootstrapping(fakeToken));
         tmd = ss.getTokenMetadata();
 
         InetAddress source4 = BootStrapper.getBootstrapSource(tmd, load);



Mime
View raw message