cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jbel...@apache.org
Subject [11/15] git commit: Automated refactor: move StorageService.Verb into MessagingService
Date Tue, 08 May 2012 17:56:15 GMT
Automated refactor: move StorageService.Verb into MessagingService


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/021ec71b
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/021ec71b
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/021ec71b

Branch: refs/heads/trunk
Commit: 021ec71be5c32788347cee077034df690a279a34
Parents: 5d46a01
Author: Jonathan Ellis <jbellis@apache.org>
Authored: Mon Mar 26 17:49:56 2012 -0500
Committer: Jonathan Ellis <jbellis@apache.org>
Committed: Tue May 8 00:13:41 2012 -0500

----------------------------------------------------------------------
 .../org/apache/cassandra/db/CounterMutation.java   |    4 +-
 .../org/apache/cassandra/db/IndexScanCommand.java  |    4 +-
 .../cassandra/db/MigrationRequestVerbHandler.java  |    3 +-
 .../org/apache/cassandra/db/RangeSliceCommand.java |    3 +-
 .../org/apache/cassandra/db/RangeSliceReply.java   |    4 +-
 src/java/org/apache/cassandra/db/ReadCommand.java  |    4 +-
 .../org/apache/cassandra/db/ReadVerbHandler.java   |    2 +-
 src/java/org/apache/cassandra/db/RowMutation.java  |    5 +-
 .../cassandra/db/RowMutationVerbHandler.java       |    3 +-
 .../cassandra/db/SchemaCheckVerbHandler.java       |    3 +-
 .../org/apache/cassandra/db/SnapshotCommand.java   |    4 +-
 .../org/apache/cassandra/db/TruncateResponse.java  |    4 +-
 src/java/org/apache/cassandra/db/Truncation.java   |    4 +-
 .../org/apache/cassandra/db/WriteResponse.java     |    4 +-
 .../org/apache/cassandra/dht/BootStrapper.java     |    4 +-
 .../cassandra/gms/GossipDigestAckVerbHandler.java  |    3 +-
 .../cassandra/gms/GossipDigestSynVerbHandler.java  |    3 +-
 src/java/org/apache/cassandra/gms/Gossiper.java    |    4 +-
 src/java/org/apache/cassandra/net/Header.java      |   11 +-
 .../apache/cassandra/net/MessageDeliveryTask.java  |    3 +-
 src/java/org/apache/cassandra/net/MessageIn.java   |    7 +-
 src/java/org/apache/cassandra/net/MessageOut.java  |   11 +-
 .../org/apache/cassandra/net/MessagingService.java |  118 +++++++++++---
 .../cassandra/net/OutboundTcpConnection.java       |    3 +-
 .../cassandra/service/AbstractRowResolver.java     |    3 +-
 .../cassandra/service/AntiEntropyService.java      |    4 +-
 .../apache/cassandra/service/MigrationManager.java |    2 +-
 .../apache/cassandra/service/MigrationTask.java    |    2 +-
 .../cassandra/service/RowRepairResolver.java       |    2 +-
 .../cassandra/service/SnapshotVerbHandler.java     |    2 +-
 .../org/apache/cassandra/service/StorageProxy.java |   14 +-
 .../apache/cassandra/service/StorageService.java   |  133 +++-----------
 .../apache/cassandra/streaming/FileStreamTask.java |    3 +-
 .../streaming/ReplicationFinishedVerbHandler.java  |    2 +-
 .../apache/cassandra/streaming/StreamReply.java    |    4 +-
 .../cassandra/streaming/StreamRequestMessage.java  |    3 +-
 .../cassandra/streaming/StreamingRepairTask.java   |    5 +-
 .../org/apache/cassandra/service/RemoveTest.java   |    4 +-
 38 files changed, 193 insertions(+), 208 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/db/CounterMutation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/CounterMutation.java b/src/java/org/apache/cassandra/db/CounterMutation.java
index d3ef1bc..123357e 100644
--- a/src/java/org/apache/cassandra/db/CounterMutation.java
+++ b/src/java/org/apache/cassandra/db/CounterMutation.java
@@ -31,7 +31,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.db.filter.QueryPath;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.thrift.ConsistencyLevel;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
@@ -116,7 +116,7 @@ public class CounterMutation implements IMutation
 
     public MessageOut<CounterMutation> makeMutationMessage() throws IOException
     {
-        return new MessageOut<CounterMutation>(StorageService.Verb.COUNTER_MUTATION, this, serializer);
+        return new MessageOut<CounterMutation>(MessagingService.Verb.COUNTER_MUTATION, this, serializer);
     }
 
     public boolean shouldReplicateOnWrite()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/db/IndexScanCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/IndexScanCommand.java b/src/java/org/apache/cassandra/db/IndexScanCommand.java
index 2522f3a..2f14849 100644
--- a/src/java/org/apache/cassandra/db/IndexScanCommand.java
+++ b/src/java/org/apache/cassandra/db/IndexScanCommand.java
@@ -27,7 +27,7 @@ import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.FastByteArrayInputStream;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.thrift.IndexClause;
 import org.apache.cassandra.thrift.SlicePredicate;
 import org.apache.cassandra.thrift.TBinaryProtocol;
@@ -57,7 +57,7 @@ public class IndexScanCommand
 
     public MessageOut<IndexScanCommand> createMessage()
     {
-        return new MessageOut<IndexScanCommand>(StorageService.Verb.INDEX_SCAN, this, serializer);
+        return new MessageOut<IndexScanCommand>(MessagingService.Verb.INDEX_SCAN, this, serializer);
     }
 
     public static IndexScanCommand read(MessageIn message) throws IOException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java b/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
