http://git-wip-us.apache.org/repos/asf/cassandra/blob/587cb582/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 947de9b..ac46c7b 100644
--- a/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java
@@ -242,9 +242,7 @@ public class StreamingRepairTask implements Runnable
             dos.writeUTF(task.cfName);
             dos.writeInt(task.ranges.size());
             for (Range<Token> range : task.ranges)
-            {
-                AbstractBounds.serializer().serialize(range, dos, version);
-            }
+                AbstractBounds.serializer.serialize(range, dos, version);
             // We don't serialize the callback on purpose
         }
 
@@ -259,9 +257,7 @@ public class StreamingRepairTask implements Runnable
             int rangesCount = dis.readInt();
             List<Range<Token>> ranges = new 
ArrayList<Range<Token>>(rangesCount);
             for (int i = 0; i < rangesCount; ++i)
-            {
-                ranges.add((Range<Token>) 
AbstractBounds.serializer().deserialize(dis, version).toTokenBounds());
-            }
+                ranges.add((Range<Token>) 
AbstractBounds.serializer.deserialize(dis, version).toTokenBounds());
             return new StreamingRepairTask(id, owner, src, dst, tableName, 
cfName, ranges, makeReplyingCallback(owner, id));
         }
 
@@ -273,7 +269,7 @@ public class StreamingRepairTask implements Runnable
             size += FBUtilities.serializedUTF8Size(task.cfName);
             size += DBTypeSizes.NATIVE.sizeof(task.ranges.size());
             for (Range<Token> range : task.ranges)
-                size += AbstractBounds.serializer().serializedSize(range, 
version);
+                size += AbstractBounds.serializer.serializedSize(range, 
version);
             return size;
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587cb582/src/java/org/apache/cassandra/utils/MerkleTree.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/MerkleTree.java 
b/src/java/org/apache/cassandra/utils/MerkleTree.java
index d012912..40d90a1 100644
--- a/src/java/org/apache/cassandra/utils/MerkleTree.java
+++ b/src/java/org/apache/cassandra/utils/MerkleTree.java
@@ -695,7 +695,7 @@ public class MerkleTree implements Serializable
                     dos.writeInt(inner.hash.length);
                     dos.write(inner.hash);
                 }
-                Token.serializer().serialize(inner.token, dos);
+                Token.serializer.serialize(inner.token, dos);
                 Hashable.serializer.serialize(inner.lchild, dos, version);
                 Hashable.serializer.serialize(inner.rchild, dos, version);
             }
@@ -706,7 +706,7 @@ public class MerkleTree implements Serializable
                 byte[] hash = hashLen >= 0 ? new byte[hashLen] : null;
                 if (hash != null)
                     dis.readFully(hash);
-                Token token = Token.serializer().deserialize(dis);
+                Token token = Token.serializer.deserialize(dis);
                 Hashable lchild = Hashable.serializer.deserialize(dis, 
version);
                 Hashable rchild = Hashable.serializer.deserialize(dis, 
version);
                 return new Inner(token, lchild, rchild);
@@ -718,7 +718,7 @@ public class MerkleTree implements Serializable
                          ? DBTypeSizes.NATIVE.sizeof(-1)
                          : DBTypeSizes.NATIVE.sizeof(inner.hash().length) + 
inner.hash().length;
 
-                size += Token.serializer().serializedSize(inner.token, 
DBTypeSizes.NATIVE)
+                size += Token.serializer.serializedSize(inner.token, 
DBTypeSizes.NATIVE)
                         + Hashable.serializer.serializedSize(inner.lchild, 
version)
                         + Hashable.serializer.serializedSize(inner.rchild, 
version);
                 return size;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587cb582/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/Util.java 
