cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From slebre...@apache.org
Subject [20/28] [3649] Remove underscores on variables
Date Mon, 27 Feb 2012 10:16:00 GMT
http://git-wip-us.apache.org/repos/asf/cassandra/blob/910b663b/src/java/org/apache/cassandra/net/MessagingService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/MessagingService.java b/src/java/org/apache/cassandra/net/MessagingService.java
index de9eeaa..7fce8e6 100644
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@ -73,9 +73,9 @@ public final class MessagingService implements MessagingServiceMBean
     public static final int VERSION_080 = 2;
     public static final int VERSION_10 = 3;
     public static final int VERSION_11 = 4;
-    public static final int version_ = VERSION_11;
+    public static final int current_version = VERSION_11;
 
-    static SerializerType serializerType_ = SerializerType.BINARY;
+    static SerializerType serializerType = SerializerType.BINARY;
 
     /** we preface every message with this number so the recipient can validate the sender is sane */
     static final int PROTOCOL_MAGIC = 0xCA552DFA;
@@ -84,7 +84,7 @@ public final class MessagingService implements MessagingServiceMBean
     private final ExpiringMap<String, CallbackInfo> callbacks;
 
     /* Lookup table for registering message handlers based on the verb. */
-    private final Map<StorageService.Verb, IVerbHandler> verbHandlers_;
+    private final Map<StorageService.Verb, IVerbHandler> verbHandlers;
 
     /** One executor per destination InetAddress for streaming.
      *
@@ -101,9 +101,9 @@ public final class MessagingService implements MessagingServiceMBean
     private final ConcurrentMap<InetAddress, DebuggableThreadPoolExecutor> streamExecutors = new NonBlockingHashMap<InetAddress, DebuggableThreadPoolExecutor>();
     private final AtomicInteger activeStreamsOutbound = new AtomicInteger(0);
 
-    private final NonBlockingHashMap<InetAddress, OutboundTcpConnectionPool> connectionManagers_ = new NonBlockingHashMap<InetAddress, OutboundTcpConnectionPool>();
+    private final NonBlockingHashMap<InetAddress, OutboundTcpConnectionPool> connectionManagers = new NonBlockingHashMap<InetAddress, OutboundTcpConnectionPool>();
 
-    private static final Logger logger_ = LoggerFactory.getLogger(MessagingService.class);
+    private static final Logger logger = LoggerFactory.getLogger(MessagingService.class);
     private static final int LOG_DROPPED_INTERVAL_IN_MS = 5000;
 
     private List<SocketThread> socketThreads = Lists.newArrayList();
@@ -153,7 +153,7 @@ public final class MessagingService implements MessagingServiceMBean
         }
 
         listenGate = new SimpleCondition();
-        verbHandlers_ = new EnumMap<StorageService.Verb, IVerbHandler>(StorageService.Verb.class);
+        verbHandlers = new EnumMap<StorageService.Verb, IVerbHandler>(StorageService.Verb.class);
         Runnable logDropped = new Runnable()
         {
             public void run()
@@ -193,7 +193,7 @@ public final class MessagingService implements MessagingServiceMBean
                     }
                     catch (IOException e)
                     {
-                        logger_.error("Unable to deserialize mutation when writting hint for: " + expiredCallbackInfo.target);
+                        logger.error("Unable to deserialize mutation when writting hint for: " + expiredCallbackInfo.target);
                     }
                 }
 
@@ -235,7 +235,7 @@ public final class MessagingService implements MessagingServiceMBean
     /** called from gossiper when it notices a node is not responding. */
     public void convict(InetAddress ep)
     {
-        logger_.debug("Resetting pool for " + ep);
+        logger.debug("Resetting pool for " + ep);
         getConnectionPool(ep).reset();
     }
 
@@ -262,7 +262,7 @@ public final class MessagingService implements MessagingServiceMBean
         {
             ss.add(SSLFactory.getServerSocket(DatabaseDescriptor.getEncryptionOptions(), localEp, DatabaseDescriptor.getSSLStoragePort()));
             // setReuseAddress happens in the factory.
-            logger_.info("Starting Encrypted Messaging Service on SSL port {}", DatabaseDescriptor.getSSLStoragePort());
+            logger.info("Starting Encrypted Messaging Service on SSL port {}", DatabaseDescriptor.getSSLStoragePort());
         }
         
         ServerSocketChannel serverChannel = ServerSocketChannel.open();
@@ -283,7 +283,7 @@ public final class MessagingService implements MessagingServiceMBean
             else
                 throw e;
         }
-        logger_.info("Starting Messaging Service on port {}", DatabaseDescriptor.getStoragePort());
+        logger.info("Starting Messaging Service on port {}", DatabaseDescriptor.getStoragePort());
         ss.add(socket);
         return ss;
     }
@@ -296,17 +296,17 @@ public final class MessagingService implements MessagingServiceMBean
         }
         catch (InterruptedException ie)
         {
-            logger_.debug("await interrupted");
+            logger.debug("await interrupted");
         }
     }
 
     public OutboundTcpConnectionPool getConnectionPool(InetAddress to)
     {
-        OutboundTcpConnectionPool cp = connectionManagers_.get(to);
+        OutboundTcpConnectionPool cp = connectionManagers.get(to);
         if (cp == null)
         {
-            connectionManagers_.putIfAbsent(to, new OutboundTcpConnectionPool(to));
-            cp = connectionManagers_.get(to);
+            connectionManagers.putIfAbsent(to, new OutboundTcpConnectionPool(to));
+            cp = connectionManagers.get(to);
         }
         return cp;
     }
@@ -324,8 +324,8 @@ public final class MessagingService implements MessagingServiceMBean
      */
     public void registerVerbHandlers(StorageService.Verb verb, IVerbHandler verbHandler)
     {
-    	assert !verbHandlers_.containsKey(verb);
-    	verbHandlers_.put(verb, verbHandler);
+    	assert !verbHandlers.containsKey(verb);
+    	verbHandlers.put(verb, verbHandler);
     }
 
     /**
@@ -336,7 +336,7 @@ public final class MessagingService implements MessagingServiceMBean
      */
     public IVerbHandler getVerbHandler(StorageService.Verb type)
     {
-        return verbHandlers_.get(type);
+        return verbHandlers.get(type);
     }
 
     public String addCallback(IMessageCallback cb, Message message, InetAddress to)
@@ -432,8 +432,8 @@ public final class MessagingService implements MessagingServiceMBean
      */
     public void sendOneWay(Message message, String id, InetAddress to)
     {
-        if (logger_.isTraceEnabled())
-            logger_.trace(FBUtilities.getBroadcastAddress() + " sending " + message.getVerb() + " to " + id + "@" + to);
+        if (logger.isTraceEnabled())
+            logger.trace(FBUtilities.getBroadcastAddress() + " sending " + message.getVerb() + " to " + id + "@" + to);
 
         // do local deliveries
         if ( message.getFrom().equals(to) )
@@ -529,7 +529,7 @@ public final class MessagingService implements MessagingServiceMBean
         for (DebuggableThreadPoolExecutor e : streamExecutors.values())
         {
             if (e.awaitTermination(24, TimeUnit.HOURS))
-                logger_.error("Stream took more than 24H to complete; skipping");
+                logger.error("Stream took more than 24H to complete; skipping");
         }
     }
 
@@ -538,7 +538,7 @@ public final class MessagingService implements MessagingServiceMBean
      */
     public void shutdown()
     {
-        logger_.info("Waiting for messaging service to quiesce");
+        logger.info("Waiting for messaging service to quiesce");
         // We may need to schedule hints on the mutation stage, so it's erroneous to shut down the mutation stage first
         assert !StageManager.getStage(Stage.MUTATION).isShutdown();
 
@@ -559,8 +559,8 @@ public final class MessagingService implements MessagingServiceMBean
 
     public void receive(Message message, String id)
     {
-        if (logger_.isTraceEnabled())
-            logger_.trace(FBUtilities.getBroadcastAddress() + " received " + message.getVerb()
+        if (logger.isTraceEnabled())
+            logger.trace(FBUtilities.getBroadcastAddress() + " received " + message.getVerb()
                           + " from " + id + "@" + message.getFrom());
 
         message = SinkManager.processServerMessage(message, id);
@@ -608,7 +608,7 @@ public final class MessagingService implements MessagingServiceMBean
         */
         int header = 0;
         // Setting up the serializer bit
-        header |= serializerType_.ordinal();
+        header |= serializerType.ordinal();
         // set compression bit.
         if ( compress )
             header |= 4;
@@ -662,7 +662,7 @@ public final class MessagingService implements MessagingServiceMBean
             if (recent > 0)
             {
                 logTpstats = true;
-                logger_.info("{} {} messages dropped in last {}ms",
+                logger.info("{} {} messages dropped in last {}ms",
                              new Object[] {recent, verb, LOG_DROPPED_INTERVAL_IN_MS});
                 lastDroppedInternal.put(verb, dropped.get());
             }
@@ -694,7 +694,7 @@ public final class MessagingService implements MessagingServiceMBean
                 catch (AsynchronousCloseException e)
                 {
                     // this happens when another thread calls close().
-                    logger_.info("MessagingService shutting down server thread.");
+                    logger.info("MessagingService shutting down server thread.");
                     break;
                 }
                 catch (IOException e)
@@ -713,7 +713,7 @@ public final class MessagingService implements MessagingServiceMBean
     public Map<String, Integer> getCommandPendingTasks()
     {
         Map<String, Integer> pendingTasks = new HashMap<String, Integer>();
-        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers_.entrySet())
+        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
             pendingTasks.put(entry.getKey().getHostAddress(), entry.getValue().cmdCon.getPendingMessages());
         return pendingTasks;
     }
@@ -721,7 +721,7 @@ public final class MessagingService implements MessagingServiceMBean
     public Map<String, Long> getCommandCompletedTasks()
     {
         Map<String, Long> completedTasks = new HashMap<String, Long>();
-        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers_.entrySet())
+        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
             completedTasks.put(entry.getKey().getHostAddress(), entry.getValue().cmdCon.getCompletedMesssages());
         return completedTasks;
     }
@@ -729,7 +729,7 @@ public final class MessagingService implements MessagingServiceMBean
     public Map<String, Long> getCommandDroppedTasks()
     {
         Map<String, Long> droppedTasks = new HashMap<String, Long>();
-        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers_.entrySet())
+        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
             droppedTasks.put(entry.getKey().getHostAddress(), entry.getValue().cmdCon.getDroppedMessages());
         return droppedTasks;
     }
@@ -737,7 +737,7 @@ public final class MessagingService implements MessagingServiceMBean
     public Map<String, Integer> getResponsePendingTasks()
     {
         Map<String, Integer> pendingTasks = new HashMap<String, Integer>();
-        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers_.entrySet())
+        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
             pendingTasks.put(entry.getKey().getHostAddress(), entry.getValue().ackCon.getPendingMessages());
         return pendingTasks;
     }
