cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jbel...@apache.org
Subject svn commit: r806015 - in /incubator/cassandra/trunk/src/java/org/apache/cassandra: db/SystemTable.java service/StorageService.java
Date Thu, 20 Aug 2009 01:03:20 GMT
Author: jbellis
Date: Thu Aug 20 01:03:20 2009
New Revision: 806015

URL: http://svn.apache.org/viewvc?rev=806015&view=rev
Log:
rename underscores away
patch by jbellis; reviewed by Sammy Yu for CASSANDRA-364

Modified:
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/SystemTable.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/SystemTable.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/SystemTable.java?rev=806015&r1=806014&r2=806015&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/SystemTable.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/SystemTable.java Thu Aug 20
01:03:20 2009
@@ -38,12 +38,12 @@
 
 public class SystemTable
 {
-    private static Logger logger_ = Logger.getLogger(SystemTable.class);
+    private static Logger logger = Logger.getLogger(SystemTable.class);
     public static final String LOCATION_CF = "LocationInfo";
     private static final String LOCATION_KEY = "L"; // only one row in Location CF
     private static final byte[] TOKEN = utf8("Token");
     private static final byte[] GENERATION = utf8("Generation");
-    private static StorageMetadata metadata_;
+    private static StorageMetadata metadata;
 
     private static byte[] utf8(String str)
     {
@@ -62,7 +62,7 @@
     */
     public static synchronized void updateToken(Token token) throws IOException
     {
-        assert metadata_ != null;
+        assert metadata != null;
         IPartitioner p = StorageService.getPartitioner();
         Table table = Table.open(Table.SYSTEM_TABLE);
         /* Retrieve the "LocationInfo" column family */
@@ -72,13 +72,13 @@
         /* create the "Token" whose value is the new token. */
         IColumn tokenColumn = new Column(SystemTable.TOKEN, p.getTokenFactory().toByteArray(token),
oldTokenColumnTimestamp + 1);
         /* replace the old "Token" column with this new one. */
-        if (logger_.isDebugEnabled())
-          logger_.debug("Replacing old token " + p.getTokenFactory().fromByteArray(cf.getColumn(SystemTable.TOKEN).value())
+ " with " + token);
+        if (logger.isDebugEnabled())
+          logger.debug("Replacing old token " + p.getTokenFactory().fromByteArray(cf.getColumn(SystemTable.TOKEN).value())
+ " with " + token);
         RowMutation rm = new RowMutation(Table.SYSTEM_TABLE, LOCATION_KEY);
         cf.addColumn(tokenColumn);
         rm.add(cf);
         rm.apply();
-        metadata_.setStorageId(token);
+        metadata.setToken(token);
     }
     
     /*
@@ -91,8 +91,8 @@
     */
     public static synchronized StorageMetadata initMetadata() throws IOException
     {
-        if (metadata_ != null)  // guard to protect against being called twice
-            return metadata_;
+        if (metadata != null)  // guard to protect against being called twice
+            return metadata;
 
         /* Read the system table to retrieve the storage ID and the generation */
         Table table = Table.open(Table.SYSTEM_TABLE);
@@ -111,8 +111,8 @@
             cf.addColumn(new Column(GENERATION, BasicUtilities.intToByteArray(generation))
);
             rm.add(cf);
             rm.apply();
-            metadata_ = new StorageMetadata(token, generation);
-            return metadata_;
+            metadata = new StorageMetadata(token, generation);
+            return metadata;
         }
 
         /* we crashed and came back up need to bump generation # */
@@ -128,34 +128,34 @@
         cf.addColumn(generation2);
         rm.add(cf);
         rm.apply();
-        metadata_ = new StorageMetadata(token, gen);
-        return metadata_;
+        metadata = new StorageMetadata(token, gen);
+        return metadata;
     }
 
     public static class StorageMetadata
     {
-        private Token myToken;
-        private int generation_;
+        private Token token;
+        private int generation;
 
         StorageMetadata(Token storageId, int generation)
         {
-            myToken = storageId;
-            generation_ = generation;
+            token = storageId;
+            this.generation = generation;
         }
 
-        public Token getStorageId()
+        public Token getToken()
         {
-            return myToken;
+            return token;
         }
 
-        public void setStorageId(Token storageId)
+        public void setToken(Token storageId)
         {
-            myToken = storageId;
+            token = storageId;
         }
 
         public int getGeneration()
         {
-            return generation_;
+            return generation;
         }
     }
 }

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java?rev=806015&r1=806014&r2=806015&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java Thu
Aug 20 01:03:20 2009
@@ -22,7 +22,6 @@
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.net.InetAddress;
-import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.util.*;
 import java.util.concurrent.ExecutorService;
@@ -207,7 +206,7 @@
         if (bootstrapSet.isEmpty())
         {
             isBootstrapMode = false;
-            tokenMetadata_.update(storageMetadata_.getStorageId(), StorageService.tcpAddr_,
false);
+            tokenMetadata_.update(storageMetadata_.getToken(), StorageService.tcpAddr_, false);
 
             logger_.info("Bootstrap completed! Now serving reads.");
             /* Tell others you're not bootstrapping anymore */
@@ -306,8 +305,8 @@
          */
         Gossiper.instance().start(udpAddr_, storageMetadata_.getGeneration());
         /* Make sure this token gets gossiped around. */
-        tokenMetadata_.update(storageMetadata_.getStorageId(), StorageService.tcpAddr_, isBootstrapMode);
-        ApplicationState state = new ApplicationState(StorageService.getPartitioner().getTokenFactory().toString(storageMetadata_.getStorageId()));
+        tokenMetadata_.update(storageMetadata_.getToken(), StorageService.tcpAddr_, isBootstrapMode);
+        ApplicationState state = new ApplicationState(StorageService.getPartitioner().getTokenFactory().toString(storageMetadata_.getToken()));
         Gossiper.instance().addApplicationState(StorageService.nodeId_, state);
         if (isBootstrapMode)
         {



Mime
View raw message