b/test/unit/org/apache/cassandra/Util.java
index a55787c..2dd0e36 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -278,7 +278,7 @@ public class Util
     {
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         DataOutputStream dos = new DataOutputStream(baos);
-        cf.serializer().serializeForSSTable(cf, dos);
+        cf.serializer.serializeForSSTable(cf, dos);
         return ByteBuffer.wrap(baos.toByteArray());
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587cb582/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cache/CacheProviderTest.java 
b/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
index bbecdcd..f6f3001 100644
--- a/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
+++ b/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
@@ -111,7 +111,7 @@ public class CacheProviderTest extends SchemaLoader
     @Test
     public void testSerializingCache() throws InterruptedException
     {
-        ICache<String, ColumnFamily> cache = new SerializingCache<String, 
ColumnFamily>(CAPACITY, false, ColumnFamily.serializer());
+        ICache<String, ColumnFamily> cache = new SerializingCache<String, 
ColumnFamily>(CAPACITY, false, ColumnFamily.serializer);
         ColumnFamily cf = createCF();
         simpleCase(cf, cache);
         concurrentCase(cf, cache);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587cb582/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java 
b/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
index b1c80bb..675b1cf 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
@@ -49,10 +49,10 @@ public class ColumnFamilyTest extends SchemaLoader
         cf = ColumnFamily.create("Keyspace1", "Standard1");
         cf.addColumn(column("C", "v", 1));
         DataOutputBuffer bufOut = new DataOutputBuffer();
-        ColumnFamily.serializer().serialize(cf, bufOut);
+        ColumnFamily.serializer.serialize(cf, bufOut);
 
         ByteArrayInputStream bufIn = new 
ByteArrayInputStream(bufOut.getData(), 0, bufOut.getLength());
-        cf = ColumnFamily.serializer().deserialize(new DataInputStream(bufIn));
+        cf = ColumnFamily.serializer.deserialize(new DataInputStream(bufIn));
         assert cf != null;
         assert cf.metadata().cfName.equals("Standard1");
         assert cf.getSortedColumns().size() == 1;
@@ -76,11 +76,11 @@ public class ColumnFamilyTest extends SchemaLoader
         {
             cf.addColumn(column(cName, map.get(cName), 314));
         }
-        ColumnFamily.serializer().serialize(cf, bufOut);
+        ColumnFamily.serializer.serialize(cf, bufOut);
 
         // verify
         ByteArrayInputStream bufIn = new 
ByteArrayInputStream(bufOut.getData(), 0, bufOut.getLength());
-        cf = ColumnFamily.serializer().deserialize(new DataInputStream(bufIn));
+        cf = ColumnFamily.serializer.deserialize(new DataInputStream(bufIn));
         for (String cName : map.navigableKeySet())
         {
             ByteBuffer val = cf.getColumn(ByteBufferUtil.bytes(cName)).value();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587cb582/test/unit/org/apache/cassandra/db/ReadMessageTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ReadMessageTest.java 
b/test/unit/org/apache/cassandra/db/ReadMessageTest.java
index 63094be..2c7f158 100644
--- a/test/unit/org/apache/cassandra/db/ReadMessageTest.java
+++ b/test/unit/org/apache/cassandra/db/ReadMessageTest.java
@@ -71,7 +71,7 @@ public class ReadMessageTest extends SchemaLoader
 
     private ReadCommand serializeAndDeserializeReadMessage(ReadCommand rm) 
throws IOException
     {
-        ReadCommandSerializer rms = ReadCommand.serializer();
+        ReadCommandSerializer rms = ReadCommand.serializer;
         DataOutputBuffer dos = new DataOutputBuffer();
         ByteArrayInputStream bis;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587cb582/test/unit/org/apache/cassandra/db/SerializationsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SerializationsTest.java 
b/test/unit/org/apache/cassandra/db/SerializationsTest.java
index 967f49e..264fd3a 100644
--- a/test/unit/org/apache/cassandra/db/SerializationsTest.java
+++ b/test/unit/org/apache/cassandra/db/SerializationsTest.java
@@ -114,17 +114,17 @@ public class SerializationsTest extends 
AbstractSerializationsTester
         SliceByNamesReadCommand superCmd = new 
SliceByNamesReadCommand(Statics.KS, Statics.Key, Statics.SuperPath, 
Statics.NamedCols);
 
         DataOutputStream out = getOutput("db.SliceByNamesReadCommand.bin");
-        SliceByNamesReadCommand.serializer().serialize(standardCmd, out, 
getVersion());
-        SliceByNamesReadCommand.serializer().serialize(superCmd, out, 
getVersion());
-        ReadCommand.serializer().serialize(standardCmd, out, getVersion());
-        ReadCommand.serializer().serialize(superCmd, out, getVersion());
+        SliceByNamesReadCommand.serializer.serialize(standardCmd, out, 
getVersion());
+        SliceByNamesReadCommand.serializer.serialize(superCmd, out, 
getVersion());
+        ReadCommand.serializer.serialize(standardCmd, out, getVersion());
+        ReadCommand.serializer.serialize(superCmd, out, getVersion());
         standardCmd.createMessage().serialize(out, getVersion());
         superCmd.createMessage().serialize(out, getVersion());
         out.close();
 
         // test serializedSize
-        testSerializedSize(standardCmd, SliceByNamesReadCommand.serializer());
-        testSerializedSize(superCmd, SliceByNamesReadCommand.serializer());
+        testSerializedSize(standardCmd, SliceByNamesReadCommand.serializer);
+        testSerializedSize(superCmd, SliceByNamesReadCommand.serializer);
     }
 
     @Test
@@ -134,10 +134,10 @@ public class SerializationsTest extends 
AbstractSerializationsTester
             testSliceByNamesReadCommandWrite();
 
         DataInputStream in = getInput("db.SliceByNamesReadCommand.bin");
-        assert SliceByNamesReadCommand.serializer().deserialize(in, 
getVersion()) != null;
-        assert SliceByNamesReadCommand.serializer().deserialize(in, 
getVersion()) != null;
-        assert ReadCommand.serializer().deserialize(in, getVersion()) != null;
-        assert ReadCommand.serializer().deserialize(in, getVersion()) != null;
+        assert SliceByNamesReadCommand.serializer.deserialize(in, 
getVersion()) != null;
+        assert SliceByNamesReadCommand.serializer.deserialize(in, 
getVersion()) != null;
+        assert ReadCommand.serializer.deserialize(in, getVersion()) != null;
+        assert ReadCommand.serializer.deserialize(in, getVersion()) != null;
         assert MessageIn.read(in, getVersion(), "id") != null;
         assert MessageIn.read(in, getVersion(), "id") != null;
         in.close();
@@ -148,17 +148,17 @@ public class SerializationsTest extends 
AbstractSerializationsTester
         SliceFromReadCommand standardCmd = new 
SliceFromReadCommand(Statics.KS, Statics.Key, Statics.StandardPath, 
Statics.Start, Statics.Stop, true, 100);
         SliceFromReadCommand superCmd = new SliceFromReadCommand(Statics.KS, 
Statics.Key, Statics.SuperPath, Statics.Start, Statics.Stop, true, 100);
         DataOutputStream out = getOutput("db.SliceFromReadCommand.bin");
-        SliceFromReadCommand.serializer().serialize(standardCmd, out, 
getVersion());
-        SliceFromReadCommand.serializer().serialize(superCmd, out, 
getVersion());
-        ReadCommand.serializer().serialize(standardCmd, out, getVersion());
-        ReadCommand.serializer().serialize(superCmd, out, getVersion());
+        SliceFromReadCommand.serializer.serialize(standardCmd, out, 
getVersion());
+        SliceFromReadCommand.serializer.serialize(superCmd, out, getVersion());
+        ReadCommand.serializer.serialize(standardCmd, out, getVersion());
+        ReadCommand.serializer.serialize(superCmd, out, getVersion());
         standardCmd.createMessage().serialize(out, getVersion());
         superCmd.createMessage().serialize(out, getVersion());
         out.close();
 
         // test serializedSize
-        testSerializedSize(standardCmd, SliceFromReadCommand.serializer());
-        testSerializedSize(superCmd, SliceFromReadCommand.serializer());
+        testSerializedSize(standardCmd, SliceFromReadCommand.serializer);
+        testSerializedSize(superCmd, SliceFromReadCommand.serializer);
     }
 
     @Test
@@ -168,10 +168,10 @@ public class SerializationsTest extends 
AbstractSerializationsTester
             testSliceFromReadCommandWrite();
 
         DataInputStream in = getInput("db.SliceFromReadCommand.bin");
-        assert SliceFromReadCommand.serializer().deserialize(in, getVersion()) 
!= null;
-        assert SliceFromReadCommand.serializer().deserialize(in, getVersion()) 
!= null;
-        assert ReadCommand.serializer().deserialize(in, getVersion()) != null;
-        assert ReadCommand.serializer().deserialize(in, getVersion()) != null;
+        assert SliceFromReadCommand.serializer.deserialize(in, getVersion()) 
!= null;
+        assert SliceFromReadCommand.serializer.deserialize(in, getVersion()) 
!= null;
+        assert ReadCommand.serializer.deserialize(in, getVersion()) != null;
+        assert ReadCommand.serializer.deserialize(in, getVersion()) != null;
         assert MessageIn.read(in, getVersion(), "id") != null;
         assert MessageIn.read(in, getVersion(), "id") != null;
         in.close();
@@ -180,15 +180,15 @@ public class SerializationsTest extends 
AbstractSerializationsTester
     private void testRowWrite() throws IOException
     {
         DataOutputStream out = getOutput("db.Row.bin");
-        Row.serializer().serialize(Statics.StandardRow, out, getVersion());
-        Row.serializer().serialize(Statics.SuperRow, out, getVersion());
-        Row.serializer().serialize(Statics.NullRow, out, getVersion());
+        Row.serializer.serialize(Statics.StandardRow, out, getVersion());
+        Row.serializer.serialize(Statics.SuperRow, out, getVersion());
+        Row.serializer.serialize(Statics.NullRow, out, getVersion());
         out.close();
 
         // test serializedSize
-        testSerializedSize(Statics.StandardRow, Row.serializer());
-        testSerializedSize(Statics.SuperRow, Row.serializer());
-        testSerializedSize(Statics.NullRow, Row.serializer());
+        testSerializedSize(Statics.StandardRow, Row.serializer);
+        testSerializedSize(Statics.SuperRow, Row.serializer);
+        testSerializedSize(Statics.NullRow, Row.serializer);
     }
 
     @Test
@@ -198,9 +198,9 @@ public class SerializationsTest extends 
AbstractSerializationsTester
             testRowWrite();
 
         DataInputStream in = getInput("db.Row.bin");
-        assert Row.serializer().deserialize(in, getVersion()) != null;
-        assert Row.serializer().deserialize(in, getVersion()) != null;
-        assert Row.serializer().deserialize(in, getVersion()) != null;
+        assert Row.serializer.deserialize(in, getVersion()) != null;
+        assert Row.serializer.deserialize(in, getVersion()) != null;
+        assert Row.serializer.deserialize(in, getVersion()) != null;
         in.close();
     }
 
@@ -219,12 +219,12 @@ public class SerializationsTest extends 
AbstractSerializationsTester
         RowMutation mixedRm = new RowMutation(Statics.KS, Statics.Key, mods);
 
         DataOutputStream out = getOutput("db.RowMutation.bin");
-        RowMutation.serializer().serialize(emptyRm, out, getVersion());
-        RowMutation.serializer().serialize(standardRowRm, out, getVersion());
-        RowMutation.serializer().serialize(superRowRm, out, getVersion());
-        RowMutation.serializer().serialize(standardRm, out, getVersion());
-        RowMutation.serializer().serialize(superRm, out, getVersion());
-        RowMutation.serializer().serialize(mixedRm, out, getVersion());
+        RowMutation.serializer.serialize(emptyRm, out, getVersion());
+        RowMutation.serializer.serialize(standardRowRm, out, getVersion());
+        RowMutation.serializer.serialize(superRowRm, out, getVersion());
+        RowMutation.serializer.serialize(standardRm, out, getVersion());
+        RowMutation.serializer.serialize(superRm, out, getVersion());
+        RowMutation.serializer.serialize(mixedRm, out, getVersion());
 
         emptyRm.createMessage().serialize(out, getVersion());
         standardRowRm.createMessage().serialize(out, getVersion());
@@ -236,12 +236,12 @@ public class SerializationsTest extends 
AbstractSerializationsTester
         out.close();
 
         // test serializedSize
-        testSerializedSize(emptyRm, RowMutation.serializer());
-        testSerializedSize(standardRowRm, RowMutation.serializer());
-        testSerializedSize(superRowRm, RowMutation.serializer());
-        testSerializedSize(standardRm, RowMutation.serializer());
-        testSerializedSize(superRm, RowMutation.serializer());
-        testSerializedSize(mixedRm, RowMutation.serializer());
+        testSerializedSize(emptyRm, RowMutation.serializer);
+        testSerializedSize(standardRowRm, RowMutation.serializer);
+        testSerializedSize(superRowRm, RowMutation.serializer);
+        testSerializedSize(standardRm, RowMutation.serializer);
+        testSerializedSize(superRm, RowMutation.serializer);
+        testSerializedSize(mixedRm, RowMutation.serializer);
     }
 
     @Test