@@ -745,7 +745,7 @@ public final class MessagingService implements MessagingServiceMBean
     public Map<String, Long> getResponseCompletedTasks()
     {
         Map<String, Long> completedTasks = new HashMap<String, Long>();
-        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers_.entrySet())
+        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
             completedTasks.put(entry.getKey().getHostAddress(), entry.getValue().ackCon.getCompletedMesssages());
         return completedTasks;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/910b663b/src/java/org/apache/cassandra/net/OutboundTcpConnection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/OutboundTcpConnection.java b/src/java/org/apache/cassandra/net/OutboundTcpConnection.java
index f9d866d..b6d2df3 100644
--- a/src/java/org/apache/cassandra/net/OutboundTcpConnection.java
+++ b/src/java/org/apache/cassandra/net/OutboundTcpConnection.java
@@ -44,7 +44,7 @@ public class OutboundTcpConnection extends Thread
     private static final Message CLOSE_SENTINEL = new Message(FBUtilities.getBroadcastAddress(),
                                                               StorageService.Verb.INTERNAL_RESPONSE,
                                                               ArrayUtils.EMPTY_BYTE_ARRAY,
-                                                              MessagingService.version_);
+                                                              MessagingService.current_version);
 
     private static final int OPEN_RETRY_DELAY = 100; // ms between retries
 
@@ -181,7 +181,7 @@ public class OutboundTcpConnection extends Thread
         */
         int header = 0;
         // Setting up the serializer bit
-        header |= MessagingService.serializerType_.ordinal();
+        header |= MessagingService.serializerType.ordinal();
         // set compression bit.
         if (false)
             header |= 4;
@@ -192,10 +192,10 @@ public class OutboundTcpConnection extends Thread
         out.writeInt(header);
         // compute total Message length for compatibility w/ 0.8 and earlier
         byte[] bytes = message.getMessageBody();
-        int total = messageLength(message.header_, id, bytes);
+        int total = messageLength(message.header, id, bytes);
         out.writeInt(total);
         out.writeUTF(id);
-        Header.serializer().serialize(message.header_, out, message.getVersion());
+        Header.serializer().serialize(message.header, out, message.getVersion());
         out.writeInt(bytes.length);
         out.write(bytes);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/910b663b/src/java/org/apache/cassandra/net/ResponseVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/ResponseVerbHandler.java b/src/java/org/apache/cassandra/net/ResponseVerbHandler.java