index 60e660e..41650fa 100644
--- a/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
@@ -27,7 +27,6 @@ import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.service.StorageService;
 
 /**
  * Sends it's current schema state in form of row mutations in reply to the remote node's request.
@@ -40,7 +39,7 @@ public class MigrationRequestVerbHandler implements IVerbHandler
     public void doVerb(MessageIn message, String id)
     {
         logger.debug("Received migration request from {}.", message.getFrom());
-        MessageOut<Collection<RowMutation>> response = new MessageOut<Collection<RowMutation>>(StorageService.Verb.INTERNAL_RESPONSE,
+        MessageOut<Collection<RowMutation>> response = new MessageOut<Collection<RowMutation>>(MessagingService.Verb.INTERNAL_RESPONSE,
                                                                                                SystemTable.serializeSchema(),
                                                                                                MigrationManager.MigrationsSerializer.instance);
         MessagingService.instance().sendReply(response, id, message.getFrom());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/db/RangeSliceCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/RangeSliceCommand.java b/src/java/org/apache/cassandra/db/RangeSliceCommand.java
index 749c735..528de5c 100644
--- a/src/java/org/apache/cassandra/db/RangeSliceCommand.java
+++ b/src/java/org/apache/cassandra/db/RangeSliceCommand.java
@@ -50,7 +50,6 @@ import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.IReadCommand;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.thrift.ColumnParent;
 import org.apache.cassandra.thrift.IndexExpression;
 import org.apache.cassandra.thrift.SlicePredicate;
@@ -117,7 +116,7 @@ public class RangeSliceCommand implements IReadCommand
 
     public MessageOut<RangeSliceCommand> createMessage()
     {
-        return new MessageOut<RangeSliceCommand>(StorageService.Verb.RANGE_SLICE, this, serializer);
+        return new MessageOut<RangeSliceCommand>(MessagingService.Verb.RANGE_SLICE, this, serializer);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/db/RangeSliceReply.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/RangeSliceReply.java b/src/java/org/apache/cassandra/db/RangeSliceReply.java
index ec82216..d112bf6 100644
--- a/src/java/org/apache/cassandra/db/RangeSliceReply.java
+++ b/src/java/org/apache/cassandra/db/RangeSliceReply.java
@@ -29,7 +29,7 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.FastByteArrayInputStream;
 import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.net.MessagingService;
 
 public class RangeSliceReply
 {
@@ -44,7 +44,7 @@ public class RangeSliceReply
 
     public MessageOut<RangeSliceReply> createMessage()
     {
-        return new MessageOut<RangeSliceReply>(StorageService.Verb.REQUEST_RESPONSE, this, serializer);
+        return new MessageOut<RangeSliceReply>(MessagingService.Verb.REQUEST_RESPONSE, this, serializer);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/db/ReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java
index 17d9998..d994362 100644
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@ -28,9 +28,9 @@ import org.apache.cassandra.db.filter.QueryPath;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.IReadCommand;
 import org.apache.cassandra.service.RepairCallback;
-import org.apache.cassandra.service.StorageService;
 
 
 public abstract class ReadCommand implements IReadCommand
@@ -47,7 +47,7 @@ public abstract class ReadCommand implements IReadCommand
 
     public MessageOut<ReadCommand> createMessage()
     {
-        return new MessageOut<ReadCommand>(StorageService.Verb.READ, this, serializer);
+        return new MessageOut<ReadCommand>(MessagingService.Verb.READ, this, serializer);
     }
 
     public final QueryPath queryPath;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/db/ReadVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ReadVerbHandler.java b/src/java/org/apache/cassandra/db/ReadVerbHandler.java
index e8f5c10..7335d51 100644
--- a/src/java/org/apache/cassandra/db/ReadVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/ReadVerbHandler.java
@@ -49,7 +49,7 @@ public class ReadVerbHandler implements IVerbHandler
             Table table = Table.open(command.table);
             Row row = command.getRow(table);
 
-            MessageOut<ReadResponse> reply = new MessageOut<ReadResponse>(StorageService.Verb.REQUEST_RESPONSE,
+            MessageOut<ReadResponse> reply = new MessageOut<ReadResponse>(MessagingService.Verb.REQUEST_RESPONSE,
                                                                           getResponse(command, row),
                                                                           ReadResponse.serializer());
             if (logger.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/db/RowMutation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/RowMutation.java b/src/java/org/apache/cassandra/db/RowMutation.java
index e629107..a0319b6 100644
--- a/src/java/org/apache/cassandra/db/RowMutation.java
+++ b/src/java/org/apache/cassandra/db/RowMutation.java
@@ -35,7 +35,6 @@ import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.FastByteArrayInputStream;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.thrift.ColumnOrSuperColumn;
 import org.apache.cassandra.thrift.Deletion;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -298,10 +297,10 @@ public class RowMutation implements IMutation
 
     public MessageOut<RowMutation> createMessage()
     {
-        return createMessage(StorageService.Verb.MUTATION);
+        return createMessage(MessagingService.Verb.MUTATION);
     }
 
-    public MessageOut<RowMutation> createMessage(StorageService.Verb verb)
+    public MessageOut<RowMutation> createMessage(MessagingService.Verb verb)
     {
         return new MessageOut<RowMutation>(verb, this, serializer);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java b/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java
index c9052c5..39e5ed3 100644
--- a/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java
@@ -26,7 +26,6 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.io.util.FastByteArrayInputStream;
 import org.apache.cassandra.net.*;
-import org.apache.cassandra.service.StorageService;
 
 public class RowMutationVerbHandler implements IVerbHandler
 {
@@ -70,7 +69,7 @@ public class RowMutationVerbHandler implements IVerbHandler
      * Older version (< 1.0) will not send this message at all, hence we don't
      * need to check the version of the data.
      */