@@ -251,12 +251,12 @@ public class SerializationsTest extends 
AbstractSerializationsTester
             testRowMutationWrite();
 
         DataInputStream in = getInput("db.RowMutation.bin");
-        assert RowMutation.serializer().deserialize(in, getVersion()) != null;
-        assert RowMutation.serializer().deserialize(in, getVersion()) != null;
-        assert RowMutation.serializer().deserialize(in, getVersion()) != null;
-        assert RowMutation.serializer().deserialize(in, getVersion()) != null;
-        assert RowMutation.serializer().deserialize(in, getVersion()) != null;
-        assert RowMutation.serializer().deserialize(in, getVersion()) != null;
+        assert RowMutation.serializer.deserialize(in, getVersion()) != null;
+        assert RowMutation.serializer.deserialize(in, getVersion()) != null;
+        assert RowMutation.serializer.deserialize(in, getVersion()) != null;
+        assert RowMutation.serializer.deserialize(in, getVersion()) != null;
+        assert RowMutation.serializer.deserialize(in, getVersion()) != null;
+        assert RowMutation.serializer.deserialize(in, getVersion()) != null;
         assert MessageIn.read(in, getVersion(), "id") != null;
         assert MessageIn.read(in, getVersion(), "id") != null;
         assert MessageIn.read(in, getVersion(), "id") != null;