index 818f703..7292507 100644
--- a/src/java/org/apache/cassandra/net/ResponseVerbHandler.java
+++ b/src/java/org/apache/cassandra/net/ResponseVerbHandler.java
@@ -23,7 +23,7 @@ import org.slf4j.LoggerFactory;
 
 public class ResponseVerbHandler implements IVerbHandler
 {
-    private static final Logger logger_ = LoggerFactory.getLogger( ResponseVerbHandler.class );
+    private static final Logger logger = LoggerFactory.getLogger( ResponseVerbHandler.class );
 
     public void doVerb(Message message, String id)
     {     
@@ -31,7 +31,7 @@ public class ResponseVerbHandler implements IVerbHandler
         CallbackInfo callbackInfo = MessagingService.instance().removeRegisteredCallback(id);
         if (callbackInfo == null)
         {
-            logger_.debug("Callback already removed for {}", id);
+            logger.debug("Callback already removed for {}", id);
             return;
         }
 
@@ -40,14 +40,14 @@ public class ResponseVerbHandler implements IVerbHandler
 
         if (cb instanceof IAsyncCallback)
         {
-            if (logger_.isDebugEnabled())
-                logger_.debug("Processing response on a callback from " + id + "@" + message.getFrom());
+            if (logger.isDebugEnabled())
+                logger.debug("Processing response on a callback from " + id + "@" + message.getFrom());
             ((IAsyncCallback) cb).response(message);
         }
         else
         {
-            if (logger_.isDebugEnabled())
-                logger_.debug("Processing response on an async result from " + id + "@" + message.getFrom());
+            if (logger.isDebugEnabled())
+                logger.debug("Processing response on an async result from " + id + "@" + message.getFrom());
             ((IAsyncResult) cb).result(message);
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/910b663b/src/java/org/apache/cassandra/security/SSLFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/security/SSLFactory.java b/src/java/org/apache/cassandra/security/SSLFactory.java
index 794f7d9..304b23d 100644
--- a/src/java/org/apache/cassandra/security/SSLFactory.java
+++ b/src/java/org/apache/cassandra/security/SSLFactory.java
@@ -48,7 +48,7 @@ import com.google.common.collect.Sets;
  */
 public final class SSLFactory
 {
-    private static final Logger logger_ = LoggerFactory.getLogger(SSLFactory.class);
+    private static final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
 
     public static SSLServerSocket getServerSocket(EncryptionOptions options, InetAddress address, int port) throws IOException
     {
@@ -130,9 +130,9 @@ public final class SSLFactory
     private static String[] filterCipherSuites(String[] supported, String[] desired)
     {
         Set<String> des = Sets.newHashSet(desired);
-        Set<String> return_ = Sets.intersection(Sets.newHashSet(supported), des);
-        if (des.size() > return_.size())
-            logger_.warn("Filtering out {} as it isnt supported by the socket", StringUtils.join(Sets.difference(des, return_), ","));
-        return return_.toArray(new String[return_.size()]);
+        Set<String> toReturn = Sets.intersection(Sets.newHashSet(supported), des);
+        if (des.size() > toReturn.size())
+            logger.warn("Filtering out {} as it isnt supported by the socket", StringUtils.join(Sets.difference(des, toReturn), ","));
+        return toReturn.toArray(new String[toReturn.size()]);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/910b663b/src/java/org/apache/cassandra/service/LoadBroadcaster.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/LoadBroadcaster.java b/src/java/org/apache/cassandra/service/LoadBroadcaster.java
index b3232fe..be5e612 100644
--- a/src/java/org/apache/cassandra/service/LoadBroadcaster.java
+++ b/src/java/org/apache/cassandra/service/LoadBroadcaster.java
@@ -33,9 +33,9 @@ public class LoadBroadcaster implements IEndpointStateChangeSubscriber
 
     public static final LoadBroadcaster instance = new LoadBroadcaster();
 
-    private static final Logger logger_ = LoggerFactory.getLogger(LoadBroadcaster.class);
+    private static final Logger logger = LoggerFactory.getLogger(LoadBroadcaster.class);
 
-    private Map<InetAddress, Double> loadInfo_ = new HashMap<InetAddress, Double>();
+    private Map<InetAddress, Double> loadInfo = new HashMap<InetAddress, Double>();
 
     private LoadBroadcaster()
     {
@@ -46,7 +46,7 @@ public class LoadBroadcaster implements IEndpointStateChangeSubscriber
     {
         if (state != ApplicationState.LOAD)
             return;
-        loadInfo_.put(endpoint, Double.valueOf(value.value));
+        loadInfo.put(endpoint, Double.valueOf(value.value));
     }
 
     public void onJoin(InetAddress endpoint, EndpointState epState)
@@ -66,12 +66,12 @@ public class LoadBroadcaster implements IEndpointStateChangeSubscriber
 
     public void onRemove(InetAddress endpoint)
     {
-        loadInfo_.remove(endpoint);
+        loadInfo.remove(endpoint);
     }
 
     public Map<InetAddress, Double> getLoadInfo()
     {
-        return loadInfo_;
+        return loadInfo;
     }
 
     public void startBroadcasting()
@@ -82,8 +82,8 @@ public class LoadBroadcaster implements IEndpointStateChangeSubscriber
         {
             public void run()
             {
-                if (logger_.isDebugEnabled())
-                    logger_.debug("Disseminating load info ...");
+                if (logger.isDebugEnabled())
+                    logger.debug("Disseminating load info ...");
                 Gossiper.instance.addLocalApplicationState(ApplicationState.LOAD,
                                                            StorageService.instance.valueFactory.load(StorageService.instance.getLoad()));
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/910b663b/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java b/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java
index 1f22cba..a21f162 100644
--- a/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java
+++ b/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java
@@ -39,7 +39,7 @@ public class SnapshotVerbHandler implements IVerbHandler
                 Table.open(command.keyspace).clearSnapshot(command.snapshot_name);
             else
                 Table.open(command.keyspace).getColumnFamilyStore(command.column_family).snapshot(command.snapshot_name);
-            Message response = message.getReply(FBUtilities.getBroadcastAddress(), new byte[0], MessagingService.version_);
+            Message response = message.getReply(FBUtilities.getBroadcastAddress(), new byte[0], MessagingService.current_version);
             if (logger.isDebugEnabled())
                 logger.debug("Sending response to snapshot request {} to {} ", command.snapshot_name, message.getFrom());
             MessagingService.instance().sendReply(response, id, message.getFrom());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/910b663b/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 57e198d..4bc24d5 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -77,7 +77,7 @@ import org.apache.cassandra.utils.WrappedRunnable;
  */
 public class StorageService implements IEndpointStateChangeSubscriber, StorageServiceMBean
 {
-    private static Logger logger_ = LoggerFactory.getLogger(StorageService.class);
+    private static Logger logger = LoggerFactory.getLogger(StorageService.class);
 
     public static final int RING_DELAY = getRingDelay(); // delay after which we assume ring has stablized
 
@@ -159,7 +159,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         String newdelay = System.getProperty("cassandra.ring_delay_ms");
         if (newdelay != null)
         {
-            logger_.warn("Overriding RING_DELAY to {}ms", newdelay);
+            logger.warn("Overriding RING_DELAY to {}ms", newdelay);
             return Integer.parseInt(newdelay);
         }
         else
@@ -186,7 +186,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
     }
 
     /* This abstraction maintains the token/endpoint metadata information */
-    private TokenMetadata tokenMetadata_ = new TokenMetadata();
+    private TokenMetadata tokenMetadata = new TokenMetadata();
 
     public VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(getPartitioner());
 
@@ -241,10 +241,10 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
     /** This method updates the local token on disk  */
     public void setToken(Token token)
     {
-        if (logger_.isDebugEnabled())
-            logger_.debug("Setting token to {}", token);
+        if (logger.isDebugEnabled())
+            logger.debug("Setting token to {}", token);
         SystemTable.updateToken(token);
-        tokenMetadata_.updateNormalToken(token, FBUtilities.getBroadcastAddress());
+        tokenMetadata.updateNormalToken(token, FBUtilities.getBroadcastAddress());
         Gossiper.instance.addLocalApplicationState(ApplicationState.STATUS, valueFactory.normal(getLocalToken()));
         setMode(Mode.NORMAL, false);
     }
@@ -307,7 +307,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
     {
         if (initialized)
         {
-            logger_.warn("Stopping gossip by operator request");
+            logger.warn("Stopping gossip by operator request");
             Gossiper.instance.stop();
             initialized = false;
         }
@@ -318,7 +318,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
     {
         if (!initialized)
         {
-            logger_.warn("Starting gossip by operator request");
+            logger.warn("Starting gossip by operator request");
             Gossiper.instance.start((int)(System.currentTimeMillis() / 1000));
             initialized = true;
         }
@@ -383,7 +383,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         }
         initialized = true;
         isClientMode = true;
-        logger_.info("Starting up client gossip");
+        logger.info("Starting up client gossip");
         setMode(Mode.CLIENT, false);
         Gossiper.instance.register(this);
         Gossiper.instance.start((int)(System.currentTimeMillis() / 1000)); // needed for node-ring gathering.
@@ -409,9 +409,9 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
     public synchronized void initServer(int delay) throws IOException, ConfigurationException
     {
-        logger_.info("Cassandra version: " + FBUtilities.getReleaseVersionString());
-        logger_.info("Thrift API version: " + Constants.VERSION);
-        logger_.info("CQL supported versions: " + StringUtils.join(ClientState.getCQLSupportedVersion(), ",") + " (default: " + ClientState.DEFAULT_CQL_VERSION + ")");
+        logger.info("Cassandra version: " + FBUtilities.getReleaseVersionString());
+        logger.info("Thrift API version: " + Constants.VERSION);
+        logger.info("CQL supported versions: " + StringUtils.join(ClientState.getCQLSupportedVersion(), ",") + " (default: " + ClientState.DEFAULT_CQL_VERSION + ")");
 
         if (initialized)
         {
@@ -424,7 +424,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
         if (Boolean.parseBoolean(System.getProperty("cassandra.load_ring_state", "true")))
         {
-            logger_.info("Loading persisted ring state");
+            logger.info("Loading persisted ring state");
             for (Map.Entry<Token, InetAddress> entry : SystemTable.loadTokens().entrySet())
             {
                 if (entry.getValue() == FBUtilities.getLocalAddress())
@@ -434,7 +434,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
                 }
                 else
                 {
-                    tokenMetadata_.updateNormalToken(entry.getKey(), entry.getValue());
+                    tokenMetadata.updateNormalToken(entry.getKey(), entry.getValue());
                     Gossiper.instance.addSavedEndpoint(entry.getValue());
                 }
             }
@@ -442,7 +442,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
         if (Boolean.parseBoolean(System.getProperty("cassandra.renew_counter_id", "false")))
         {
-            logger_.info("Renewing local node id (as requested)");
+            logger.info("Renewing local node id (as requested)");
             NodeId.renewLocalId();
         }
 
@@ -488,7 +488,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
                 // wait for miscellaneous tasks like sstable and commitlog segment deletion
                 tasks.shutdown();
                 if (!tasks.awaitTermination(1, TimeUnit.MINUTES))
-                    logger_.warn("Miscellaneous task executor still busy after one minute; proceeding with shutdown");
+                    logger.warn("Miscellaneous task executor still busy after one minute; proceeding with shutdown");
             }
         }, "StorageServiceShutdownHook");
         Runtime.getRuntime().addShutdownHook(drainOnShutdown);
@@ -499,13 +499,13 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         }
         else
         {
-            logger_.info("Not joining ring as requested. Use JMX (StorageService->joinRing()) to initiate ring joining");
+            logger.info("Not joining ring as requested. Use JMX (StorageService->joinRing()) to initiate ring joining");
         }
     }
 
     private void joinTokenRing(int delay) throws IOException, org.apache.cassandra.config.ConfigurationException
     {
-        logger_.info("Starting up server gossip");
+        logger.info("Starting up server gossip");
         joined = true;
 
 
@@ -532,7 +532,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         if (DatabaseDescriptor.isAutoBootstrap()
                 && DatabaseDescriptor.getSeeds().contains(FBUtilities.getBroadcastAddress())
                 && !SystemTable.isBootstrapped())
-            logger_.info("This node will not auto bootstrap because it is configured to be a seed node.");
+            logger.info("This node will not auto bootstrap because it is configured to be a seed node.");
 
         InetAddress current = null;
         // first startup is only chance to bootstrap
@@ -568,18 +568,18 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
             setMode(Mode.JOINING, "schema complete, ready to bootstrap", true);
 
 
-            if (logger_.isDebugEnabled())
-                logger_.debug("... got ring + schema info");
+            if (logger.isDebugEnabled())
+                logger.debug("... got ring + schema info");
 
             if (DatabaseDescriptor.getReplaceToken() == null)
             {
-                if (tokenMetadata_.isMember(FBUtilities.getBroadcastAddress()))
+                if (tokenMetadata.isMember(FBUtilities.getBroadcastAddress()))
                 {
                     String s = "This node is already a member of the token ring; bootstrap aborted. (If replacing a dead node, remove the old one from the ring first.)";
                     throw new UnsupportedOperationException(s);
                 }
                 setMode(Mode.JOINING, "getting bootstrap token", true);
-                token = BootStrapper.getBootstrapToken(tokenMetadata_, LoadBroadcaster.instance.getLoadInfo());
+                token = BootStrapper.getBootstrapToken(tokenMetadata, LoadBroadcaster.instance.getLoadInfo());
             }
             else
             {
@@ -595,7 +595,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
                 }
                 token = StorageService.getPartitioner().getTokenFactory().fromString(DatabaseDescriptor.getReplaceToken());
                 // check for operator errors...
-                current = tokenMetadata_.getEndpoint(token);
+                current = tokenMetadata.getEndpoint(token);
                 if (null != current && Gossiper.instance.getEndpointStateForEndpoint(current).getUpdateTimestamp() > (System.currentTimeMillis() - delay))
                     throw new UnsupportedOperationException("Cannnot replace a token for a Live node... ");
                 setMode(Mode.JOINING, "Replacing a node with token: " + token, true);
@@ -613,17 +613,17 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
                 if (initialToken == null)
                 {
                     token = getPartitioner().getRandomToken();
-                    logger_.warn("Generated random token " + token + ". Random tokens will result in an unbalanced ring; see http://wiki.apache.org/cassandra/Operations");
+                    logger.warn("Generated random token " + token + ". Random tokens will result in an unbalanced ring; see http://wiki.apache.org/cassandra/Operations");
                 }
                 else
                 {
                     token = getPartitioner().getTokenFactory().fromString(initialToken);
-                    logger_.info("Saved token not found. Using " + token + " from configuration");
+                    logger.info("Saved token not found. Using " + token + " from configuration");
                 }
             }
             else
             {
-                logger_.info("Using saved token " + token);
+                logger.info("Using saved token " + token);
             }
         }
 
@@ -635,12 +635,12 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
             // remove the existing info about the replaced node.
             if (current != null)
                 Gossiper.instance.replacedEndpoint(current);
-            logger_.info("Bootstrap/Replace/Move completed! Now serving reads.");
-            assert tokenMetadata_.sortedTokens().size() > 0;
+            logger.info("Bootstrap/Replace/Move completed! Now serving reads.");
+            assert tokenMetadata.sortedTokens().size() > 0;
         }
         else
         {
-            logger_.info("Bootstrap complete, but write survey mode is active, not becoming an active ring member. Use JMX (StorageService->joinRing()) to finalize ring joining.");
+            logger.info("Bootstrap complete, but write survey mode is active, not becoming an active ring member. Use JMX (StorageService->joinRing()) to finalize ring joining.");
         }
     }
 
@@ -648,7 +648,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
     {
         if (!joined)
         {
-            logger_.info("Joining ring by operator request");
+            logger.info("Joining ring by operator request");
             joinTokenRing(0);
         }
         else if (isSurveyMode)
@@ -656,8 +656,8 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
             setToken(SystemTable.getSavedToken());
             SystemTable.setBootstrapped(true);
             isSurveyMode = false;
-            logger_.info("Leaving write survey mode and joining ring at operator request");
-            assert tokenMetadata_.sortedTokens().size() > 0;
+            logger.info("Leaving write survey mode and joining ring at operator request");
+            assert tokenMetadata.sortedTokens().size() > 0;
         }
     }
 
@@ -668,9 +668,9 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
     public void rebuild(String sourceDc)
     {
-        logger_.info("rebuild from dc: {}", sourceDc == null ? "(any dc)" : sourceDc);
+        logger.info("rebuild from dc: {}", sourceDc == null ? "(any dc)" : sourceDc);
 
-        RangeStreamer streamer = new RangeStreamer(tokenMetadata_, FBUtilities.getBroadcastAddress(), OperationType.REBUILD);
+        RangeStreamer streamer = new RangeStreamer(tokenMetadata, FBUtilities.getBroadcastAddress(), OperationType.REBUILD);
         streamer.addSourceFilter(new RangeStreamer.FailureDetectorSourceFilter(FailureDetector.instance));
         if (sourceDc != null)
             streamer.addSourceFilter(new RangeStreamer.SingleDatacenterFilter(DatabaseDescriptor.getEndpointSnitch(), sourceDc));
@@ -684,7 +684,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
     public void setStreamThroughputMbPerSec(int value)
     {
         DatabaseDescriptor.setStreamThroughputOutboundMegabitsPerSec(value);
-        logger_.info("setstreamthroughput: throttle set to {}", value);
+        logger.info("setstreamthroughput: throttle set to {}", value);
     }
 
     public int getStreamThroughputMbPerSec()
@@ -722,9 +722,9 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         operationMode = m;
         String logMsg = msg == null ? m.toString() : String.format("%s: %s", m, msg);
         if (log)
-            logger_.info(logMsg);
+            logger.info(logMsg);
         else
-            logger_.debug(logMsg);
+            logger.debug(logMsg);
     }
 
     private void bootstrap(Token token) throws IOException
@@ -748,10 +748,10 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         else
         {
             // Dont set any state for the node which is bootstrapping the existing token...
-            tokenMetadata_.updateNormalToken(token, FBUtilities.getBroadcastAddress());
+            tokenMetadata.updateNormalToken(token, FBUtilities.getBroadcastAddress());
         }
         setMode(Mode.JOINING, "Starting to bootstrap...", true);
-        new BootStrapper(FBUtilities.getBroadcastAddress(), token, tokenMetadata_).bootstrap(); // handles token update
+        new BootStrapper(FBUtilities.getBroadcastAddress(), token, tokenMetadata).bootstrap(); // handles token update
     }
 
     public boolean isBootstrapMode()
@@ -761,7 +761,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
     public TokenMetadata getTokenMetadata()
     {
-        return tokenMetadata_;
+        return tokenMetadata;
     }
 
     /**
@@ -824,7 +824,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
             keyspace = Schema.instance.getNonSystemTables().get(0);
 
         Map<List<String>, List<String>> map = new HashMap<List<String>, List<String>>();
-        for (Map.Entry<Range<Token>, Collection<InetAddress>> entry : tokenMetadata_.getPendingRanges(keyspace).entrySet())
+        for (Map.Entry<Range<Token>, Collection<InetAddress>> entry : tokenMetadata.getPendingRanges(keyspace).entrySet())
         {
             List<InetAddress> l = new ArrayList<InetAddress>(entry.getValue());
             map.put(entry.getKey().asList(), stringify(l));
@@ -839,7 +839,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         if (keyspace == null)
             keyspace = Schema.instance.getNonSystemTables().get(0);
 
-        List<Range<Token>> ranges = getAllRanges(tokenMetadata_.sortedTokens());
+        List<Range<Token>> ranges = getAllRanges(tokenMetadata.sortedTokens());
         return constructRangeToEndpointMap(keyspace, ranges);
     }
 
@@ -913,7 +913,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
     public Map<String, String> getTokenToEndpointMap()
     {
-        Map<Token, InetAddress> mapInetAddress = tokenMetadata_.getNormalAndBootstrappingTokenToEndpointMap();
+        Map<Token, InetAddress> mapInetAddress = tokenMetadata.getNormalAndBootstrappingTokenToEndpointMap();
         // in order to preserve tokens in ascending order, we use LinkedHashMap here
         Map<String, String> mapString = new LinkedHashMap<String, String>(mapInetAddress.size());
         List<Token> tokens = new ArrayList<Token>(mapInetAddress.keySet());
@@ -951,7 +951,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
             }
         });
         
-        List<Range<Token>> ranges = getAllRanges(tokenMetadata_.sortedTokens());
+        List<Range<Token>> ranges = getAllRanges(tokenMetadata.sortedTokens());
         for (Range<Token> range : ranges)
         {
             for (InetAddress endpoint : Table.open(keyspace).getReplicationStrategy().getNaturalEndpoints(range.left))
@@ -1029,25 +1029,25 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         assert pieces.length >= 2;
         Token token = getPartitioner().getTokenFactory().fromString(pieces[1]);
 
-        if (logger_.isDebugEnabled())
-            logger_.debug("Node " + endpoint + " state bootstrapping, token " + token);
+        if (logger.isDebugEnabled())
+            logger.debug("Node " + endpoint + " state bootstrapping, token " + token);
 
         // if this node is present in token metadata, either we have missed intermediate states
         // or the node had crashed. Print warning if needed, clear obsolete stuff and
         // continue.
-        if (tokenMetadata_.isMember(endpoint))
+        if (tokenMetadata.isMember(endpoint))
         {
             // If isLeaving is false, we have missed both LEAVING and LEFT. However, if
             // isLeaving is true, we have only missed LEFT. Waiting time between completing
             // leave operation and rebootstrapping is relatively short, so the latter is quite
             // common (not enough time for gossip to spread). Therefore we report only the
             // former in the log.
-            if (!tokenMetadata_.isLeaving(endpoint))
-                logger_.info("Node " + endpoint + " state jump to bootstrap");
-            tokenMetadata_.removeEndpoint(endpoint);
+            if (!tokenMetadata.isLeaving(endpoint))
+                logger.info("Node " + endpoint + " state jump to bootstrap");
+            tokenMetadata.removeEndpoint(endpoint);
         }
 
-        tokenMetadata_.addBootstrapToken(token, endpoint);
+        tokenMetadata.addBootstrapToken(token, endpoint);
         calculatePendingRanges();
     }
 
@@ -1063,18 +1063,18 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         assert pieces.length >= 2;
         Token token = getPartitioner().getTokenFactory().fromString(pieces[1]);
 
-        if (logger_.isDebugEnabled())
-            logger_.debug("Node " + endpoint + " state normal, token " + token);
+        if (logger.isDebugEnabled())
+            logger.debug("Node " + endpoint + " state normal, token " + token);
 
-        if (tokenMetadata_.isMember(endpoint))
-            logger_.info("Node " + endpoint + " state jump to normal");
+        if (tokenMetadata.isMember(endpoint))
+            logger.info("Node " + endpoint + " state jump to normal");
 
         // we don't want to update if this node is responsible for the token and it has a later startup time than endpoint.
-        InetAddress currentOwner = tokenMetadata_.getEndpoint(token);
+        InetAddress currentOwner = tokenMetadata.getEndpoint(token);
         if (currentOwner == null)
         {
-            logger_.debug("New node " + endpoint + " at token " + token);
-            tokenMetadata_.updateNormalToken(token, endpoint);
+            logger.debug("New node " + endpoint + " at token " + token);
+            tokenMetadata.updateNormalToken(token, endpoint);
             if (!isClientMode)
                 SystemTable.updateToken(endpoint, token);
         }
@@ -1082,25 +1082,25 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         {
             // set state back to normal, since the node may have tried to leave, but failed and is now back up
             // no need to persist, token/ip did not change
-            tokenMetadata_.updateNormalToken(token, endpoint);
+            tokenMetadata.updateNormalToken(token, endpoint);
         }
         else if (Gossiper.instance.compareEndpointStartup(endpoint, currentOwner) > 0)
         {
-            logger_.info(String.format("Nodes %s and %s have the same token %s.  %s is the new owner",
+            logger.info(String.format("Nodes %s and %s have the same token %s.  %s is the new owner",
                                        endpoint, currentOwner, token, endpoint));
-            tokenMetadata_.updateNormalToken(token, endpoint);
+            tokenMetadata.updateNormalToken(token, endpoint);
             Gossiper.instance.removeEndpoint(currentOwner);
             if (!isClientMode)
                 SystemTable.updateToken(endpoint, token);
         }
         else
         {
-            logger_.info(String.format("Nodes %s and %s have the same token %s.  Ignoring %s",
+            logger.info(String.format("Nodes %s and %s have the same token %s.  Ignoring %s",
                                        endpoint, currentOwner, token, endpoint));
         }
 
-        if (tokenMetadata_.isMoving(endpoint)) // if endpoint was moving to a new token
-            tokenMetadata_.removeFromMoving(endpoint);
+        if (tokenMetadata.isMoving(endpoint)) // if endpoint was moving to a new token
+            tokenMetadata.removeFromMoving(endpoint);
 
         calculatePendingRanges();
     }
@@ -1117,26 +1117,26 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         String moveValue = pieces[1];
         Token token = getPartitioner().getTokenFactory().fromString(moveValue);
 
-        if (logger_.isDebugEnabled())
-            logger_.debug("Node " + endpoint + " state leaving, token " + token);
+        if (logger.isDebugEnabled())
+            logger.debug("Node " + endpoint + " state leaving, token " + token);
 
         // If the node is previously unknown or tokens do not match, update tokenmetadata to
         // have this node as 'normal' (it must have been using this token before the
         // leave). This way we'll get pending ranges right.
-        if (!tokenMetadata_.isMember(endpoint))
+        if (!tokenMetadata.isMember(endpoint))
         {
-            logger_.info("Node " + endpoint + " state jump to leaving");
-            tokenMetadata_.updateNormalToken(token, endpoint);
+            logger.info("Node " + endpoint + " state jump to leaving");
+            tokenMetadata.updateNormalToken(token, endpoint);
         }
-        else if (!tokenMetadata_.getToken(endpoint).equals(token))
+        else if (!tokenMetadata.getToken(endpoint).equals(token))
         {
-            logger_.warn("Node " + endpoint + " 'leaving' token mismatch. Long network partition?");
-            tokenMetadata_.updateNormalToken(token, endpoint);
+            logger.warn("Node " + endpoint + " 'leaving' token mismatch. Long network partition?");
+            tokenMetadata.updateNormalToken(token, endpoint);
         }
 
         // at this point the endpoint is certainly a member with this token, so let's proceed
         // normally
-        tokenMetadata_.addLeavingEndpoint(endpoint);
+        tokenMetadata.addLeavingEndpoint(endpoint);
         calculatePendingRanges();
     }
 
@@ -1151,8 +1151,8 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         assert pieces.length >= 2;
         Token token = getPartitioner().getTokenFactory().fromString(pieces[1]);
 
-        if (logger_.isDebugEnabled())
-            logger_.debug("Node " + endpoint + " state left, token " + token);
+        if (logger.isDebugEnabled())
+            logger.debug("Node " + endpoint + " state left, token " + token);
 
         excise(token, endpoint, extractExpireTime(pieces));
     }
@@ -1168,10 +1168,10 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         assert pieces.length >= 2;
         Token token = getPartitioner().getTokenFactory().fromString(pieces[1]);
 
-        if (logger_.isDebugEnabled())
-            logger_.debug("Node " + endpoint + " state moving, new token " + token);
+        if (logger.isDebugEnabled())
+            logger.debug("Node " + endpoint + " state moving, new token " + token);
 
-        tokenMetadata_.addMovingEndpoint(token, endpoint);
+        tokenMetadata.addMovingEndpoint(token, endpoint);
 
         calculatePendingRanges();
     }
@@ -1188,7 +1188,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
         if (endpoint.equals(FBUtilities.getBroadcastAddress()))
         {
-            logger_.info("Received removeToken gossip about myself. Is this node rejoining after an explicit removetoken?");
+            logger.info("Received removeToken gossip about myself. Is this node rejoining after an explicit removetoken?");
             try
             {
                 drain();
@@ -1199,10 +1199,10 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
             }
             return;
         }
-        if (tokenMetadata_.isMember(endpoint))
+        if (tokenMetadata.isMember(endpoint))
         {
             String state = pieces[0];
-            Token removeToken = tokenMetadata_.getToken(endpoint);
+            Token removeToken = tokenMetadata.getToken(endpoint);
 
             if (VersionedValue.REMOVED_TOKEN.equals(state))
             {
@@ -1210,18 +1210,18 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
             }
             else if (VersionedValue.REMOVING_TOKEN.equals(state))
             {
-                if (logger_.isDebugEnabled())
-                    logger_.debug("Token " + removeToken + " removed manually (endpoint was " + endpoint + ")");
+                if (logger.isDebugEnabled())
+                    logger.debug("Token " + removeToken + " removed manually (endpoint was " + endpoint + ")");
 
                 // Note that the endpoint is being removed
-                tokenMetadata_.addLeavingEndpoint(endpoint);
+                tokenMetadata.addLeavingEndpoint(endpoint);
                 calculatePendingRanges();
 
                 // find the endpoint coordinating this removal that we need to notify when we're done
                 String[] coordinator = Gossiper.instance.getEndpointStateForEndpoint(endpoint).getApplicationState(ApplicationState.REMOVAL_COORDINATOR).value.split(VersionedValue.DELIMITER_STR, -1);
                 Token coordtoken = getPartitioner().getTokenFactory().fromString(coordinator[1]);
                 // grab any data we are now responsible for and notify responsible node
-                restoreReplicaCount(endpoint, tokenMetadata_.getEndpoint(coordtoken));
+                restoreReplicaCount(endpoint, tokenMetadata.getEndpoint(coordtoken));
             }
         } // not a member, nothing to do
     }
@@ -1230,12 +1230,12 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
     {
         HintedHandOffManager.instance.deleteHintsForEndpoint(endpoint);
         Gossiper.instance.removeEndpoint(endpoint);
-        tokenMetadata_.removeEndpoint(endpoint);
-        tokenMetadata_.removeBootstrapToken(token);
+        tokenMetadata.removeEndpoint(endpoint);
+        tokenMetadata.removeBootstrapToken(token);
         calculatePendingRanges();
         if (!isClientMode)
         {
-            logger_.info("Removing token " + token + " for " + endpoint);
+            logger.info("Removing token " + token + " for " + endpoint);
             SystemTable.removeToken(token);
         }
     }
@@ -1303,8 +1303,8 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
         if (bootstrapTokens.isEmpty() && leavingEndpoints.isEmpty() && tm.getMovingEndpoints().isEmpty())
         {
-            if (logger_.isDebugEnabled())
-                logger_.debug("No bootstrapping, leaving or moving nodes -> empty pending ranges for {}", table);
+            if (logger.isDebugEnabled())
+                logger.debug("No bootstrapping, leaving or moving nodes -> empty pending ranges for {}", table);
             tm.setPendingRanges(table, pendingRanges);
             return;
         }
@@ -1368,8 +1368,8 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
         tm.setPendingRanges(table, pendingRanges);
 
-        if (logger_.isDebugEnabled())
-            logger_.debug("Pending ranges:\n" + (pendingRanges.isEmpty() ? "<empty>" : tm.printPendingRanges()));
+        if (logger.isDebugEnabled())
+            logger.debug("Pending ranges:\n" + (pendingRanges.isEmpty() ? "<empty>" : tm.printPendingRanges()));
     }
 
     /**
@@ -1382,7 +1382,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
     private Multimap<InetAddress, Range<Token>> getNewSourceRanges(String table, Set<Range<Token>> ranges)
     {
         InetAddress myAddress = FBUtilities.getBroadcastAddress();
-        Multimap<Range<Token>, InetAddress> rangeAddresses = Table.open(table).getReplicationStrategy().getRangeAddresses(tokenMetadata_);
+        Multimap<Range<Token>, InetAddress> rangeAddresses = Table.open(table).getReplicationStrategy().getRangeAddresses(tokenMetadata);
         Multimap<InetAddress, Range<Token>> sourceRanges = HashMultimap.create();
         IFailureDetector failureDetector = FailureDetector.instance;
 
@@ -1418,8 +1418,8 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         // notify the remote token
         Message msg = new Message(local, StorageService.Verb.REPLICATION_FINISHED, new byte[0], Gossiper.instance.getVersion(remote));
         IFailureDetector failureDetector = FailureDetector.instance;
-        if (logger_.isDebugEnabled())
-            logger_.debug("Notifying " + remote.toString() + " of replication completion\n");
+        if (logger.isDebugEnabled())
+            logger.debug("Notifying " + remote.toString() + " of replication completion\n");
         while (failureDetector.isAlive(remote))
         {
             IAsyncResult iar = MessagingService.instance().sendRR(msg, remote);
@@ -1487,8 +1487,8 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
                         }
                     }
                 };
-                if (logger_.isDebugEnabled())
-                    logger_.debug("Requesting from " + source + " ranges " + StringUtils.join(ranges, ", "));
+                if (logger.isDebugEnabled())
+                    logger.debug("Requesting from " + source + " ranges " + StringUtils.join(ranges, ", "));
                 StreamIn.requestRanges(source, table, ranges, callback, OperationType.RESTORE_REPLICA_COUNT);
             }
         }
@@ -1500,16 +1500,16 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         // First get all ranges the leaving endpoint is responsible for
         Collection<Range<Token>> ranges = getRangesForEndpoint(table, endpoint);
 
-        if (logger_.isDebugEnabled())
-            logger_.debug("Node " + endpoint + " ranges [" + StringUtils.join(ranges, ", ") + "]");
+        if (logger.isDebugEnabled())
+            logger.debug("Node " + endpoint + " ranges [" + StringUtils.join(ranges, ", ") + "]");
 
         Map<Range<Token>, List<InetAddress>> currentReplicaEndpoints = new HashMap<Range<Token>, List<InetAddress>>();
 
         // Find (for each range) all nodes that store replicas for these ranges as well
         for (Range<Token> range : ranges)
-            currentReplicaEndpoints.put(range, Table.open(table).getReplicationStrategy().calculateNaturalEndpoints(range.right, tokenMetadata_));
+            currentReplicaEndpoints.put(range, Table.open(table).getReplicationStrategy().calculateNaturalEndpoints(range.right, tokenMetadata));
 
-        TokenMetadata temp = tokenMetadata_.cloneAfterAllLeft();
+        TokenMetadata temp = tokenMetadata.cloneAfterAllLeft();
 
         // endpoint might or might not be 'leaving'. If it was not leaving (that is, removetoken
         // command was used), it is still present in temp and must be removed.
@@ -1527,11 +1527,11 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         {
             Collection<InetAddress> newReplicaEndpoints = Table.open(table).getReplicationStrategy().calculateNaturalEndpoints(range.right, temp);
             newReplicaEndpoints.removeAll(currentReplicaEndpoints.get(range));
-            if (logger_.isDebugEnabled())
+            if (logger.isDebugEnabled())
                 if (newReplicaEndpoints.isEmpty())
-                    logger_.debug("Range " + range + " already in all replicas");
+                    logger.debug("Range " + range + " already in all replicas");
                 else
-                    logger_.debug("Range " + range + " will be responsibility of " + StringUtils.join(newReplicaEndpoints, ", "));
+                    logger.debug("Range " + range + " will be responsibility of " + StringUtils.join(newReplicaEndpoints, ", "));
             changedRanges.putAll(range, newReplicaEndpoints);
         }
 
@@ -1554,7 +1554,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
     public void onRemove(InetAddress endpoint)
     {
-        tokenMetadata_.removeEndpoint(endpoint);
+        tokenMetadata.removeEndpoint(endpoint);
         calculatePendingRanges();
     }
 
@@ -1626,14 +1626,14 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
     public List<String> getLeavingNodes()
     {
-        return stringify(tokenMetadata_.getLeavingEndpoints());
+        return stringify(tokenMetadata.getLeavingEndpoints());
     }
 
     public List<String> getMovingNodes()
     {
         List<String> endpoints = new ArrayList<String>();
 
-        for (Pair<Token, InetAddress> node : tokenMetadata_.getMovingEndpoints())
+        for (Pair<Token, InetAddress> node : tokenMetadata.getMovingEndpoints())
         {
             endpoints.add(node.right.getHostAddress());
         }
@@ -1643,7 +1643,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
     public List<String> getJoiningNodes()
     {
-        return stringify(tokenMetadata_.getBootstrapTokens().values());
+        return stringify(tokenMetadata.getBootstrapTokens().values());
     }
 
     public List<String> getLiveNodes()
@@ -1801,8 +1801,8 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         for (Table table : tables)
             table.clearSnapshot(tag);
 
-        if (logger_.isDebugEnabled())
-            logger_.debug("Cleared out snapshot directories");
+        if (logger.isDebugEnabled())
+            logger.debug("Cleared out snapshot directories");
     }
 
     public Iterable<ColumnFamilyStore> getValidColumnFamilies(String tableName, String... cfNames) throws IOException
@@ -1821,7 +1821,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
             if (cfStore == null)
             {
                 // this means there was a cf passed in that is not recognized in the keyspace. report it and continue.
-                logger_.warn(String.format("Invalid column family specified: %s. Proceeding with others.", cfName));
+                logger.warn(String.format("Invalid column family specified: %s. Proceeding with others.", cfName));
                 continue;
             }
             valid.add(cfStore);
@@ -1840,7 +1840,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
     {
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(tableName, columnFamilies))
         {
-            logger_.debug("Forcing flush on keyspace " + tableName + ", CF " + cfStore.getColumnFamilyName());
+            logger.debug("Forcing flush on keyspace " + tableName + ", CF " + cfStore.getColumnFamilyName());
             cfStore.forceBlockingFlush();
         }
     }
@@ -1858,7 +1858,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
         Collection<Range<Token>> ranges = getLocalRanges(tableName);
         int cmd = nextRepairCommand.incrementAndGet();
-        logger_.info("Starting repair command #{}, repairing {} ranges.", cmd, ranges.size());
+        logger.info("Starting repair command #{}, repairing {} ranges.", cmd, ranges.size());
 
         List<AntiEntropyService.RepairFuture> futures = new ArrayList<AntiEntropyService.RepairFuture>(ranges.size());
         for (Range<Token> range : ranges)
@@ -1872,7 +1872,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
             }
             catch (InterruptedException e)
             {
-                logger_.error("Interrupted while waiting for the differencing of repair session " + future.session + " to be done. Repair may be imprecise.", e);
+                logger.error("Interrupted while waiting for the differencing of repair session " + future.session + " to be done. Repair may be imprecise.", e);
             }
         }
 
@@ -1887,7 +1887,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
             }
             catch (Exception e)
             {
-                logger_.error("Repair session " + future.session.getName() + " failed.", e);
+                logger.error("Repair session " + future.session.getName() + " failed.", e);
                 failedSession = true;
             }
         }
@@ -1895,7 +1895,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         if (failedSession)
             throw new IOException("Repair command #" + cmd + ": some repair session(s) failed (see log for details).");
         else
-            logger_.info("Repair command #{} completed successfully", cmd);
+            logger.info("Repair command #{} completed successfully", cmd);
     }
 
     public void forceTableRepairPrimaryRange(final String tableName, boolean isSequential, final String... columnFamilies) throws IOException
@@ -1910,7 +1910,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         }
         catch (Exception e)
         {
-            logger_.error("Repair session " + future.session.getName() + " failed.", e);
+            logger.error("Repair session " + future.session.getName() + " failed.", e);
             throw new IOException("Some repair session(s) failed (see log for details).");
         }
     }
@@ -1938,8 +1938,8 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
      */
     InetAddress getPredecessor(InetAddress ep)
     {
-        Token token = tokenMetadata_.getToken(ep);
-        return tokenMetadata_.getEndpoint(tokenMetadata_.getPredecessor(token));
+        Token token = tokenMetadata.getToken(ep);
+        return tokenMetadata.getEndpoint(tokenMetadata.getPredecessor(token));
     }
 
     /*
@@ -1948,8 +1948,8 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
      */
     public InetAddress getSuccessor(InetAddress ep)
     {
-        Token token = tokenMetadata_.getToken(ep);
-        return tokenMetadata_.getEndpoint(tokenMetadata_.getSuccessor(token));
+        Token token = tokenMetadata.getToken(ep);
+        return tokenMetadata.getEndpoint(tokenMetadata.getSuccessor(token));
     }
 
     /**
@@ -1959,7 +1959,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
      */
     public Range<Token> getPrimaryRangeForEndpoint(InetAddress ep)
     {
-        return tokenMetadata_.getPrimaryRangeFor(tokenMetadata_.getToken(ep));
+        return tokenMetadata.getPrimaryRangeFor(tokenMetadata.getToken(ep));
     }
 
     /**
@@ -1980,8 +1980,8 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
     */
     public List<Range<Token>> getAllRanges(List<Token> sortedTokens)
     {
-        if (logger_.isDebugEnabled())
-            logger_.debug("computing ranges for " + StringUtils.join(sortedTokens, ", "));
+        if (logger.isDebugEnabled())
+            logger.debug("computing ranges for " + StringUtils.join(sortedTokens, ", "));
 
         if (sortedTokens.isEmpty())
             return Collections.emptyList();
@@ -2062,7 +2062,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
     {
         Level level = Level.toLevel(rawLevel);
         org.apache.log4j.Logger.getLogger(classQualifier).setLevel(level);
-        logger_.info("set log level to " + level + " for classes under '" + classQualifier + "' (if the level doesn't look like '" + rawLevel + "' then log4j couldn't parse '" + rawLevel + "')");
+        logger.info("set log level to " + level + " for classes under '" + classQualifier + "' (if the level doesn't look like '" + rawLevel + "' then log4j couldn't parse '" + rawLevel + "')");
     }
 
     /**
@@ -2112,49 +2112,49 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         if (keys.size() < 3)
         {
             token = getPartitioner().midpoint(range.left, range.right);
-            logger_.debug("Used midpoint to assign token " + token);
+            logger.debug("Used midpoint to assign token " + token);
         }
         else
         {
             token = keys.get(keys.size() / 2).token;
-            logger_.debug("Used key sample of size " + keys.size() + " to assign token " + token);
+            logger.debug("Used key sample of size " + keys.size() + " to assign token " + token);
         }
-        if (tokenMetadata_.getEndpoint(token) != null && tokenMetadata_.isMember(tokenMetadata_.getEndpoint(token)))
-            throw new RuntimeException("Chose token " + token + " which is already in use by " + tokenMetadata_.getEndpoint(token) + " -- specify one manually with initial_token");
+        if (tokenMetadata.getEndpoint(token) != null && tokenMetadata.isMember(tokenMetadata.getEndpoint(token)))
+            throw new RuntimeException("Chose token " + token + " which is already in use by " + tokenMetadata.getEndpoint(token) + " -- specify one manually with initial_token");
         // Hack to prevent giving nodes tokens with DELIMITER_STR in them (which is fine in a row key/token)
         if (token instanceof StringToken)
         {
             token = new StringToken(((String)token.token).replaceAll(VersionedValue.DELIMITER_STR, ""));
-            if (tokenMetadata_.getNormalAndBootstrappingTokenToEndpointMap().containsKey(token))
+            if (tokenMetadata.getNormalAndBootstrappingTokenToEndpointMap().containsKey(token))
                 throw new RuntimeException("Unable to compute unique token for new node -- specify one manually with initial_token");
         }
         return token;
     }
 
     /**
-     * Broadcast leaving status and update local tokenMetadata_ accordingly
+     * Broadcast leaving status and update local tokenMetadata accordingly
      */
     private void startLeaving()
     {
         Gossiper.instance.addLocalApplicationState(ApplicationState.STATUS, valueFactory.leaving(getLocalToken()));
-        tokenMetadata_.addLeavingEndpoint(FBUtilities.getBroadcastAddress());
+        tokenMetadata.addLeavingEndpoint(FBUtilities.getBroadcastAddress());
         calculatePendingRanges();
     }
 
     public void decommission() throws InterruptedException
     {
-        if (!tokenMetadata_.isMember(FBUtilities.getBroadcastAddress()))
+        if (!tokenMetadata.isMember(FBUtilities.getBroadcastAddress()))
             throw new UnsupportedOperationException("local node is not a member of the token ring yet");
-        if (tokenMetadata_.cloneAfterAllLeft().sortedTokens().size() < 2)
+        if (tokenMetadata.cloneAfterAllLeft().sortedTokens().size() < 2)
             throw new UnsupportedOperationException("no other normal nodes in the ring; decommission would be pointless");
         for (String table : Schema.instance.getNonSystemTables())
         {
-            if (tokenMetadata_.getPendingRanges(table, FBUtilities.getBroadcastAddress()).size() > 0)
+            if (tokenMetadata.getPendingRanges(table, FBUtilities.getBroadcastAddress()).size() > 0)
                 throw new UnsupportedOperationException("data is currently moving to this node; unable to leave the ring");
         }
 
-        if (logger_.isDebugEnabled())
-            logger_.debug("DECOMMISSIONING");
+        if (logger.isDebugEnabled())
+            logger.debug("DECOMMISSIONING");
         startLeaving();
         setMode(Mode.LEAVING, "sleeping " + RING_DELAY + " ms for pending range setup", true);
         Thread.sleep(RING_DELAY);
@@ -2176,11 +2176,11 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
     private void leaveRing()
     {
         SystemTable.setBootstrapped(false);
-        tokenMetadata_.removeEndpoint(FBUtilities.getBroadcastAddress());
+        tokenMetadata.removeEndpoint(FBUtilities.getBroadcastAddress());
         calculatePendingRanges();
 
         Gossiper.instance.addLocalApplicationState(ApplicationState.STATUS, valueFactory.left(getLocalToken(),Gossiper.computeExpireTime()));
-        logger_.info("Announcing that I have left the ring for " + RING_DELAY + "ms");
+        logger.info("Announcing that I have left the ring for " + RING_DELAY + "ms");
         try
         {
             Thread.sleep(RING_DELAY);
@@ -2199,8 +2199,8 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         {
             Multimap<Range<Token>, InetAddress> rangesMM = getChangedRangesForLeaving(table, FBUtilities.getBroadcastAddress());
 
-            if (logger_.isDebugEnabled())
-                logger_.debug("Ranges needing transfer are [" + StringUtils.join(rangesMM.keySet(), ",") + "]");
+            if (logger.isDebugEnabled())
+                logger.debug("Ranges needing transfer are [" + StringUtils.join(rangesMM.keySet(), ",") + "]");
 
             rangesToStream.put(table, rangesMM);
         }
@@ -2210,7 +2210,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         CountDownLatch latch = streamRanges(rangesToStream);
 
         // wait for the transfer runnables to signal the latch.
-        logger_.debug("waiting for stream aks.");
+        logger.debug("waiting for stream aks.");
         try
         {
             latch.await();
@@ -2219,7 +2219,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         {
             throw new RuntimeException(e);
         }
-        logger_.debug("stream acks all received.");
+        logger.debug("stream acks all received.");
         leaveRing();
         onFinish.run();
     }
@@ -2242,7 +2242,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         if (newToken == null)
             throw new IOException("Can't move to the undefined (null) token.");
 
-        if (tokenMetadata_.sortedTokens().contains(newToken))
+        if (tokenMetadata.sortedTokens().contains(newToken))
             throw new IOException("target token " + newToken + " is already owned by another node.");
 
         // address of the current node
@@ -2252,21 +2252,21 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         // checking if data is moving to this node
         for (String table : tablesToProcess)
         {
-            if (tokenMetadata_.getPendingRanges(table, localAddress).size() > 0)
+            if (tokenMetadata.getPendingRanges(table, localAddress).size() > 0)
                 throw new UnsupportedOperationException("data is currently moving to this node; unable to leave the ring");
         }
 
         // setting 'moving' application state
         Gossiper.instance.addLocalApplicationState(ApplicationState.STATUS, valueFactory.moving(newToken));
 
-        logger_.info(String.format("Moving %s from %s to %s.", localAddress, getLocalToken(), newToken));
+        logger.info(String.format("Moving %s from %s to %s.", localAddress, getLocalToken(), newToken));
 
         IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
 
         Map<String, Multimap<InetAddress, Range<Token>>> rangesToFetch = new HashMap<String, Multimap<InetAddress, Range<Token>>>();
         Map<String, Multimap<Range<Token>, InetAddress>> rangesToStreamByTable = new HashMap<String, Multimap<Range<Token>, InetAddress>>();
 
-        TokenMetadata tokenMetaClone = tokenMetadata_.cloneAfterAllSettled();
+        TokenMetadata tokenMetaClone = tokenMetadata.cloneAfterAllSettled();
 
         // for each of the non system tables calculating new ranges
         // which current node will handle after move to the new token
@@ -2278,11 +2278,11 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
             // getting collection of the currently used ranges by this keyspace
             Collection<Range<Token>> currentRanges = getRangesForEndpoint(table, localAddress);
             // collection of ranges which this node will serve after move to the new token
-            Collection<Range<Token>> updatedRanges = strategy.getPendingAddressRanges(tokenMetadata_, newToken, localAddress);
+            Collection<Range<Token>> updatedRanges = strategy.getPendingAddressRanges(tokenMetadata, newToken, localAddress);
 
             // ring ranges and endpoints associated with them
             // this used to determine what nodes should we ping about range data
-            Multimap<Range<Token>, InetAddress> rangeAddresses = strategy.getRangeAddresses(tokenMetadata_);
+            Multimap<Range<Token>, InetAddress> rangeAddresses = strategy.getRangeAddresses(tokenMetadata);
 
             // calculated parts of the ranges to request/stream from/to nodes in the ring
             Pair<Set<Range<Token>>, Set<Range<Token>>> rangesPerTable = calculateStreamAndFetchRanges(currentRanges, updatedRanges);
@@ -2311,7 +2311,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
             for (Range<Token> toStream : rangesPerTable.left)
             {
-                Set<InetAddress> currentEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(toStream.right, tokenMetadata_));
+                Set<InetAddress> currentEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(toStream.right, tokenMetadata));
                 Set<InetAddress> newEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(toStream.right, tokenMetaClone));
                 rangeWithEndpoints.putAll(toStream, Sets.difference(newEndpoints, currentEndpoints));
             }
@@ -2322,13 +2322,13 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
             Multimap<InetAddress, Range<Token>> workMap = RangeStreamer.getWorkMap(rangesToFetchWithPreferredEndpoints);
             rangesToFetch.put(table, workMap);
 
-            if (logger_.isDebugEnabled())
-                logger_.debug("Table {}: work map {}.", table, workMap);
+            if (logger.isDebugEnabled())
+                logger.debug("Table {}: work map {}.", table, workMap);
         }
 
         if (!rangesToStreamByTable.isEmpty() || !rangesToFetch.isEmpty())
         {
-            logger_.info("Sleeping {} ms before start streaming/fetching ranges.", RING_DELAY);
+            logger.info("Sleeping {} ms before start streaming/fetching ranges.", RING_DELAY);
 
             try
             {
@@ -2341,13 +2341,13 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
             setMode(Mode.MOVING, "fetching new ranges and streaming old ranges", true);
 
-            if (logger_.isDebugEnabled())
-                logger_.debug("[Move->STREAMING] Work Map: " + rangesToStreamByTable);
+            if (logger.isDebugEnabled())
+                logger.debug("[Move->STREAMING] Work Map: " + rangesToStreamByTable);
 
             CountDownLatch streamLatch = streamRanges(rangesToStreamByTable);
 
-            if (logger_.isDebugEnabled())
-                logger_.debug("[Move->FETCHING] Work Map: " + rangesToFetch);
+            if (logger.isDebugEnabled())
+                logger.debug("[Move->FETCHING] Work Map: " + rangesToFetch);
 
             CountDownLatch fetchLatch = requestRanges(rangesToFetch);
 
@@ -2364,8 +2364,8 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
         setToken(newToken); // setting new token as we have everything settled
 
-        if (logger_.isDebugEnabled())
-            logger_.debug("Successfully moved to new token {}", getLocalToken());
+        if (logger.isDebugEnabled())
+            logger.debug("Successfully moved to new token {}", getLocalToken());
     }
 
     /**
@@ -2377,7 +2377,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
             return "No token removals in process.";
         }
         return String.format("Removing token (%s). Waiting for replication confirmation from [%s].",
-                             tokenMetadata_.getToken(removingNode),
+                             tokenMetadata.getToken(removingNode),
                              StringUtils.join(replicatingNodes, ","));
     }
 
@@ -2388,12 +2388,12 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
      */
     public void forceRemoveCompletion()
     {
-        if (!replicatingNodes.isEmpty()  || !tokenMetadata_.getLeavingEndpoints().isEmpty())
+        if (!replicatingNodes.isEmpty()  || !tokenMetadata.getLeavingEndpoints().isEmpty())
         {
-            logger_.warn("Removal not confirmed for for " + StringUtils.join(this.replicatingNodes, ","));
-            for (InetAddress endpoint : tokenMetadata_.getLeavingEndpoints())
+            logger.warn("Removal not confirmed for for " + StringUtils.join(this.replicatingNodes, ","));
+            for (InetAddress endpoint : tokenMetadata.getLeavingEndpoints())
             {
-                Token token = tokenMetadata_.getToken(endpoint);
+                Token token = tokenMetadata.getToken(endpoint);
                 Gossiper.instance.advertiseTokenRemoved(endpoint, token);
                 excise(token, endpoint);
             }
@@ -2418,9 +2418,9 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
     public void removeToken(String tokenString)
     {
         InetAddress myAddress = FBUtilities.getBroadcastAddress();
-        Token localToken = tokenMetadata_.getToken(myAddress);
+        Token localToken = tokenMetadata.getToken(myAddress);
         Token token = getPartitioner().getTokenFactory().fromString(tokenString);
-        InetAddress endpoint = tokenMetadata_.getEndpoint(token);
+        InetAddress endpoint = tokenMetadata.getEndpoint(token);
 
         if (endpoint == null)
             throw new UnsupportedOperationException("Token not found.");
@@ -2432,8 +2432,8 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
             throw new UnsupportedOperationException("Node " + endpoint + " is alive and owns this token. Use decommission command to remove it from the ring");
 
         // A leaving endpoint that is dead is already being removed.
-        if (tokenMetadata_.isLeaving(endpoint))
-            logger_.warn("Node " + endpoint + " is already being removed, continuing removal anyway");
+        if (tokenMetadata.isLeaving(endpoint))
+            logger.warn("Node " + endpoint + " is already being removed, continuing removal anyway");
 
         if (!replicatingNodes.isEmpty())
             throw new UnsupportedOperationException("This node is already processing a removal. Wait for it to complete, or use 'removetoken force' if this has failed.");
@@ -2454,12 +2454,12 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
                 if (failureDetector.isAlive(ep))
                     replicatingNodes.add(ep);
                 else
-                    logger_.warn("Endpoint " + ep + " is down and will not receive data for re-replication of " + endpoint);
+                    logger.warn("Endpoint " + ep + " is down and will not receive data for re-replication of " + endpoint);
             }
         }
         removingNode = endpoint;
 
-        tokenMetadata_.addLeavingEndpoint(endpoint);
+        tokenMetadata.addLeavingEndpoint(endpoint);
         calculatePendingRanges();
         // the gossiper will handle spoofing this node's state to REMOVING_TOKEN for us
         // we add our own token so other nodes to let us know when they're done
@@ -2501,7 +2501,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         }
         else
         {
-            logger_.info("Received unexpected REPLICATION_FINISHED message from " + node
+            logger.info("Received unexpected REPLICATION_FINISHED message from " + node
                          + ". Was this node recently a removal coordinator?");
         }
     }
@@ -2515,7 +2515,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
     {
         if (hasUnreclaimedSpace())
         {
-            logger_.info("requesting GC to free disk space");
+            logger.info("requesting GC to free disk space");
             System.gc();
             try
             {
@@ -2559,7 +2559,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         ExecutorService mutationStage = StageManager.getStage(Stage.MUTATION);
         if (mutationStage.isTerminated())
         {
-            logger_.warn("Cannot drain node (did it already happen?)");
+            logger.warn("Cannot drain node (did it already happen?)");
             return;
         }
         setMode(Mode.DRAINING, "starting drain process", true);
@@ -2600,7 +2600,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         // wait for miscellaneous tasks like sstable and commitlog segment deletion
         tasks.shutdown();
         if (!tasks.awaitTermination(1, TimeUnit.MINUTES))
-            logger_.warn("Miscellaneous task executor still busy after one minute; proceeding with shutdown");
+            logger.warn("Miscellaneous task executor still busy after one minute; proceeding with shutdown");
 
         setMode(Mode.DRAINED, true);
     }
@@ -2616,8 +2616,8 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
     TokenMetadata setTokenMetadataUnsafe(TokenMetadata tmd)
     {
-        TokenMetadata old = tokenMetadata_;
-        tokenMetadata_ = tmd;
+        TokenMetadata old = tokenMetadata;
+        tokenMetadata = tmd;
         return old;
     }
 
@@ -2628,7 +2628,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
     public Map<String, Float> getOwnership()
     {
-        List<Token> sortedTokens = new ArrayList<Token>(tokenMetadata_.getTokenToEndpointMapForReading().keySet());
+        List<Token> sortedTokens = new ArrayList<Token>(tokenMetadata.getTokenToEndpointMapForReading().keySet());
         Collections.sort(sortedTokens);
         Map<Token, Float> token_map = getPartitioner().describeOwnership(sortedTokens);
         Map<String, Float> string_map = new HashMap<String, Float>();
@@ -2650,13 +2650,13 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         if (keyspace == null)
             keyspace = Schema.instance.getNonSystemTables().get(0);
         
-        List<Token> sortedTokens = new ArrayList<Token>(tokenMetadata_.getTokenToEndpointMapForReading().keySet());
+        List<Token> sortedTokens = new ArrayList<Token>(tokenMetadata.getTokenToEndpointMapForReading().keySet());
         Collections.sort(sortedTokens);
         Map<Token, Float> ownership = getPartitioner().describeOwnership(sortedTokens);
         
         for (Entry<InetAddress, Collection<Range<Token>>> ranges : constructEndpointToRangeMap(keyspace).entrySet())
         {
-            Token token = tokenMetadata_.getToken(ranges.getKey());
+            Token token = tokenMetadata.getToken(ranges.getKey());
             for (Range<Token> range: ranges.getValue())
             {
                 float value = effective.get(token.toString()) == null ? 0.0F : effective.get(token.toString());
@@ -2725,17 +2725,17 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
             if (total > 0 && (largest == null || total > largest.getTotalMemtableLiveSize()))
             {
-                logger_.debug(total + " estimated memtable size for " + cfs);
+                logger.debug(total + " estimated memtable size for " + cfs);
                 largest = cfs;
             }
         }
         if (largest == null)
         {
-            logger_.info("Unable to reduce heap usage since there are no dirty column families");
+            logger.info("Unable to reduce heap usage since there are no dirty column families");
             return;
         }
 
-        logger_.warn("Flushing " + largest + " to relieve memory pressure");
+        logger.warn("Flushing " + largest + " to relieve memory pressure");
         largest.forceFlush();
     }
 
@@ -2831,8 +2831,8 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
                     }
                 };
 
-                if (logger_.isDebugEnabled())
-                    logger_.debug("Requesting from " + source + " ranges " + StringUtils.join(toFetch, ", "));
+                if (logger.isDebugEnabled())
+                    logger.debug("Requesting from " + source + " ranges " + StringUtils.join(toFetch, ", "));
 
                 // sending actual request
                 StreamIn.requestRanges(source, table, toFetch, callback, OperationType.BOOTSTRAP);
@@ -2925,8 +2925,8 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
         SSTableLoader.OutputHandler oh = new SSTableLoader.OutputHandler()
         {
-            public void output(String msg) { logger_.info(msg); }
-            public void debug(String msg) { logger_.debug(msg); }
+            public void output(String msg) { logger.info(msg); }
+            public void debug(String msg) { logger.debug(msg); }
         };
 
         SSTableLoader loader = new SSTableLoader(dir, client, oh);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/910b663b/src/java/org/apache/cassandra/streaming/PendingFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/PendingFile.java b/src/java/org/apache/cassandra/streaming/PendingFile.java
index 50e0921..46ccdc4 100644
--- a/src/java/org/apache/cassandra/streaming/PendingFile.java
+++ b/src/java/org/apache/cassandra/streaming/PendingFile.java
@@ -36,11 +36,11 @@ import org.apache.cassandra.utils.Pair;
  */
 public class PendingFile
 {
-    private static PendingFileSerializer serializer_ = new PendingFileSerializer();
+    private static PendingFileSerializer serializer = new PendingFileSerializer();
 
     public static PendingFileSerializer serializer()
     {
-        return serializer_;
+        return serializer;
     }
 
     // NB: this reference is used to be able to release the acquired reference upon completion

http://git-wip-us.apache.org/repos/asf/cassandra/blob/910b663b/src/java/org/apache/cassandra/streaming/StreamRequestMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamRequestMessage.java b/src/java/org/apache/cassandra/streaming/StreamRequestMessage.java
index fc913c1..7da544f 100644
--- a/src/java/org/apache/cassandra/streaming/StreamRequestMessage.java
+++ b/src/java/org/apache/cassandra/streaming/StreamRequestMessage.java
@@ -50,15 +50,15 @@ import org.apache.cassandra.utils.FBUtilities;
 */
 class StreamRequestMessage implements MessageProducer
 {
-    private static IVersionedSerializer<StreamRequestMessage> serializer_;
+    private static IVersionedSerializer<StreamRequestMessage> serializer;
     static
     {
-        serializer_ = new StreamRequestMessageSerializer();
+        serializer = new StreamRequestMessageSerializer();
     }
 
     protected static IVersionedSerializer<StreamRequestMessage> serializer()
     {
-        return serializer_;
+        return serializer;
     }
 
     protected final long sessionId;


Mime
View raw message