-    private void forwardToLocalNodes(RowMutation rm, StorageService.Verb verb, byte[] forwardBytes, InetAddress from) throws IOException
+    private void forwardToLocalNodes(RowMutation rm, MessagingService.Verb verb, byte[] forwardBytes, InetAddress from) throws IOException
     {
         DataInputStream dis = new DataInputStream(new FastByteArrayInputStream(forwardBytes));
         int size = dis.readInt();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/db/SchemaCheckVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SchemaCheckVerbHandler.java b/src/java/org/apache/cassandra/db/SchemaCheckVerbHandler.java
index 64e4080..0003d82 100644
--- a/src/java/org/apache/cassandra/db/SchemaCheckVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/SchemaCheckVerbHandler.java
@@ -24,7 +24,6 @@ import org.apache.cassandra.net.IVerbHandler;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.service.StorageService;
 
 
 public class SchemaCheckVerbHandler implements IVerbHandler
@@ -34,7 +33,7 @@ public class SchemaCheckVerbHandler implements IVerbHandler
     public void doVerb(MessageIn message, String id)
     {
         logger.debug("Received schema check request.");
-        MessageOut response = new MessageOut(StorageService.Verb.INTERNAL_RESPONSE);
+        MessageOut response = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE);
         MessagingService.instance().sendReply(response, id, message.getFrom());
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/db/SnapshotCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SnapshotCommand.java b/src/java/org/apache/cassandra/db/SnapshotCommand.java
index b92ee5e..5463298 100644
--- a/src/java/org/apache/cassandra/db/SnapshotCommand.java
+++ b/src/java/org/apache/cassandra/db/SnapshotCommand.java
@@ -26,7 +26,7 @@ import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.FastByteArrayInputStream;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.net.MessagingService;
 
 public class SnapshotCommand
 {
@@ -47,7 +47,7 @@ public class SnapshotCommand
 
     public MessageOut createMessage()
     {
-        return new MessageOut<SnapshotCommand>(StorageService.Verb.SNAPSHOT, this, serializer);
+        return new MessageOut<SnapshotCommand>(MessagingService.Verb.SNAPSHOT, this, serializer);
     }
 
     public static SnapshotCommand read(MessageIn message) throws IOException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/db/TruncateResponse.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/TruncateResponse.java b/src/java/org/apache/cassandra/db/TruncateResponse.java
index 45f368f..eb5cb08 100644
--- a/src/java/org/apache/cassandra/db/TruncateResponse.java
+++ b/src/java/org/apache/cassandra/db/TruncateResponse.java
@@ -23,7 +23,7 @@ import java.io.IOException;
 
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.net.MessagingService;
 
 
 /**
@@ -51,7 +51,7 @@ public class TruncateResponse
 
     public MessageOut<TruncateResponse> createMessage()
     {
-        return new MessageOut<TruncateResponse>(StorageService.Verb.REQUEST_RESPONSE, this, serializer);
+        return new MessageOut<TruncateResponse>(MessagingService.Verb.REQUEST_RESPONSE, this, serializer);
     }
 
     public static class TruncateResponseSerializer implements IVersionedSerializer<TruncateResponse>

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/db/Truncation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Truncation.java b/src/java/org/apache/cassandra/db/Truncation.java
index 6dafc31..afb115b 100644
--- a/src/java/org/apache/cassandra/db/Truncation.java
+++ b/src/java/org/apache/cassandra/db/Truncation.java
@@ -23,7 +23,7 @@ import java.io.IOException;
 
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.net.MessagingService;
 
 /**
  * A truncate operation descriptor
@@ -53,7 +53,7 @@ public class Truncation
 
     public MessageOut<Truncation> createMessage()
     {
-        return new MessageOut<Truncation>(StorageService.Verb.TRUNCATE, this, serializer);
+        return new MessageOut<Truncation>(MessagingService.Verb.TRUNCATE, this, serializer);
     }
 
     public String toString()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/db/WriteResponse.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/WriteResponse.java b/src/java/org/apache/cassandra/db/WriteResponse.java
index ef03f6a..426e55c 100644
--- a/src/java/org/apache/cassandra/db/WriteResponse.java
+++ b/src/java/org/apache/cassandra/db/WriteResponse.java
@@ -24,7 +24,7 @@ import java.nio.ByteBuffer;
 
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -45,7 +45,7 @@ public class WriteResponse
 
     public MessageOut<WriteResponse> createMessage()
     {
-        return new MessageOut<WriteResponse>(StorageService.Verb.REQUEST_RESPONSE, this, serializer);
+        return new MessageOut<WriteResponse>(MessagingService.Verb.REQUEST_RESPONSE, this, serializer);
     }
 
     private final String table;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/dht/BootStrapper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/BootStrapper.java b/src/java/org/apache/cassandra/dht/BootStrapper.java
index bfe80c8..e31f908 100644
--- a/src/java/org/apache/cassandra/dht/BootStrapper.java
+++ b/src/java/org/apache/cassandra/dht/BootStrapper.java
@@ -154,7 +154,7 @@ public class BootStrapper
 
     static Token<?> getBootstrapTokenFrom(InetAddress maxEndpoint)
     {
-        MessageOut message = new MessageOut(StorageService.Verb.BOOTSTRAP_TOKEN);
+        MessageOut message = new MessageOut(MessagingService.Verb.BOOTSTRAP_TOKEN);
         int retries = 5;
         long timeout = Math.max(MessagingService.getDefaultCallbackTimeout(), BOOTSTRAP_TIMEOUT);
 
@@ -177,7 +177,7 @@ public class BootStrapper
         {
             StorageService ss = StorageService.instance;
             String tokenString = StorageService.getPartitioner().getTokenFactory().toString(ss.getBootstrapToken());
-            MessageOut<String> response = new MessageOut<String>(StorageService.Verb.INTERNAL_RESPONSE, tokenString, StringSerializer.instance);
+            MessageOut<String> response = new MessageOut<String>(MessagingService.Verb.INTERNAL_RESPONSE, tokenString, StringSerializer.instance);
             MessagingService.instance().sendReply(response, id, message.getFrom());
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java b/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java
index f747bdd..5c3e06b 100644
--- a/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java
+++ b/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java
@@ -32,7 +32,6 @@ import org.apache.cassandra.net.IVerbHandler;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.service.StorageService;
 
 public class GossipDigestAckVerbHandler implements IVerbHandler
 {
@@ -76,7 +75,7 @@ public class GossipDigestAckVerbHandler implements IVerbHandler
                     deltaEpStateMap.put(addr, localEpStatePtr);
             }
 
-            MessageOut<GossipDigestAck2Message> gDigestAck2Message = new MessageOut<GossipDigestAck2Message>(StorageService.Verb.GOSSIP_DIGEST_ACK2,
+            MessageOut<GossipDigestAck2Message> gDigestAck2Message = new MessageOut<GossipDigestAck2Message>(MessagingService.Verb.GOSSIP_DIGEST_ACK2,
                                                                                                              new GossipDigestAck2Message(deltaEpStateMap), 
                                                                                                              GossipDigestAck2Message.serializer());
             if (logger.isTraceEnabled())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java b/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java
index 2f45095..14bb324 100644
--- a/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java
+++ b/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java
@@ -31,7 +31,6 @@ import org.apache.cassandra.net.IVerbHandler;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.service.StorageService;
 
 public class GossipDigestSynVerbHandler implements IVerbHandler
 {
@@ -82,7 +81,7 @@ public class GossipDigestSynVerbHandler implements IVerbHandler
             Map<InetAddress, EndpointState> deltaEpStateMap = new HashMap<InetAddress, EndpointState>();
             Gossiper.instance.examineGossiper(gDigestList, deltaGossipDigestList, deltaEpStateMap);
 
-            MessageOut<GossipDigestAckMessage> gDigestAckMessage = new MessageOut<GossipDigestAckMessage>(StorageService.Verb.GOSSIP_DIGEST_ACK,
+            MessageOut<GossipDigestAckMessage> gDigestAckMessage = new MessageOut<GossipDigestAckMessage>(MessagingService.Verb.GOSSIP_DIGEST_ACK,
                                                                                                           new GossipDigestAckMessage(deltaGossipDigestList, deltaEpStateMap),
                                                                                                           GossipDigestAckMessage.serializer());
             if (logger.isTraceEnabled())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/gms/Gossiper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java b/src/java/org/apache/cassandra/gms/Gossiper.java
index 4ee8358..f19291c 100644
--- a/src/java/org/apache/cassandra/gms/Gossiper.java
+++ b/src/java/org/apache/cassandra/gms/Gossiper.java
@@ -123,7 +123,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
                 if ( gDigests.size() > 0 )
                 {
                     GossipDigestSynMessage digestSynMessage = new GossipDigestSynMessage(DatabaseDescriptor.getClusterName(), gDigests);
-                    MessageOut<GossipDigestSynMessage> message = new MessageOut<GossipDigestSynMessage>(StorageService.Verb.GOSSIP_DIGEST_SYN,
+                    MessageOut<GossipDigestSynMessage> message = new MessageOut<GossipDigestSynMessage>(MessagingService.Verb.GOSSIP_DIGEST_SYN,
                                                                                                         digestSynMessage,
                                                                                                         GossipDigestSynMessage.serializer());
                     /* Gossip to some random live member */
@@ -1090,7 +1090,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
         {
             throw new RuntimeException(e);
         }
-        MessageOut<GossipDigestSynMessage> message = new MessageOut<GossipDigestSynMessage>(StorageService.Verb.GOSSIP_SHUTDOWN,
+        MessageOut<GossipDigestSynMessage> message = new MessageOut<GossipDigestSynMessage>(MessagingService.Verb.GOSSIP_SHUTDOWN,
                                                                                             null,
                                                                                             GossipDigestSynMessage.serializer());
         for (InetAddress ep : liveEndpoints)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/net/Header.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/Header.java b/src/java/org/apache/cassandra/net/Header.java
index e5c9232..2b4c2b2 100644
--- a/src/java/org/apache/cassandra/net/Header.java
+++ b/src/java/org/apache/cassandra/net/Header.java
@@ -24,7 +24,6 @@ import java.util.Hashtable;
 import java.util.Map;
 
 import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 
 import com.google.common.collect.ImmutableMap;
@@ -48,15 +47,15 @@ public class Header
     // is not necessarily the same as the node that forwards us the request (see StorageProxy.sendMessages
     // and RowMutationVerbHandler.forwardToLocalNodes)
     private final InetAddress from;
-    private final StorageService.Verb verb;
+    private final MessagingService.Verb verb;
     protected final Map<String, byte[]> details;
 
-    Header(InetAddress from, StorageService.Verb verb)
+    Header(InetAddress from, MessagingService.Verb verb)
     {
         this(from, verb, Collections.<String, byte[]>emptyMap());
     }
 
-    Header(InetAddress from, StorageService.Verb verb, Map<String, byte[]> details)
+    Header(InetAddress from, MessagingService.Verb verb, Map<String, byte[]> details)
     {
         assert from != null;
         assert verb != null;
@@ -71,7 +70,7 @@ public class Header
         return from;
     }
 
-    StorageService.Verb getVerb()
+    MessagingService.Verb getVerb()
     {
         return verb;
     }
@@ -143,7 +142,7 @@ class HeaderSerializer implements IVersionedSerializer<Header>
             dis.readFully(bytes);
             details.put(key, bytes);
         }
-        return new Header(from, StorageService.VERBS[verbOrdinal], details);
+        return new Header(from, MessagingService.VERBS[verbOrdinal], details);
     }
 
     public long serializedSize(Header header, int version)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java
index 2a947e8..b0603ff 100644
--- a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java
+++ b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java
@@ -21,7 +21,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.service.StorageService;
 
 public class MessageDeliveryTask implements Runnable
 {
@@ -40,7 +39,7 @@ public class MessageDeliveryTask implements Runnable
 
     public void run()
     {
-        StorageService.Verb verb = message.getVerb();
+        MessagingService.Verb verb = message.getVerb();
         if (MessagingService.DROPPABLE_VERBS.contains(verb)
             && System.currentTimeMillis() > constructionTime + DatabaseDescriptor.getRpcTimeout())
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/net/MessageIn.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/MessageIn.java b/src/java/org/apache/cassandra/net/MessageIn.java
index 885b7ef..f326802d 100644
--- a/src/java/org/apache/cassandra/net/MessageIn.java
+++ b/src/java/org/apache/cassandra/net/MessageIn.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.net;
 import java.net.InetAddress;
 
 import org.apache.cassandra.concurrent.Stage;
-import org.apache.cassandra.service.StorageService;
 
 public class MessageIn
 {
@@ -38,7 +37,7 @@ public class MessageIn
         this.version = version;
     }
 
-    public MessageIn(InetAddress from, StorageService.Verb verb, byte[] body, int version)
+    public MessageIn(InetAddress from, MessagingService.Verb verb, byte[] body, int version)
     {
         this(new Header(from, verb), body, version);
     }
@@ -65,10 +64,10 @@ public class MessageIn
 
     public Stage getMessageType()
     {
-        return StorageService.verbStages.get(getVerb());
+        return MessagingService.verbStages.get(getVerb());
     }
 
-    public StorageService.Verb getVerb()
+    public MessagingService.Verb getVerb()
     {
         return header.getVerb();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/net/MessageOut.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/MessageOut.java b/src/java/org/apache/cassandra/net/MessageOut.java
index 42c80ad..510223d 100644
--- a/src/java/org/apache/cassandra/net/MessageOut.java
+++ b/src/java/org/apache/cassandra/net/MessageOut.java
@@ -27,28 +27,27 @@ import com.google.common.collect.ImmutableMap;
 
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 
 public class MessageOut<T>
 {
-    public final StorageService.Verb verb;
+    public final MessagingService.Verb verb;
     public final T payload;
     public final IVersionedSerializer<T> serializer;
     public final Map<String, byte[]> parameters;
 
     // we do support messages that just consist of a verb
-    public MessageOut(StorageService.Verb verb)
+    public MessageOut(MessagingService.Verb verb)
     {
         this(verb, null, null);
     }
 
-    public MessageOut(StorageService.Verb verb, T payload, IVersionedSerializer<T> serializer)
+    public MessageOut(MessagingService.Verb verb, T payload, IVersionedSerializer<T> serializer)
     {
         this(verb, payload, serializer, Collections.<String, byte[]>emptyMap());
     }
 
-    public MessageOut(StorageService.Verb verb, T payload, IVersionedSerializer<T> serializer, Map<String, byte[]> parameters)
+    public MessageOut(MessagingService.Verb verb, T payload, IVersionedSerializer<T> serializer, Map<String, byte[]> parameters)
     {
         this.verb = verb;
         this.payload = payload;
@@ -76,7 +75,7 @@ public class MessageOut<T>
 
     public Stage getStage()
     {
-        return StorageService.verbStages.get(verb);
+        return MessagingService.verbStages.get(verb);
     }
 
     public String toString()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/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 05dc07f..649f81d 100644
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@ -75,11 +75,86 @@ public final class MessagingService implements MessagingServiceMBean
     /** we preface every message with this number so the recipient can validate the sender is sane */
     static final int PROTOCOL_MAGIC = 0xCA552DFA;
 
+    /* All verb handler identifiers */
+    public enum Verb
+    {
+        MUTATION,
+        BINARY, // Deprecated
+        READ_REPAIR,
+        READ,
+        REQUEST_RESPONSE, // client-initiated reads and writes
+        STREAM_INITIATE, // Deprecated
+        STREAM_INITIATE_DONE, // Deprecated
+        STREAM_REPLY,
+        STREAM_REQUEST,
+        RANGE_SLICE,
+        BOOTSTRAP_TOKEN,
+        TREE_REQUEST,
+        TREE_RESPONSE,
+        JOIN, // Deprecated
+        GOSSIP_DIGEST_SYN,
+        GOSSIP_DIGEST_ACK,
+        GOSSIP_DIGEST_ACK2,
+        DEFINITIONS_ANNOUNCE, // Deprecated
+        DEFINITIONS_UPDATE,
+        TRUNCATE,
+        SCHEMA_CHECK,
+        INDEX_SCAN, // Deprecated
+        REPLICATION_FINISHED,
+        INTERNAL_RESPONSE, // responses to internal calls
+        COUNTER_MUTATION,
+        STREAMING_REPAIR_REQUEST,
+        STREAMING_REPAIR_RESPONSE,
+        SNAPSHOT, // Similar to nt snapshot
+        MIGRATION_REQUEST,
+        GOSSIP_SHUTDOWN,
+        // use as padding for backwards compatability where a previous version needs to validate a verb from the future.
+        UNUSED_1,
+        UNUSED_2,
+        UNUSED_3,
+        ;
+        // remember to add new verbs at the end, since we serialize by ordinal
+    }
+    public static final Verb[] VERBS = Verb.values();
+
+    public static final EnumMap<MessagingService.Verb, Stage> verbStages = new EnumMap<MessagingService.Verb, Stage>(MessagingService.Verb.class)
+    {{
+        put(Verb.MUTATION, Stage.MUTATION);
+        put(Verb.BINARY, Stage.MUTATION);
+        put(Verb.READ_REPAIR, Stage.MUTATION);
+        put(Verb.TRUNCATE, Stage.MUTATION);
+        put(Verb.READ, Stage.READ);
+        put(Verb.REQUEST_RESPONSE, Stage.REQUEST_RESPONSE);
+        put(Verb.STREAM_REPLY, Stage.MISC); // TODO does this really belong on misc? I've just copied old behavior here
+        put(Verb.STREAM_REQUEST, Stage.STREAM);
+        put(Verb.RANGE_SLICE, Stage.READ);
+        put(Verb.BOOTSTRAP_TOKEN, Stage.MISC);
+        put(Verb.TREE_REQUEST, Stage.ANTI_ENTROPY);
+        put(Verb.TREE_RESPONSE, Stage.ANTI_ENTROPY);
+        put(Verb.STREAMING_REPAIR_REQUEST, Stage.ANTI_ENTROPY);
+        put(Verb.STREAMING_REPAIR_RESPONSE, Stage.ANTI_ENTROPY);
+        put(Verb.GOSSIP_DIGEST_ACK, Stage.GOSSIP);
+        put(Verb.GOSSIP_DIGEST_ACK2, Stage.GOSSIP);
+        put(Verb.GOSSIP_DIGEST_SYN, Stage.GOSSIP);
+        put(Verb.GOSSIP_SHUTDOWN, Stage.GOSSIP);
+        put(Verb.DEFINITIONS_UPDATE, Stage.MIGRATION);
+        put(Verb.SCHEMA_CHECK, Stage.MIGRATION);
+        put(Verb.MIGRATION_REQUEST, Stage.MIGRATION);
+        put(Verb.INDEX_SCAN, Stage.READ);
+        put(Verb.REPLICATION_FINISHED, Stage.MISC);
+        put(Verb.INTERNAL_RESPONSE, Stage.INTERNAL_RESPONSE);
+        put(Verb.COUNTER_MUTATION, Stage.MUTATION);
+        put(Verb.SNAPSHOT, Stage.MISC);
+        put(Verb.UNUSED_1, Stage.INTERNAL_RESPONSE);
+        put(Verb.UNUSED_2, Stage.INTERNAL_RESPONSE);
+        put(Verb.UNUSED_3, Stage.INTERNAL_RESPONSE);
+    }};
+
     /* This records all the results mapped by message Id */
     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<Verb, IVerbHandler> verbHandlers;
 
     /** One executor per destination InetAddress for streaming.
      *
@@ -109,18 +184,18 @@ public final class MessagingService implements MessagingServiceMBean
      * all correspond to client requests or something triggered by them; we don't want to
      * drop internal messages like bootstrap or repair notifications.
      */
-    public static final EnumSet<StorageService.Verb> DROPPABLE_VERBS = EnumSet.of(StorageService.Verb.BINARY,
-                                                                                  StorageService.Verb.MUTATION,
-                                                                                  StorageService.Verb.READ_REPAIR,
-                                                                                  StorageService.Verb.READ,
-                                                                                  StorageService.Verb.RANGE_SLICE,
-                                                                                  StorageService.Verb.REQUEST_RESPONSE);
+    public static final EnumSet<Verb> DROPPABLE_VERBS = EnumSet.of(Verb.BINARY,
+                                                                                  Verb.MUTATION,
+                                                                                  Verb.READ_REPAIR,
+                                                                                  Verb.READ,
+                                                                                  Verb.RANGE_SLICE,
+                                                                                  Verb.REQUEST_RESPONSE);
 
     // total dropped message counts for server lifetime
-    private final Map<StorageService.Verb, AtomicInteger> droppedMessages = new EnumMap<StorageService.Verb, AtomicInteger>(StorageService.Verb.class);
+    private final Map<Verb, AtomicInteger> droppedMessages = new EnumMap<Verb, AtomicInteger>(Verb.class);
     // dropped count when last requested for the Recent api.  high concurrency isn't necessary here.
-    private final Map<StorageService.Verb, Integer> lastDropped = Collections.synchronizedMap(new EnumMap<StorageService.Verb, Integer>(StorageService.Verb.class));
-    private final Map<StorageService.Verb, Integer> lastDroppedInternal = new EnumMap<StorageService.Verb, Integer>(StorageService.Verb.class);
+    private final Map<Verb, Integer> lastDropped = Collections.synchronizedMap(new EnumMap<Verb, Integer>(Verb.class));
+    private final Map<Verb, Integer> lastDroppedInternal = new EnumMap<Verb, Integer>(Verb.class);
 
     private long totalTimeouts = 0;
     private long recentTotalTimeouts = 0;
@@ -140,7 +215,7 @@ public final class MessagingService implements MessagingServiceMBean
 
     private MessagingService()
     {
-        for (StorageService.Verb verb : DROPPABLE_VERBS)
+        for (Verb verb : DROPPABLE_VERBS)
         {
             droppedMessages.put(verb, new AtomicInteger());
             lastDropped.put(verb, 0);
@@ -148,7 +223,7 @@ public final class MessagingService implements MessagingServiceMBean
         }
 
         listenGate = new SimpleCondition();
-        verbHandlers = new EnumMap<StorageService.Verb, IVerbHandler>(StorageService.Verb.class);
+        verbHandlers = new EnumMap<Verb, IVerbHandler>(Verb.class);
         Runnable logDropped = new Runnable()
         {
             public void run()
@@ -310,7 +385,7 @@ public final class MessagingService implements MessagingServiceMBean
      * @param verb
      * @param verbHandler handler for the specified verb
      */
-    public void registerVerbHandlers(StorageService.Verb verb, IVerbHandler verbHandler)
+    public void registerVerbHandlers(Verb verb, IVerbHandler verbHandler)
     {
         assert !verbHandlers.containsKey(verb);
         verbHandlers.put(verb, verbHandler);
@@ -322,7 +397,7 @@ public final class MessagingService implements MessagingServiceMBean
      * @param type for which the verb handler is sought
      * @return a reference to IVerbHandler which is the handler for the specified verb
      */
-    public IVerbHandler getVerbHandler(StorageService.Verb type)
+    public IVerbHandler getVerbHandler(Verb type)
     {
         return verbHandlers.get(type);
     }
@@ -338,7 +413,7 @@ public final class MessagingService implements MessagingServiceMBean
         CallbackInfo previous;
 
         // If HH is enabled and this is a mutation message => store the message to track for potential hints.
-        if (DatabaseDescriptor.hintedHandoffEnabled() && message.verb == StorageService.Verb.MUTATION)
+        if (DatabaseDescriptor.hintedHandoffEnabled() && message.verb == Verb.MUTATION)
             previous = callbacks.put(messageId, new CallbackInfo(to, cb, message), timeout);
         else
             previous = callbacks.put(messageId, new CallbackInfo(to, cb), timeout);
@@ -604,7 +679,7 @@ public final class MessagingService implements MessagingServiceMBean
         return buffer;
     }
 
-    public void incrementDroppedMessages(StorageService.Verb verb)
+    public void incrementDroppedMessages(Verb verb)
     {
         assert DROPPABLE_VERBS.contains(verb) : "Verb " + verb + " should not legally be dropped";
         droppedMessages.get(verb).incrementAndGet();
@@ -613,10 +688,10 @@ public final class MessagingService implements MessagingServiceMBean
     private void logDroppedMessages()
     {
         boolean logTpstats = false;
-        for (Map.Entry<StorageService.Verb, AtomicInteger> entry : droppedMessages.entrySet())
+        for (Map.Entry<Verb, AtomicInteger> entry : droppedMessages.entrySet())
         {
             AtomicInteger dropped = entry.getValue();
-            StorageService.Verb verb = entry.getKey();
+            Verb verb = entry.getKey();
             int recent = dropped.get() - lastDroppedInternal.get(verb);
             if (recent > 0)
             {
@@ -723,7 +798,7 @@ public final class MessagingService implements MessagingServiceMBean
     public Map<String, Integer> getDroppedMessages()
     {
         Map<String, Integer> map = new HashMap<String, Integer>();
-        for (Map.Entry<StorageService.Verb, AtomicInteger> entry : droppedMessages.entrySet())
+        for (Map.Entry<Verb, AtomicInteger> entry : droppedMessages.entrySet())
             map.put(entry.getKey().toString(), entry.getValue().get());
         return map;
     }
@@ -731,9 +806,9 @@ public final class MessagingService implements MessagingServiceMBean
     public Map<String, Integer> getRecentlyDroppedMessages()
     {
         Map<String, Integer> map = new HashMap<String, Integer>();
-        for (Map.Entry<StorageService.Verb, AtomicInteger> entry : droppedMessages.entrySet())
+        for (Map.Entry<Verb, AtomicInteger> entry : droppedMessages.entrySet())
         {
-            StorageService.Verb verb = entry.getKey();
+            Verb verb = entry.getKey();
             Integer dropped = entry.getValue().get();
             Integer recentlyDropped = dropped - lastDropped.get(verb);
             map.put(verb.toString(), recentlyDropped);
@@ -776,5 +851,4 @@ public final class MessagingService implements MessagingServiceMBean
         }
         return result;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/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 2aa75ef..72164d7 100644
--- a/src/java/org/apache/cassandra/net/OutboundTcpConnection.java
+++ b/src/java/org/apache/cassandra/net/OutboundTcpConnection.java
@@ -30,14 +30,13 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.gms.Gossiper;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 
 public class OutboundTcpConnection extends Thread
 {
     private static final Logger logger = LoggerFactory.getLogger(OutboundTcpConnection.class);
 
-    private static final MessageOut CLOSE_SENTINEL = new MessageOut(StorageService.Verb.INTERNAL_RESPONSE);
+    private static final MessageOut CLOSE_SENTINEL = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE);
 
     private static final int OPEN_RETRY_DELAY = 100; // ms between retries
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/service/AbstractRowResolver.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/AbstractRowResolver.java b/src/java/org/apache/cassandra/service/AbstractRowResolver.java
index 7dc7589..d4a3b65 100644
--- a/src/java/org/apache/cassandra/service/AbstractRowResolver.java
+++ b/src/java/org/apache/cassandra/service/AbstractRowResolver.java
@@ -32,6 +32,7 @@ import org.apache.cassandra.db.ReadResponse;
 import org.apache.cassandra.db.Row;
 import org.apache.cassandra.io.util.FastByteArrayInputStream;
 import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.FBUtilities;
 import org.cliffc.high_scale_lib.NonBlockingHashMap;
 
@@ -39,7 +40,7 @@ public abstract class AbstractRowResolver implements IResponseResolver<Row>
 {
     protected static final Logger logger = LoggerFactory.getLogger(AbstractRowResolver.class);
 
-    private static final MessageIn FAKE_MESSAGE = new MessageIn(FBUtilities.getBroadcastAddress(), StorageService.Verb.INTERNAL_RESPONSE, ArrayUtils.EMPTY_BYTE_ARRAY, -1);
+    private static final MessageIn FAKE_MESSAGE = new MessageIn(FBUtilities.getBroadcastAddress(), MessagingService.Verb.INTERNAL_RESPONSE, ArrayUtils.EMPTY_BYTE_ARRAY, -1);
 
     protected final String table;
     protected final ConcurrentMap<MessageIn, ReadResponse> replies = new NonBlockingHashMap<MessageIn, ReadResponse>();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/service/AntiEntropyService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/AntiEntropyService.java b/src/java/org/apache/cassandra/service/AntiEntropyService.java
index 0cce133..f9728ad 100644
--- a/src/java/org/apache/cassandra/service/AntiEntropyService.java
+++ b/src/java/org/apache/cassandra/service/AntiEntropyService.java
@@ -418,7 +418,7 @@ public class AntiEntropyService
 
         public MessageOut<Validator> createMessage()
         {
-            return new MessageOut<Validator>(StorageService.Verb.TREE_RESPONSE, this, Validator.serializer);
+            return new MessageOut<Validator>(MessagingService.Verb.TREE_RESPONSE, this, Validator.serializer);
         }
 
         public static class ValidatorSerializer implements IVersionedSerializer<Validator>
@@ -563,7 +563,7 @@ public class AntiEntropyService
 
         public MessageOut<TreeRequest> createMessage()
         {
-            return new MessageOut<TreeRequest>(StorageService.Verb.TREE_REQUEST, this, TreeRequest.serializer);
+            return new MessageOut<TreeRequest>(MessagingService.Verb.TREE_REQUEST, this, TreeRequest.serializer);
         }
 
         public static class TreeRequestSerializer implements IVersionedSerializer<TreeRequest>

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/service/MigrationManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/MigrationManager.java b/src/java/org/apache/cassandra/service/MigrationManager.java
index 4477f25..3e88984 100644
--- a/src/java/org/apache/cassandra/service/MigrationManager.java
+++ b/src/java/org/apache/cassandra/service/MigrationManager.java
@@ -183,7 +183,7 @@ public class MigrationManager implements IEndpointStateChangeSubscriber
 
     private static void pushSchemaMutation(InetAddress endpoint, Collection<RowMutation> schema)
     {
-        MessageOut<Collection<RowMutation>> msg = new MessageOut<Collection<RowMutation>>(StorageService.Verb.DEFINITIONS_UPDATE,
+        MessageOut<Collection<RowMutation>> msg = new MessageOut<Collection<RowMutation>>(MessagingService.Verb.DEFINITIONS_UPDATE,
                                                                                           schema,
                                                                                           MigrationsSerializer.instance);
         MessagingService.instance().sendOneWay(msg, endpoint);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/service/MigrationTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/MigrationTask.java b/src/java/org/apache/cassandra/service/MigrationTask.java
index 982bc8e..2b33a03 100644
--- a/src/java/org/apache/cassandra/service/MigrationTask.java
+++ b/src/java/org/apache/cassandra/service/MigrationTask.java
@@ -35,7 +35,7 @@ class MigrationTask extends WrappedRunnable
 
     public void runMayThrow() throws Exception
     {
-        MessageOut<MigrationTask> message = new MessageOut<MigrationTask>(StorageService.Verb.MIGRATION_REQUEST,
+        MessageOut<MigrationTask> message = new MessageOut<MigrationTask>(MessagingService.Verb.MIGRATION_REQUEST,
                                                                           this,
                                                                           serializer);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/service/RowRepairResolver.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/RowRepairResolver.java b/src/java/org/apache/cassandra/service/RowRepairResolver.java
index 547eb25..4796836 100644
--- a/src/java/org/apache/cassandra/service/RowRepairResolver.java
+++ b/src/java/org/apache/cassandra/service/RowRepairResolver.java
@@ -122,7 +122,7 @@ public class RowRepairResolver extends AbstractRowResolver
             MessageOut repairMessage;
             // use a separate verb here because we don't want these to be get the white glove hint-
             // on-timeout behavior that a "real" mutation gets
-            repairMessage = rowMutation.createMessage(StorageService.Verb.READ_REPAIR);
+            repairMessage = rowMutation.createMessage(MessagingService.Verb.READ_REPAIR);
             results.add(MessagingService.instance().sendRR(repairMessage, endpoints.get(i)));
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/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 c00cc53..ff678c2 100644
--- a/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java
+++ b/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java
@@ -41,7 +41,7 @@ public class SnapshotVerbHandler implements IVerbHandler
                 Table.open(command.keyspace).getColumnFamilyStore(command.column_family).snapshot(command.snapshot_name);
             if (logger.isDebugEnabled())
                 logger.debug("Sending response to snapshot request {} to {} ", command.snapshot_name, message.getFrom());
-            MessagingService.instance().sendReply(new MessageOut(StorageService.Verb.REQUEST_RESPONSE), id, message.getFrom());
+            MessagingService.instance().sendReply(new MessageOut(MessagingService.Verb.REQUEST_RESPONSE), id, message.getFrom());
         }
         catch (Exception ex)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java
index f4c1654..05772d3 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -444,7 +444,7 @@ public class StorageProxy implements StorageProxyMBean
     {
         if (logger.isDebugEnabled())
             logger.debug("insert writing local " + rm.toString(true));
-        Runnable runnable = new DroppableRunnable(StorageService.Verb.MUTATION)
+        Runnable runnable = new DroppableRunnable(MessagingService.Verb.MUTATION)
         {
             public void runMayThrow() throws IOException
             {
@@ -553,7 +553,7 @@ public class StorageProxy implements StorageProxyMBean
                                              final String localDataCenter,
                                              final ConsistencyLevel consistency_level)
     {
-        return new DroppableRunnable(StorageService.Verb.MUTATION)
+        return new DroppableRunnable(MessagingService.Verb.MUTATION)
         {
             public void runMayThrow() throws IOException
             {
@@ -570,7 +570,7 @@ public class StorageProxy implements StorageProxyMBean
                 {
                     // We do the replication on another stage because it involves a read (see CM.makeReplicationMutation)
                     // and we want to avoid blocking too much the MUTATION stage
-                    StageManager.getStage(Stage.REPLICATE_ON_WRITE).execute(new DroppableRunnable(StorageService.Verb.READ)
+                    StageManager.getStage(Stage.REPLICATE_ON_WRITE).execute(new DroppableRunnable(MessagingService.Verb.READ)
                     {
                         public void runMayThrow() throws IOException, TimeoutException
                         {
@@ -801,7 +801,7 @@ public class StorageProxy implements StorageProxyMBean
 
         LocalReadRunnable(ReadCommand command, ReadCallback<Row> handler)
         {
-            super(StorageService.Verb.READ);
+            super(MessagingService.Verb.READ);
             this.command = command;
             this.handler = handler;
         }
@@ -964,7 +964,7 @@ public class StorageProxy implements StorageProxyMBean
             }
         };
         // an empty message acts as a request to the SchemaCheckVerbHandler.
-        MessageOut message = new MessageOut(StorageService.Verb.SCHEMA_CHECK);
+        MessageOut message = new MessageOut(MessagingService.Verb.SCHEMA_CHECK);
         for (InetAddress endpoint : liveHosts)
             MessagingService.instance().sendRR(message, endpoint, cb);
 
@@ -1225,9 +1225,9 @@ public class StorageProxy implements StorageProxyMBean
     private static abstract class DroppableRunnable implements Runnable
     {
         private final long constructionTime = System.currentTimeMillis();
-        private final StorageService.Verb verb;
+        private final MessagingService.Verb verb;
 
-        public DroppableRunnable(StorageService.Verb verb)
+        public DroppableRunnable(MessagingService.Verb verb)
         {
             this.verb = verb;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/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 d8b4d34..eed37df 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -77,81 +77,6 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
     public static final int RING_DELAY = getRingDelay(); // delay after which we assume ring has stablized
 
-    /* All verb handler identifiers */
-    public enum Verb
-    {
-        MUTATION,
-        BINARY, // Deprecated
-        READ_REPAIR,
-        READ,
-        REQUEST_RESPONSE, // client-initiated reads and writes
-        STREAM_INITIATE, // Deprecated
-        STREAM_INITIATE_DONE, // Deprecated
-        STREAM_REPLY,
-        STREAM_REQUEST,
-        RANGE_SLICE,
-        BOOTSTRAP_TOKEN,
-        TREE_REQUEST,
-        TREE_RESPONSE,
-        JOIN, // Deprecated
-        GOSSIP_DIGEST_SYN,
-        GOSSIP_DIGEST_ACK,
-        GOSSIP_DIGEST_ACK2,
-        DEFINITIONS_ANNOUNCE, // Deprecated
-        DEFINITIONS_UPDATE,
-        TRUNCATE,
-        SCHEMA_CHECK,
-        INDEX_SCAN, // Deprecated
-        REPLICATION_FINISHED,
-        INTERNAL_RESPONSE, // responses to internal calls
-        COUNTER_MUTATION,
-        STREAMING_REPAIR_REQUEST,
-        STREAMING_REPAIR_RESPONSE,
-        SNAPSHOT, // Similar to nt snapshot
-        MIGRATION_REQUEST,
-        GOSSIP_SHUTDOWN,
-        // use as padding for backwards compatability where a previous version needs to validate a verb from the future.
-        UNUSED_1,
-        UNUSED_2,
-        UNUSED_3,
-        ;
-        // remember to add new verbs at the end, since we serialize by ordinal
-    }
-    public static final Verb[] VERBS = Verb.values();
-
-    public static final EnumMap<StorageService.Verb, Stage> verbStages = new EnumMap<StorageService.Verb, Stage>(StorageService.Verb.class)
-    {{
-        put(Verb.MUTATION, Stage.MUTATION);
-        put(Verb.BINARY, Stage.MUTATION);
-        put(Verb.READ_REPAIR, Stage.MUTATION);
-        put(Verb.TRUNCATE, Stage.MUTATION);
-        put(Verb.READ, Stage.READ);
-        put(Verb.REQUEST_RESPONSE, Stage.REQUEST_RESPONSE);
-        put(Verb.STREAM_REPLY, Stage.MISC); // TODO does this really belong on misc? I've just copied old behavior here
-        put(Verb.STREAM_REQUEST, Stage.STREAM);
-        put(Verb.RANGE_SLICE, Stage.READ);
-        put(Verb.BOOTSTRAP_TOKEN, Stage.MISC);
-        put(Verb.TREE_REQUEST, Stage.ANTI_ENTROPY);
-        put(Verb.TREE_RESPONSE, Stage.ANTI_ENTROPY);
-        put(Verb.STREAMING_REPAIR_REQUEST, Stage.ANTI_ENTROPY);
-        put(Verb.STREAMING_REPAIR_RESPONSE, Stage.ANTI_ENTROPY);
-        put(Verb.GOSSIP_DIGEST_ACK, Stage.GOSSIP);
-        put(Verb.GOSSIP_DIGEST_ACK2, Stage.GOSSIP);
-        put(Verb.GOSSIP_DIGEST_SYN, Stage.GOSSIP);
-        put(Verb.GOSSIP_SHUTDOWN, Stage.GOSSIP);
-        put(Verb.DEFINITIONS_UPDATE, Stage.MIGRATION);
-        put(Verb.SCHEMA_CHECK, Stage.MIGRATION);
-        put(Verb.MIGRATION_REQUEST, Stage.MIGRATION);
-        put(Verb.INDEX_SCAN, Stage.READ);
-        put(Verb.REPLICATION_FINISHED, Stage.MISC);
-        put(Verb.INTERNAL_RESPONSE, Stage.INTERNAL_RESPONSE);
-        put(Verb.COUNTER_MUTATION, Stage.MUTATION);
-        put(Verb.SNAPSHOT, Stage.MISC);
-        put(Verb.UNUSED_1, Stage.INTERNAL_RESPONSE);
-        put(Verb.UNUSED_2, Stage.INTERNAL_RESPONSE);
-        put(Verb.UNUSED_3, Stage.INTERNAL_RESPONSE);
-    }};
-
     private static int getRingDelay()
     {
         String newdelay = System.getProperty("cassandra.ring_delay_ms");
@@ -267,36 +192,36 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         }
 
         /* register the verb handlers */
-        MessagingService.instance().registerVerbHandlers(Verb.MUTATION, new RowMutationVerbHandler());
-        MessagingService.instance().registerVerbHandlers(Verb.READ_REPAIR, new ReadRepairVerbHandler());
-        MessagingService.instance().registerVerbHandlers(Verb.READ, new ReadVerbHandler());
-        MessagingService.instance().registerVerbHandlers(Verb.RANGE_SLICE, new RangeSliceVerbHandler());
-        MessagingService.instance().registerVerbHandlers(Verb.INDEX_SCAN, new IndexScanVerbHandler());
-        MessagingService.instance().registerVerbHandlers(Verb.COUNTER_MUTATION, new CounterMutationVerbHandler());
-        MessagingService.instance().registerVerbHandlers(Verb.TRUNCATE, new TruncateVerbHandler());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.MUTATION, new RowMutationVerbHandler());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.READ_REPAIR, new ReadRepairVerbHandler());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.READ, new ReadVerbHandler());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.RANGE_SLICE, new RangeSliceVerbHandler());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.INDEX_SCAN, new IndexScanVerbHandler());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.COUNTER_MUTATION, new CounterMutationVerbHandler());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.TRUNCATE, new TruncateVerbHandler());
 
         // see BootStrapper for a summary of how the bootstrap verbs interact
-        MessagingService.instance().registerVerbHandlers(Verb.BOOTSTRAP_TOKEN, new BootStrapper.BootstrapTokenVerbHandler());
-        MessagingService.instance().registerVerbHandlers(Verb.STREAM_REQUEST, new StreamRequestVerbHandler());
-        MessagingService.instance().registerVerbHandlers(Verb.STREAM_REPLY, new StreamReplyVerbHandler());
-        MessagingService.instance().registerVerbHandlers(Verb.REPLICATION_FINISHED, new ReplicationFinishedVerbHandler());
-        MessagingService.instance().registerVerbHandlers(Verb.REQUEST_RESPONSE, new ResponseVerbHandler());
-        MessagingService.instance().registerVerbHandlers(Verb.INTERNAL_RESPONSE, new ResponseVerbHandler());
-        MessagingService.instance().registerVerbHandlers(Verb.TREE_REQUEST, new TreeRequestVerbHandler());
-        MessagingService.instance().registerVerbHandlers(Verb.TREE_RESPONSE, new AntiEntropyService.TreeResponseVerbHandler());
-        MessagingService.instance().registerVerbHandlers(Verb.STREAMING_REPAIR_REQUEST, new StreamingRepairTask.StreamingRepairRequest());
-        MessagingService.instance().registerVerbHandlers(Verb.STREAMING_REPAIR_RESPONSE, new StreamingRepairTask.StreamingRepairResponse());
-
-        MessagingService.instance().registerVerbHandlers(Verb.GOSSIP_DIGEST_SYN, new GossipDigestSynVerbHandler());
-        MessagingService.instance().registerVerbHandlers(Verb.GOSSIP_DIGEST_ACK, new GossipDigestAckVerbHandler());
-        MessagingService.instance().registerVerbHandlers(Verb.GOSSIP_DIGEST_ACK2, new GossipDigestAck2VerbHandler());
-        MessagingService.instance().registerVerbHandlers(Verb.GOSSIP_SHUTDOWN, new GossipShutdownVerbHandler());
-
-        MessagingService.instance().registerVerbHandlers(Verb.DEFINITIONS_UPDATE, new DefinitionsUpdateVerbHandler());
-        MessagingService.instance().registerVerbHandlers(Verb.SCHEMA_CHECK, new SchemaCheckVerbHandler());
-        MessagingService.instance().registerVerbHandlers(Verb.MIGRATION_REQUEST, new MigrationRequestVerbHandler());
-
-        MessagingService.instance().registerVerbHandlers(Verb.SNAPSHOT, new SnapshotVerbHandler());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.BOOTSTRAP_TOKEN, new BootStrapper.BootstrapTokenVerbHandler());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.STREAM_REQUEST, new StreamRequestVerbHandler());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.STREAM_REPLY, new StreamReplyVerbHandler());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.REPLICATION_FINISHED, new ReplicationFinishedVerbHandler());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.REQUEST_RESPONSE, new ResponseVerbHandler());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.INTERNAL_RESPONSE, new ResponseVerbHandler());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.TREE_REQUEST, new TreeRequestVerbHandler());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.TREE_RESPONSE, new AntiEntropyService.TreeResponseVerbHandler());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.STREAMING_REPAIR_REQUEST, new StreamingRepairTask.StreamingRepairRequest());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.STREAMING_REPAIR_RESPONSE, new StreamingRepairTask.StreamingRepairResponse());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.GOSSIP_SHUTDOWN, new GossipShutdownVerbHandler());
+
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.GOSSIP_DIGEST_SYN, new GossipDigestSynVerbHandler());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.GOSSIP_DIGEST_ACK, new GossipDigestAckVerbHandler());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.GOSSIP_DIGEST_ACK2, new GossipDigestAck2VerbHandler());
+
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.DEFINITIONS_UPDATE, new DefinitionsUpdateVerbHandler());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.SCHEMA_CHECK, new SchemaCheckVerbHandler());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.MIGRATION_REQUEST, new MigrationRequestVerbHandler());
+
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.SNAPSHOT, new SnapshotVerbHandler());
 
         // spin up the streaming service so it is available for jmx tools.
         if (StreamingService.instance == null)
@@ -1507,7 +1432,7 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
     private void sendReplicationNotification(InetAddress remote)
     {
         // notify the remote token
-        MessageOut msg = new MessageOut(StorageService.Verb.REPLICATION_FINISHED);
+        MessageOut msg = new MessageOut(MessagingService.Verb.REPLICATION_FINISHED);
         IFailureDetector failureDetector = FailureDetector.instance;
         if (logger.isDebugEnabled())
             logger.debug("Notifying " + remote.toString() + " of replication completion\n");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/streaming/FileStreamTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/FileStreamTask.java b/src/java/org/apache/cassandra/streaming/FileStreamTask.java
index cc09a36..e4f3796 100644
--- a/src/java/org/apache/cassandra/streaming/FileStreamTask.java
+++ b/src/java/org/apache/cassandra/streaming/FileStreamTask.java
@@ -30,7 +30,6 @@ import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.net.Header;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.Throttle;
@@ -203,7 +202,7 @@ public class FileStreamTask extends WrappedRunnable
         byte[] body = new byte[bodySize];
         input.readFully(body);
         MessageIn message = new MessageIn(header, body, version);
-        assert message.getVerb() == StorageService.Verb.STREAM_REPLY : "Non-reply message received on stream socket";
+        assert message.getVerb() == MessagingService.Verb.STREAM_REPLY : "Non-reply message received on stream socket";
         handler.doVerb(message, id);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/streaming/ReplicationFinishedVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/ReplicationFinishedVerbHandler.java b/src/java/org/apache/cassandra/streaming/ReplicationFinishedVerbHandler.java
index f5e27b6..88d621b 100644
--- a/src/java/org/apache/cassandra/streaming/ReplicationFinishedVerbHandler.java
+++ b/src/java/org/apache/cassandra/streaming/ReplicationFinishedVerbHandler.java
@@ -33,7 +33,7 @@ public class ReplicationFinishedVerbHandler implements IVerbHandler
     public void doVerb(MessageIn msg, String id)
     {
         StorageService.instance.confirmReplication(msg.getFrom());
-        MessageOut response = new MessageOut(StorageService.Verb.INTERNAL_RESPONSE);
+        MessageOut response = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE);
         if (logger.isDebugEnabled())
             logger.debug("Replying to " + id + "@" + msg.getFrom());
         MessagingService.instance().sendReply(response, id, msg.getFrom());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/streaming/StreamReply.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamReply.java b/src/java/org/apache/cassandra/streaming/StreamReply.java
index 269d878..a44c215 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReply.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReply.java
@@ -23,7 +23,7 @@ import java.io.IOException;
 
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.net.MessagingService;
 
 class StreamReply
 {
@@ -50,7 +50,7 @@ class StreamReply
 
     public MessageOut<StreamReply> createMessage()
     {
-        return new MessageOut<StreamReply>(StorageService.Verb.STREAM_REPLY, this, serializer);
+        return new MessageOut<StreamReply>(MessagingService.Verb.STREAM_REPLY, this, serializer);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/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 6367cc5..561cd1f 100644
--- a/src/java/org/apache/cassandra/streaming/StreamRequestMessage.java
+++ b/src/java/org/apache/cassandra/streaming/StreamRequestMessage.java
@@ -36,7 +36,6 @@ import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.net.CompactEndpointSerializationHelper;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.service.StorageService;
 
 /**
 * This class encapsulates the message that needs to be sent to nodes
@@ -94,7 +93,7 @@ class StreamRequestMessage // TODO rename to StreamRequest
 
     public MessageOut<StreamRequestMessage> createMessage()
     {
-        return new MessageOut<StreamRequestMessage>(StorageService.Verb.STREAM_REQUEST, this, serializer);
+        return new MessageOut<StreamRequestMessage>(MessagingService.Verb.STREAM_REQUEST, this, serializer);
     }
 
     public String toString()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java b/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java
index ad2c77c..03d0b6e 100644
--- a/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java
@@ -35,7 +35,6 @@ import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.net.*;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.UUIDGen;
 
@@ -131,7 +130,7 @@ public class StreamingRepairTask implements Runnable
     private void forwardToSource()
     {
         logger.info(String.format("[streaming task #%s] Forwarding streaming repair of %d ranges to %s (to be streamed with %s)", id, ranges.size(), src, dst));
-        MessageOut<StreamingRepairTask> msg = new MessageOut<StreamingRepairTask>(StorageService.Verb.STREAMING_REPAIR_REQUEST,
+        MessageOut<StreamingRepairTask> msg = new MessageOut<StreamingRepairTask>(MessagingService.Verb.STREAMING_REPAIR_REQUEST,
                                                                                   this,
                                                                                   StreamingRepairTask.serializer);
         MessagingService.instance().sendOneWay(msg, src);
@@ -248,7 +247,7 @@ public class StreamingRepairTask implements Runnable
         private static void reply(InetAddress remote, UUID taskid) throws IOException
         {
             logger.info(String.format("[streaming task #%s] task suceed, forwarding response to %s", taskid, remote));
-            MessageOut<UUID> message = new MessageOut<UUID>(StorageService.Verb.STREAMING_REPAIR_RESPONSE, taskid, UUIDGen.serializer);
+            MessageOut<UUID> message = new MessageOut<UUID>(MessagingService.Verb.STREAMING_REPAIR_RESPONSE, taskid, UUIDGen.serializer);
             MessagingService.instance().sendOneWay(message, remote);
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/021ec71b/test/unit/org/apache/cassandra/service/RemoveTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/RemoveTest.java b/test/unit/org/apache/cassandra/service/RemoveTest.java
index 6dfb301..418db98 100644
--- a/test/unit/org/apache/cassandra/service/RemoveTest.java
+++ b/test/unit/org/apache/cassandra/service/RemoveTest.java
@@ -154,7 +154,7 @@ public class RemoveTest
         for (InetAddress host : hosts)
         {
             // TODO how to spoof host here?
-            MessageOut msg = new MessageOut(StorageService.Verb.REPLICATION_FINISHED);
+            MessageOut msg = new MessageOut(MessagingService.Verb.REPLICATION_FINISHED);
             MessagingService.instance().sendRR(msg, FBUtilities.getBroadcastAddress());
         }
 
@@ -171,7 +171,7 @@ public class RemoveTest
     {
         public MessageIn handleMessage(MessageIn msg, String id, InetAddress to)
         {
-            if (!msg.getVerb().equals(StorageService.Verb.STREAM_REQUEST))
+            if (!msg.getVerb().equals(MessagingService.Verb.STREAM_REQUEST))
                 return msg;
 
             StreamUtil.finishStreamRequest(msg, to);


Mime
View raw message