@@ -272,9 +272,9 @@ public class SerializationsTest extends 
AbstractSerializationsTester
         TruncateResponse aff = new TruncateResponse(Statics.KS, "Doesn't 
Matter Either", true);
         TruncateResponse neg = new TruncateResponse(Statics.KS, "Still Doesn't 
Matter", false);
         DataOutputStream out = getOutput("db.Truncation.bin");
-        Truncation.serializer().serialize(tr, out, getVersion());
-        TruncateResponse.serializer().serialize(aff, out, getVersion());
-        TruncateResponse.serializer().serialize(neg, out, getVersion());
+        Truncation.serializer.serialize(tr, out, getVersion());
+        TruncateResponse.serializer.serialize(aff, out, getVersion());
+        TruncateResponse.serializer.serialize(neg, out, getVersion());
 
         tr.createMessage().serialize(out, getVersion());
         aff.createMessage().serialize(out, getVersion());
@@ -283,9 +283,9 @@ public class SerializationsTest extends 
AbstractSerializationsTester
         out.close();
 
         // test serializedSize
-        testSerializedSize(tr, Truncation.serializer());
-        testSerializedSize(aff, TruncateResponse.serializer());
-        testSerializedSize(neg, TruncateResponse.serializer());
+        testSerializedSize(tr, Truncation.serializer);
+        testSerializedSize(aff, TruncateResponse.serializer);
+        testSerializedSize(neg, TruncateResponse.serializer);
     }
 
     @Test
@@ -295,14 +295,14 @@ public class SerializationsTest extends 
AbstractSerializationsTester
             testTruncateWrite();
 
         DataInputStream in = getInput("db.Truncation.bin");
-        assert Truncation.serializer().deserialize(in, getVersion()) != null;
-        assert TruncateResponse.serializer().deserialize(in, getVersion()) != 
null;
-        assert TruncateResponse.serializer().deserialize(in, getVersion()) != 
null;
+        assert Truncation.serializer.deserialize(in, getVersion()) != null;
+        assert TruncateResponse.serializer.deserialize(in, getVersion()) != 
null;
+        assert TruncateResponse.serializer.deserialize(in, getVersion()) != 
null;
         assert MessageIn.read(in, getVersion(), "id") != null;
 
         // set up some fake callbacks so deserialization knows that what it's 
deserializing is a TruncateResponse
-        MessagingService.instance().setCallbackForTests("tr1", new 
CallbackInfo(null, null, TruncateResponse.serializer()));
-        MessagingService.instance().setCallbackForTests("tr2", new 
CallbackInfo(null, null, TruncateResponse.serializer()));
+        MessagingService.instance().setCallbackForTests("tr1", new 
CallbackInfo(null, null, TruncateResponse.serializer));
+        MessagingService.instance().setCallbackForTests("tr2", new 
CallbackInfo(null, null, TruncateResponse.serializer));
 
         assert MessageIn.read(in, getVersion(), "tr1") != null;
         assert MessageIn.read(in, getVersion(), "tr2") != null;
@@ -314,13 +314,13 @@ public class SerializationsTest extends 
AbstractSerializationsTester
         WriteResponse aff = new WriteResponse(Statics.KS, Statics.Key, true);
         WriteResponse neg = new WriteResponse(Statics.KS, Statics.Key, false);
         DataOutputStream out = getOutput("db.WriteResponse.bin");
-        WriteResponse.serializer().serialize(aff, out, getVersion());
-        WriteResponse.serializer().serialize(neg, out, getVersion());
+        WriteResponse.serializer.serialize(aff, out, getVersion());
+        WriteResponse.serializer.serialize(neg, out, getVersion());
         out.close();
 
         // test serializedSize
-        testSerializedSize(aff, WriteResponse.serializer());
-        testSerializedSize(neg, WriteResponse.serializer());
+        testSerializedSize(aff, WriteResponse.serializer);
+        testSerializedSize(neg, WriteResponse.serializer);
     }
 
     @Test
@@ -330,8 +330,8 @@ public class SerializationsTest extends 
AbstractSerializationsTester
             testWriteResponseWrite();
 
         DataInputStream in = getInput("db.WriteResponse.bin");
-        assert WriteResponse.serializer().deserialize(in, getVersion()) != 
null;
-        assert WriteResponse.serializer().deserialize(in, getVersion()) != 
null;
+        assert WriteResponse.serializer.deserialize(in, getVersion()) != null;
+        assert WriteResponse.serializer.deserialize(in, getVersion()) != null;
         in.close();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587cb582/test/unit/org/apache/cassandra/gms/GossipDigestTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/gms/GossipDigestTest.java 
b/test/unit/org/apache/cassandra/gms/GossipDigestTest.java
index 4f3e8e4..2928b12 100644
--- a/test/unit/org/apache/cassandra/gms/GossipDigestTest.java
+++ b/test/unit/org/apache/cassandra/gms/GossipDigestTest.java
@@ -1,6 +1,4 @@
-package org.apache.cassandra.gms;
 /*
- *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -17,9 +15,8 @@ package org.apache.cassandra.gms;
  * KIND, either express or implied.  See the License for the
  * specific language governing permissions and limitations
  * under the License.
- *
  */
-
+package org.apache.cassandra.gms;
 
 import static org.junit.Assert.*;
 
@@ -49,10 +46,10 @@ public class GossipDigestTest
 
         //test the serialization and equals
         DataOutputBuffer output = new DataOutputBuffer();
-        GossipDigest.serializer().serialize(expected, output, 
MessagingService.current_version);
+        GossipDigest.serializer.serialize(expected, output, 
MessagingService.current_version);
 
         ByteArrayInputStream input = new 
ByteArrayInputStream(output.getData(), 0, output.getLength());
-        GossipDigest actual = GossipDigest.serializer().deserialize(new 
DataInputStream(input), MessagingService.current_version);
+        GossipDigest actual = GossipDigest.serializer.deserialize(new 
DataInputStream(input), MessagingService.current_version);
         assertEquals(0, expected.compareTo(actual));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587cb582/test/unit/org/apache/cassandra/gms/SerializationsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/gms/SerializationsTest.java 
b/test/unit/org/apache/cassandra/gms/SerializationsTest.java
index a86e11c..afeaab3 100644
--- a/test/unit/org/apache/cassandra/gms/SerializationsTest.java
+++ b/test/unit/org/apache/cassandra/gms/SerializationsTest.java
@@ -38,15 +38,15 @@ public class SerializationsTest extends 
AbstractSerializationsTester
     private void testEndpointStateWrite() throws IOException
     {
         DataOutputStream out = getOutput("gms.EndpointState.bin");
-        HeartBeatState.serializer().serialize(Statics.HeartbeatSt, out, 
getVersion());
-        EndpointState.serializer().serialize(Statics.EndpointSt, out, 
getVersion());
+        HeartBeatState.serializer.serialize(Statics.HeartbeatSt, out, 
getVersion());
+        EndpointState.serializer.serialize(Statics.EndpointSt, out, 
getVersion());
         VersionedValue.serializer.serialize(Statics.vv0, out, getVersion());
         VersionedValue.serializer.serialize(Statics.vv1, out, getVersion());
         out.close();
 
         // test serializedSize
-        testSerializedSize(Statics.HeartbeatSt, HeartBeatState.serializer());
-        testSerializedSize(Statics.EndpointSt, EndpointState.serializer());
+        testSerializedSize(Statics.HeartbeatSt, HeartBeatState.serializer);
+        testSerializedSize(Statics.EndpointSt, EndpointState.serializer);
         testSerializedSize(Statics.vv0, VersionedValue.serializer);
         testSerializedSize(Statics.vv1, VersionedValue.serializer);
     }
@@ -58,8 +58,8 @@ public class SerializationsTest extends 
AbstractSerializationsTester
             testEndpointStateWrite();
 
         DataInputStream in = getInput("gms.EndpointState.bin");
-        assert HeartBeatState.serializer().deserialize(in, getVersion()) != 
null;
-        assert EndpointState.serializer().deserialize(in, getVersion()) != 
null;
+        assert HeartBeatState.serializer.deserialize(in, getVersion()) != null;
+        assert EndpointState.serializer.deserialize(in, getVersion()) != null;
         assert VersionedValue.serializer.deserialize(in, getVersion()) != null;
         assert VersionedValue.serializer.deserialize(in, getVersion()) != null;
         in.close();
@@ -76,18 +76,18 @@ public class SerializationsTest extends 
AbstractSerializationsTester
 
         DataOutputStream out = getOutput("gms.Gossip.bin");
         for (GossipDigest gd : Statics.Digests)
-            GossipDigest.serializer().serialize(gd, out, getVersion());
-        GossipDigestAck.serializer().serialize(ack, out, getVersion());
-        GossipDigestAck2.serializer().serialize(ack2, out, getVersion());
-        GossipDigestSyn.serializer().serialize(syn, out, getVersion());
+            GossipDigest.serializer.serialize(gd, out, getVersion());
+        GossipDigestAck.serializer.serialize(ack, out, getVersion());
+        GossipDigestAck2.serializer.serialize(ack2, out, getVersion());
+        GossipDigestSyn.serializer.serialize(syn, out, getVersion());
         out.close();
 
         // test serializedSize
         for (GossipDigest gd : Statics.Digests)
-            testSerializedSize(gd, GossipDigest.serializer());
-        testSerializedSize(ack, GossipDigestAck.serializer());
-        testSerializedSize(ack2, GossipDigestAck2.serializer());
-        testSerializedSize(syn, GossipDigestSyn.serializer());
+            testSerializedSize(gd, GossipDigest.serializer);
+        testSerializedSize(ack, GossipDigestAck.serializer);
+        testSerializedSize(ack2, GossipDigestAck2.serializer);
+        testSerializedSize(syn, GossipDigestSyn.serializer);
     }
 
     @Test
@@ -99,10 +99,10 @@ public class SerializationsTest extends 
AbstractSerializationsTester
         int count = 0;
         DataInputStream in = getInput("gms.Gossip.bin");
         while (count < Statics.Digests.size())
-            assert GossipDigestAck2.serializer().deserialize(in, getVersion()) 
!= null;
-        assert GossipDigestAck.serializer().deserialize(in, getVersion()) != 
null;
-        assert GossipDigestAck2.serializer().deserialize(in, getVersion()) != 
null;
-        assert GossipDigestSyn.serializer().deserialize(in, getVersion()) != 
null;
+            assert GossipDigestAck2.serializer.deserialize(in, getVersion()) 
!= null;
+        assert GossipDigestAck.serializer.deserialize(in, getVersion()) != 
null;
+        assert GossipDigestAck2.serializer.deserialize(in, getVersion()) != 
null;
+        assert GossipDigestSyn.serializer.deserialize(in, getVersion()) != 
null;
         in.close();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587cb582/test/unit/org/apache/cassandra/io/CompactSerializerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/CompactSerializerTest.java 
b/test/unit/org/apache/cassandra/io/CompactSerializerTest.java
index 9ba3948..c3aa220 100644
--- a/test/unit/org/apache/cassandra/io/CompactSerializerTest.java
+++ b/test/unit/org/apache/cassandra/io/CompactSerializerTest.java
@@ -71,7 +71,6 @@ public class CompactSerializerTest extends SchemaLoader
         expectedClassNames.add("SnapshotCommandSerializer");
         expectedClassNames.add("RangeSliceReplySerializer");
         expectedClassNames.add("StringSerializer");
-        expectedClassNames.add("GossipShutdownMessageSerializer");
         expectedClassNames.add("CallbackDeterminedSerializer");
         expectedClassNames.add("TreeRequestSerializer");
         expectedClassNames.add("ValidatorSerializer");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587cb582/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java 
b/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java
index caf7147..ab1b9e3 100644
--- a/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java
+++ b/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java
@@ -1,6 +1,4 @@
-package org.apache.cassandra.io;
 /*
- *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -17,9 +15,8 @@ package org.apache.cassandra.io;
  * KIND, either express or implied.  See the License for the
  * specific language governing permissions and limitations
  * under the License.
- *
  */
-
+package org.apache.cassandra.io;
 
 import java.io.File;
 import java.io.FileInputStream;
@@ -123,8 +120,8 @@ public class LazilyCompactedRowTest extends SchemaLoader
             // cf metadata
             ColumnFamily cf1 = ColumnFamily.create(cfs.metadata);
             ColumnFamily cf2 = ColumnFamily.create(cfs.metadata);
-            ColumnFamily.serializer().deserializeFromSSTableNoColumns(cf1, 
in1);
-            ColumnFamily.serializer().deserializeFromSSTableNoColumns(cf2, 
in2);
+            ColumnFamily.serializer.deserializeFromSSTableNoColumns(cf1, in1);
+            ColumnFamily.serializer.deserializeFromSSTableNoColumns(cf2, in2);
             assert cf1.getLocalDeletionTime() == cf2.getLocalDeletionTime();
             assert cf1.getMarkedForDeleteAt() == cf2.getMarkedForDeleteAt();
             // columns
@@ -222,7 +219,7 @@ public class LazilyCompactedRowTest extends SchemaLoader
             rm.add(new QueryPath("Standard1", null, ByteBufferUtil.bytes(i)), 
ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
         rm.apply();
         DataOutputBuffer out = new DataOutputBuffer();
-        RowMutation.serializer().serialize(rm, out, 
MessagingService.current_version);
+        RowMutation.serializer.serialize(rm, out, 
MessagingService.current_version);
         assert out.getLength() > DatabaseDescriptor.getColumnIndexSize();
         cfs.forceBlockingFlush();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587cb582/test/unit/org/apache/cassandra/streaming/SerializationsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/SerializationsTest.java 
b/test/unit/org/apache/cassandra/streaming/SerializationsTest.java
index 94fb3bd..c904881 100644
--- a/test/unit/org/apache/cassandra/streaming/SerializationsTest.java
+++ b/test/unit/org/apache/cassandra/streaming/SerializationsTest.java
@@ -55,17 +55,17 @@ public class SerializationsTest extends 
AbstractSerializationsTester
         PendingFile noSST = makePendingFile(false, 100, 
OperationType.RESTORE_REPLICA_COUNT);
 
         DataOutputStream out = getOutput("streaming.PendingFile.bin");
-        PendingFile.serializer().serialize(normal, out, getVersion());
-        PendingFile.serializer().serialize(noSections, out, getVersion());
-        PendingFile.serializer().serialize(noSST, out, getVersion());
-        PendingFile.serializer().serialize(null, out, getVersion());
+        PendingFile.serializer.serialize(normal, out, getVersion());
+        PendingFile.serializer.serialize(noSections, out, getVersion());
+        PendingFile.serializer.serialize(noSST, out, getVersion());
+        PendingFile.serializer.serialize(null, out, getVersion());
         out.close();
 
         // test serializedSize
-        testSerializedSize(normal, PendingFile.serializer());
-        testSerializedSize(noSections, PendingFile.serializer());
-        testSerializedSize(noSST, PendingFile.serializer());
-        testSerializedSize(null, PendingFile.serializer());
+        testSerializedSize(normal, PendingFile.serializer);
+        testSerializedSize(noSections, PendingFile.serializer);
+        testSerializedSize(noSST, PendingFile.serializer);
+        testSerializedSize(null, PendingFile.serializer);
     }
 
     @Test
@@ -75,10 +75,10 @@ public class SerializationsTest extends 
AbstractSerializationsTester
             testPendingFileWrite();
 
         DataInputStream in = getInput("streaming.PendingFile.bin");
-        assert PendingFile.serializer().deserialize(in, getVersion()) != null;
-        assert PendingFile.serializer().deserialize(in, getVersion()) != null;
-        assert PendingFile.serializer().deserialize(in, getVersion()) != null;
-        assert PendingFile.serializer().deserialize(in, getVersion()) == null;
+        assert PendingFile.serializer.deserialize(in, getVersion()) != null;
+        assert PendingFile.serializer.deserialize(in, getVersion()) != null;
+        assert PendingFile.serializer.deserialize(in, getVersion()) != null;
+        assert PendingFile.serializer.deserialize(in, getVersion()) == null;
         in.close();
     }
 
@@ -94,19 +94,19 @@ public class SerializationsTest extends 
AbstractSerializationsTester
         StreamHeader sh4 = new StreamHeader("Keyspace1", 125L, 
makePendingFile(true, 100, OperationType.BOOTSTRAP), new 
ArrayList<PendingFile>());
 
         DataOutputStream out = getOutput("streaming.StreamHeader.bin");
-        StreamHeader.serializer().serialize(sh0, out, getVersion());
-        StreamHeader.serializer().serialize(sh1, out, getVersion());
-        StreamHeader.serializer().serialize(sh2, out, getVersion());
-        StreamHeader.serializer().serialize(sh3, out, getVersion());
-        StreamHeader.serializer().serialize(sh4, out, getVersion());
+        StreamHeader.serializer.serialize(sh0, out, getVersion());
+        StreamHeader.serializer.serialize(sh1, out, getVersion());
+        StreamHeader.serializer.serialize(sh2, out, getVersion());
+        StreamHeader.serializer.serialize(sh3, out, getVersion());
+        StreamHeader.serializer.serialize(sh4, out, getVersion());
         out.close();
 
         // test serializedSize
-        testSerializedSize(sh0, StreamHeader.serializer());
-        testSerializedSize(sh1, StreamHeader.serializer());
-        testSerializedSize(sh2, StreamHeader.serializer());
-        testSerializedSize(sh3, StreamHeader.serializer());
-        testSerializedSize(sh4, StreamHeader.serializer());
+        testSerializedSize(sh0, StreamHeader.serializer);
+        testSerializedSize(sh1, StreamHeader.serializer);
+        testSerializedSize(sh2, StreamHeader.serializer);
+        testSerializedSize(sh3, StreamHeader.serializer);
+        testSerializedSize(sh4, StreamHeader.serializer);
     }
 
     @Test
@@ -116,11 +116,11 @@ public class SerializationsTest extends 
AbstractSerializationsTester
             testStreamHeaderWrite();
 
         DataInputStream in = getInput("streaming.StreamHeader.bin");
-        assert StreamHeader.serializer().deserialize(in, getVersion()) != null;
-        assert StreamHeader.serializer().deserialize(in, getVersion()) != null;
-        assert StreamHeader.serializer().deserialize(in, getVersion()) != null;
-        assert StreamHeader.serializer().deserialize(in, getVersion()) != null;
-        assert StreamHeader.serializer().deserialize(in, getVersion()) != null;
+        assert StreamHeader.serializer.deserialize(in, getVersion()) != null;
+        assert StreamHeader.serializer.deserialize(in, getVersion()) != null;
+        assert StreamHeader.serializer.deserialize(in, getVersion()) != null;
+        assert StreamHeader.serializer.deserialize(in, getVersion()) != null;
+        assert StreamHeader.serializer.deserialize(in, getVersion()) != null;
         in.close();
     }
 
@@ -168,18 +168,18 @@ public class SerializationsTest extends 
AbstractSerializationsTester
         StreamRequest msg2 = new 
StreamRequest(FBUtilities.getBroadcastAddress(), makePendingFile(false, 100, 
OperationType.BOOTSTRAP), 124L);
 
         DataOutputStream out = getOutput("streaming.StreamRequestMessage.bin");
-        StreamRequest.serializer().serialize(msg0, out, getVersion());
-        StreamRequest.serializer().serialize(msg1, out, getVersion());
-        StreamRequest.serializer().serialize(msg2, out, getVersion());
+        StreamRequest.serializer.serialize(msg0, out, getVersion());
+        StreamRequest.serializer.serialize(msg1, out, getVersion());
+        StreamRequest.serializer.serialize(msg2, out, getVersion());
         msg0.createMessage().serialize(out, getVersion());
         msg1.createMessage().serialize(out, getVersion());
         msg2.createMessage().serialize(out, getVersion());
         out.close();
 
         // test serializedSize
-        testSerializedSize(msg0, StreamRequest.serializer());
-        testSerializedSize(msg1, StreamRequest.serializer());
-        testSerializedSize(msg2, StreamRequest.serializer());
+        testSerializedSize(msg0, StreamRequest.serializer);
+        testSerializedSize(msg1, StreamRequest.serializer);
+        testSerializedSize(msg2, StreamRequest.serializer);
     }
 
     @Test
@@ -189,9 +189,9 @@ public class SerializationsTest extends 
AbstractSerializationsTester
             testStreamRequestMessageWrite();
 
         DataInputStream in = getInput("streaming.StreamRequestMessage.bin");
-        assert StreamRequest.serializer().deserialize(in, getVersion()) != 
null;
-        assert StreamRequest.serializer().deserialize(in, getVersion()) != 
null;
-        assert StreamRequest.serializer().deserialize(in, getVersion()) != 
null;
+        assert StreamRequest.serializer.deserialize(in, getVersion()) != null;
+        assert StreamRequest.serializer.deserialize(in, getVersion()) != null;
+        assert StreamRequest.serializer.deserialize(in, getVersion()) != null;
         assert MessageIn.read(in, getVersion(), "id") != null;
         assert MessageIn.read(in, getVersion(), "id") != null;
         assert MessageIn.read(in, getVersion(), "id") != null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587cb582/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java 
b/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java
index 1907c83..9c08e49 100644
--- a/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java
+++ b/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java
@@ -126,7 +126,7 @@ public class EncodedStreamsTest extends SchemaLoader
         EncodedDataInputStream odis = new EncodedDataInputStream(new 
DataInputStream(byteArrayIStream1));
         ColumnFamily cf = ColumnFamily.serializer.deserialize(odis);
         Assert.assertEquals(cf, createCF());
-        Assert.assertEquals(byteArrayOStream1.size(), 
(int)ColumnFamily.serializer().serializedSize(cf, DBTypeSizes.VINT));
+        Assert.assertEquals(byteArrayOStream1.size(), (int) 
ColumnFamily.serializer.serializedSize(cf, DBTypeSizes.VINT));
     }
 
     @Test
@@ -140,7 +140,7 @@ public class EncodedStreamsTest extends SchemaLoader
         EncodedDataInputStream odis = new EncodedDataInputStream(new 
DataInputStream(byteArrayIStream1));
         ColumnFamily cf = ColumnFamily.serializer.deserialize(odis);
         Assert.assertEquals(cf, createCounterCF());
-        Assert.assertEquals(byteArrayOStream1.size(), 
(int)ColumnFamily.serializer().serializedSize(cf, DBTypeSizes.VINT));
+        Assert.assertEquals(byteArrayOStream1.size(), (int) 
ColumnFamily.serializer.serializedSize(cf, DBTypeSizes.VINT));
     }
 
     @Test
@@ -154,7 +154,7 @@ public class EncodedStreamsTest extends SchemaLoader
         EncodedDataInputStream odis = new EncodedDataInputStream(new 
DataInputStream(byteArrayIStream1));
         ColumnFamily cf = ColumnFamily.serializer.deserialize(odis);
         Assert.assertEquals(cf, createSuperCF());
-        Assert.assertEquals(byteArrayOStream1.size(), 
(int)ColumnFamily.serializer().serializedSize(cf, DBTypeSizes.VINT));
+        Assert.assertEquals(byteArrayOStream1.size(), (int) 
ColumnFamily.serializer.serializedSize(cf, DBTypeSizes.VINT));
     }
 }
 

Reply via email to