You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by gd...@apache.org on 2011/02/07 16:38:09 UTC

svn commit: r1067975 [2/2] - in /cassandra/trunk: src/java/org/apache/cassandra/db/ src/java/org/apache/cassandra/db/commitlog/ src/java/org/apache/cassandra/db/migration/ src/java/org/apache/cassandra/gms/ src/java/org/apache/cassandra/io/ src/java/or...

Modified: cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamRequestVerbHandler.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamRequestVerbHandler.java?rev=1067975&r1=1067974&r2=1067975&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamRequestVerbHandler.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamRequestVerbHandler.java Mon Feb  7 15:38:07 2011
@@ -46,7 +46,7 @@ public class StreamRequestVerbHandler im
         ByteArrayInputStream bufIn = new ByteArrayInputStream(body);
         try
         {
-            StreamRequestMessage srm = StreamRequestMessage.serializer().deserialize(new DataInputStream(bufIn));
+            StreamRequestMessage srm = StreamRequestMessage.serializer().deserialize(new DataInputStream(bufIn), message.getVersion());
             if (logger.isDebugEnabled())
                 logger.debug(srm.toString());
 

Modified: cassandra/trunk/src/java/org/apache/cassandra/utils/BloomFilter.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/utils/BloomFilter.java?rev=1067975&r1=1067974&r2=1067975&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/utils/BloomFilter.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/utils/BloomFilter.java Mon Feb  7 15:38:07 2011
@@ -20,6 +20,7 @@ package org.apache.cassandra.utils;
 
 import java.nio.ByteBuffer;
 
+import org.apache.cassandra.io.ICompactSerializer2;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -31,7 +32,7 @@ public class BloomFilter extends Filter
 
     private static final Logger logger = LoggerFactory.getLogger(BloomFilter.class);
     private static final int EXCESS = 20;
-    static ICompactSerializer<BloomFilter> serializer_ = new BloomFilterSerializer();
+    static ICompactSerializer2<BloomFilter> serializer_ = new BloomFilterSerializer();
 
     public OpenBitSet bitset;
 
@@ -41,7 +42,7 @@ public class BloomFilter extends Filter
         bitset = bs;
     }
 
-    public static ICompactSerializer<BloomFilter> serializer()
+    public static ICompactSerializer2<BloomFilter> serializer()
     {
         return serializer_;
     }

Modified: cassandra/trunk/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java?rev=1067975&r1=1067974&r2=1067975&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java Mon Feb  7 15:38:07 2011
@@ -21,15 +21,16 @@ package org.apache.cassandra.utils;
  */
 
 
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
+import java.io.DataInput;
+import java.io.DataOutput;
 import java.io.IOException;
+
+import org.apache.cassandra.io.ICompactSerializer2;
 import org.apache.cassandra.utils.obs.OpenBitSet;
-import org.apache.cassandra.io.ICompactSerializer;
 
-class BloomFilterSerializer implements ICompactSerializer<BloomFilter>
+class BloomFilterSerializer implements ICompactSerializer2<BloomFilter>
 {
-    public void serialize(BloomFilter bf, DataOutputStream dos) throws IOException
+    public void serialize(BloomFilter bf, DataOutput dos) throws IOException
     {
         long[] bits = bf.bitset.getBits();
         int bitLength = bits.length;
@@ -39,10 +40,9 @@ class BloomFilterSerializer implements I
 
         for (int i = 0; i < bitLength; i++)
             dos.writeLong(bits[i]);
-        dos.flush();
     }
 
-    public BloomFilter deserialize(DataInputStream dis) throws IOException
+    public BloomFilter deserialize(DataInput dis) throws IOException
     {
         int hashes = dis.readInt();
         int bitLength = dis.readInt();

Modified: cassandra/trunk/src/java/org/apache/cassandra/utils/EstimatedHistogram.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/utils/EstimatedHistogram.java?rev=1067975&r1=1067974&r2=1067975&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/utils/EstimatedHistogram.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/utils/EstimatedHistogram.java Mon Feb  7 15:38:07 2011
@@ -18,13 +18,16 @@
 */
 package org.apache.cassandra.utils;
 
+import java.io.DataInput;
 import java.io.DataInputStream;
+import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.concurrent.atomic.AtomicLongArray;
 
 import org.apache.cassandra.io.ICompactSerializer;
+import org.apache.cassandra.io.ICompactSerializer2;
 
 public class EstimatedHistogram
 {
@@ -154,9 +157,9 @@ public class EstimatedHistogram
         return median;
     }
 
-    public static class EstimatedHistogramSerializer implements ICompactSerializer<EstimatedHistogram>
+    public static class EstimatedHistogramSerializer implements ICompactSerializer2<EstimatedHistogram>
     {
-        public void serialize(EstimatedHistogram eh, DataOutputStream dos) throws IOException
+        public void serialize(EstimatedHistogram eh, DataOutput dos) throws IOException
         {
             long[] offsets = eh.getBucketOffsets();
             long[] buckets = eh.get(false);
@@ -168,7 +171,7 @@ public class EstimatedHistogram
             }
         }
 
-        public EstimatedHistogram deserialize(DataInputStream dis) throws IOException
+        public EstimatedHistogram deserialize(DataInput dis) throws IOException
         {
             int size = dis.readInt();
             long[] offsets = new long[size - 1];

Modified: cassandra/trunk/src/java/org/apache/cassandra/utils/LegacyBloomFilterSerializer.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/utils/LegacyBloomFilterSerializer.java?rev=1067975&r1=1067974&r2=1067975&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/utils/LegacyBloomFilterSerializer.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/utils/LegacyBloomFilterSerializer.java Mon Feb  7 15:38:07 2011
@@ -25,23 +25,22 @@ import java.util.BitSet;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
-import java.io.ObjectOutputStream;
 import java.io.ObjectInputStream;
-import org.apache.cassandra.utils.obs.OpenBitSet;
 import org.apache.cassandra.io.ICompactSerializer;
 
 class LegacyBloomFilterSerializer implements ICompactSerializer<LegacyBloomFilter>
 {
-    public void serialize(LegacyBloomFilter bf, DataOutputStream dos)
+    public void serialize(LegacyBloomFilter bf, DataOutputStream dos, int version)
             throws IOException
     {
-        dos.writeInt(bf.getHashCount());
-        ObjectOutputStream oos = new ObjectOutputStream(dos);
-        oos.writeObject(bf.getBitSet());
-        oos.flush();
+        throw new UnsupportedOperationException("Shouldn't be serializing legacy bloom filters");
+//        dos.writeInt(bf.getHashCount());
+//        ObjectOutputStream oos = new ObjectOutputStream(dos);
+//        oos.writeObject(bf.getBitSet());
+//        oos.flush();
     }
 
-    public LegacyBloomFilter deserialize(DataInputStream dis) throws IOException
+    public LegacyBloomFilter deserialize(DataInputStream dis, int version) throws IOException
     {
         int hashes = dis.readInt();
         ObjectInputStream ois = new ObjectInputStream(dis);

Modified: cassandra/trunk/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/AbstractSerializationsTester.java?rev=1067975&r1=1067974&r2=1067975&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/AbstractSerializationsTester.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/AbstractSerializationsTester.java Mon Feb  7 15:38:07 2011
@@ -27,13 +27,24 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
 
 public class AbstractSerializationsTester extends SchemaLoader
 {
     protected static final String CUR_VER = System.getProperty("cassandra.version", "0.7");
+    protected static final Map<String, Integer> VERSION_MAP = new HashMap<String, Integer> () 
+    {{
+        put("0.7", 1);
+    }};
     
     protected static final boolean EXECUTE_WRITES = new Boolean(System.getProperty("cassandra.test-serialization-writes", "False")).booleanValue();
     
+    protected final int getVersion()
+    {
+        return VERSION_MAP.get(CUR_VER);
+    }
+    
     protected static DataInputStream getInput(String name) throws IOException
     {
         File f = new File("test/data/serialization/" + CUR_VER + "/" + name);

Modified: cassandra/trunk/test/unit/org/apache/cassandra/db/DefsTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/db/DefsTest.java?rev=1067975&r1=1067974&r2=1067975&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/db/DefsTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/db/DefsTest.java Mon Feb  7 15:38:07 2011
@@ -58,6 +58,7 @@ import org.apache.cassandra.db.migration
 import org.apache.cassandra.io.SerDeUtils;
 import org.apache.cassandra.locator.OldNetworkTopologyStrategy;
 import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.thrift.IndexType;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.UUIDGen;
@@ -238,7 +239,7 @@ public class DefsTest extends CleanupHel
         for (IColumn col : serializedMigrations)
         {
             UUID version = UUIDGen.getUUID(col.name());
-            reconstituded[i] = Migration.deserialize(col.value());
+            reconstituded[i] = Migration.deserialize(col.value(), MessagingService.version_);
             assert version.equals(reconstituded[i].getVersion());
             i++;
         }

Modified: cassandra/trunk/test/unit/org/apache/cassandra/db/ReadMessageTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/db/ReadMessageTest.java?rev=1067975&r1=1067974&r2=1067975&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/db/ReadMessageTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/db/ReadMessageTest.java Mon Feb  7 15:38:07 2011
@@ -32,6 +32,7 @@ import org.apache.cassandra.Util;
 import org.apache.cassandra.db.filter.QueryPath;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 
+import org.apache.cassandra.net.MessagingService;
 import org.junit.Test;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -75,9 +76,9 @@ public class ReadMessageTest extends Sch
         DataOutputBuffer dos = new DataOutputBuffer();
         ByteArrayInputStream bis;
 
-        rms.serialize(rm, dos);
+        rms.serialize(rm, dos, MessagingService.version_);
         bis = new ByteArrayInputStream(dos.getData(), 0, dos.getLength());
-        return rms.deserialize(new DataInputStream(bis));
+        return rms.deserialize(new DataInputStream(bis), MessagingService.version_);
     }
     
     @Test

Modified: cassandra/trunk/test/unit/org/apache/cassandra/db/SerializationsTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/db/SerializationsTest.java?rev=1067975&r1=1067974&r2=1067975&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/db/SerializationsTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/db/SerializationsTest.java Mon Feb  7 15:38:07 2011
@@ -71,12 +71,12 @@ public class SerializationsTest extends 
         
         DataOutputStream dout = getOutput("db.RangeSliceCommand.bin");
         
-        Message.serializer().serialize(namesCmd, dout);
-        Message.serializer().serialize(emptyRangeCmd, dout);
-        Message.serializer().serialize(regRangeCmd, dout);
-        Message.serializer().serialize(namesCmdSup, dout);
-        Message.serializer().serialize(emptyRangeCmdSup, dout);
-        Message.serializer().serialize(regRangeCmdSup, dout);
+        Message.serializer().serialize(namesCmd, dout, getVersion());
+        Message.serializer().serialize(emptyRangeCmd, dout, getVersion());
+        Message.serializer().serialize(regRangeCmd, dout, getVersion());
+        Message.serializer().serialize(namesCmdSup, dout, getVersion());
+        Message.serializer().serialize(emptyRangeCmdSup, dout, getVersion());
+        Message.serializer().serialize(regRangeCmdSup, dout, getVersion());
         dout.close();
     }
     
@@ -89,7 +89,7 @@ public class SerializationsTest extends 
         DataInputStream in = getInput("db.RangeSliceCommand.bin");
         for (int i = 0; i < 6; i++)
         {
-            Message msg = Message.serializer().deserialize(in);
+            Message msg = Message.serializer().deserialize(in, getVersion());
             RangeSliceCommand cmd = RangeSliceCommand.read(msg);
         }
         in.close();
@@ -101,12 +101,12 @@ public class SerializationsTest extends 
         SliceByNamesReadCommand superCmd = new SliceByNamesReadCommand(Statics.KS, Statics.Key, Statics.SuperPath, Statics.NamedCols);
         
         DataOutputStream out = getOutput("db.SliceByNamesReadCommand.bin");
-        SliceByNamesReadCommand.serializer().serialize(standardCmd, out);
-        SliceByNamesReadCommand.serializer().serialize(superCmd, out);
-        ReadCommand.serializer().serialize(standardCmd, out);
-        ReadCommand.serializer().serialize(superCmd, out);
-        Message.serializer().serialize(standardCmd.getMessage(MessagingService.version_), out);
-        Message.serializer().serialize(superCmd.getMessage(MessagingService.version_), out);
+        SliceByNamesReadCommand.serializer().serialize(standardCmd, out, getVersion());
+        SliceByNamesReadCommand.serializer().serialize(superCmd, out, getVersion());
+        ReadCommand.serializer().serialize(standardCmd, out, getVersion());
+        ReadCommand.serializer().serialize(superCmd, out, getVersion());
+        Message.serializer().serialize(standardCmd.getMessage(getVersion()), out, getVersion());
+        Message.serializer().serialize(superCmd.getMessage(getVersion()), out, getVersion());
         out.close();
     }
     
@@ -117,12 +117,12 @@ public class SerializationsTest extends 
             testSliceByNamesReadCommandWrite();
         
         DataInputStream in = getInput("db.SliceByNamesReadCommand.bin");
-        assert SliceByNamesReadCommand.serializer().deserialize(in) != null;
-        assert SliceByNamesReadCommand.serializer().deserialize(in) != null;
-        assert ReadCommand.serializer().deserialize(in) != null;
-        assert ReadCommand.serializer().deserialize(in) != null;
-        assert Message.serializer().deserialize(in) != null;
-        assert Message.serializer().deserialize(in) != 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 Message.serializer().deserialize(in, getVersion()) != null;
+        assert Message.serializer().deserialize(in, getVersion()) != null;
         in.close();
     }
     
@@ -131,12 +131,12 @@ public class SerializationsTest extends 
         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);
-        SliceFromReadCommand.serializer().serialize(superCmd, out);
-        ReadCommand.serializer().serialize(standardCmd, out);
-        ReadCommand.serializer().serialize(superCmd, out);
-        Message.serializer().serialize(standardCmd.getMessage(MessagingService.version_), out);
-        Message.serializer().serialize(superCmd.getMessage(MessagingService.version_), out);
+        SliceFromReadCommand.serializer().serialize(standardCmd, out, getVersion());
+        SliceFromReadCommand.serializer().serialize(superCmd, out, getVersion());
+        ReadCommand.serializer().serialize(standardCmd, out, getVersion());
+        ReadCommand.serializer().serialize(superCmd, out, getVersion());
+        Message.serializer().serialize(standardCmd.getMessage(getVersion()), out, getVersion());
+        Message.serializer().serialize(superCmd.getMessage(getVersion()), out, getVersion());
         out.close();
     }
     
@@ -147,21 +147,21 @@ public class SerializationsTest extends 
             testSliceFromReadCommandWrite();
         
         DataInputStream in = getInput("db.SliceFromReadCommand.bin");
-        assert SliceFromReadCommand.serializer().deserialize(in) != null;
-        assert SliceFromReadCommand.serializer().deserialize(in) != null;
-        assert ReadCommand.serializer().deserialize(in) != null;
-        assert ReadCommand.serializer().deserialize(in) != null;
-        assert Message.serializer().deserialize(in) != null;
-        assert Message.serializer().deserialize(in) != 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 Message.serializer().deserialize(in, getVersion()) != null;
+        assert Message.serializer().deserialize(in, getVersion()) != null;
         in.close();
     }
     
     private void testRowWrite() throws IOException
     {
         DataOutputStream out = getOutput("db.Row.bin");
-        Row.serializer().serialize(Statics.StandardRow, out);
-        Row.serializer().serialize(Statics.SuperRow, out);
-        Row.serializer().serialize(Statics.NullRow, out);
+        Row.serializer().serialize(Statics.StandardRow, out, getVersion());
+        Row.serializer().serialize(Statics.SuperRow, out, getVersion());
+        Row.serializer().serialize(Statics.NullRow, out, getVersion());
         out.close();
     }
     
@@ -172,9 +172,9 @@ public class SerializationsTest extends 
             testRowWrite();
         
         DataInputStream in = getInput("db.Row.bin");
-        assert Row.serializer().deserialize(in) != null;
-        assert Row.serializer().deserialize(in) != null;
-        assert Row.serializer().deserialize(in) != null;
+        assert Row.serializer().deserialize(in, getVersion()) != null;
+        assert Row.serializer().deserialize(in, getVersion()) != null;
+        assert Row.serializer().deserialize(in, getVersion()) != null;
         in.close();
     }
     
@@ -193,18 +193,18 @@ public class SerializationsTest extends 
         RowMutation mixedRm = new RowMutation(Statics.KS, Statics.Key, mods);
         
         DataOutputStream out = getOutput("db.RowMutation.bin");
-        RowMutation.serializer().serialize(emptyRm, out);
-        RowMutation.serializer().serialize(standardRowRm, out);
-        RowMutation.serializer().serialize(superRowRm, out);
-        RowMutation.serializer().serialize(standardRm, out);
-        RowMutation.serializer().serialize(superRm, out);
-        RowMutation.serializer().serialize(mixedRm, out);
-        Message.serializer().serialize(emptyRm.getMessage(MessagingService.version_), out);
-        Message.serializer().serialize(standardRowRm.getMessage(MessagingService.version_), out);
-        Message.serializer().serialize(superRowRm.getMessage(MessagingService.version_), out);
-        Message.serializer().serialize(standardRm.getMessage(MessagingService.version_), out);
-        Message.serializer().serialize(superRm.getMessage(MessagingService.version_), out);
-        Message.serializer().serialize(mixedRm.getMessage(MessagingService.version_), out);
+        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());
+        Message.serializer().serialize(emptyRm.getMessage(getVersion()), out, getVersion());
+        Message.serializer().serialize(standardRowRm.getMessage(getVersion()), out, getVersion());
+        Message.serializer().serialize(superRowRm.getMessage(getVersion()), out, getVersion());
+        Message.serializer().serialize(standardRm.getMessage(getVersion()), out, getVersion());
+        Message.serializer().serialize(superRm.getMessage(getVersion()), out, getVersion());
+        Message.serializer().serialize(mixedRm.getMessage(getVersion()), out, getVersion());
         out.close(); 
     }
     
@@ -215,18 +215,18 @@ public class SerializationsTest extends 
             restRowMutationWrite();
         
         DataInputStream in = getInput("db.RowMutation.bin");
-        assert RowMutation.serializer().deserialize(in) != null;
-        assert RowMutation.serializer().deserialize(in) != null;
-        assert RowMutation.serializer().deserialize(in) != null;
-        assert RowMutation.serializer().deserialize(in) != null;
-        assert RowMutation.serializer().deserialize(in) != null;
-        assert RowMutation.serializer().deserialize(in) != null;
-        assert Message.serializer().deserialize(in) != null;
-        assert Message.serializer().deserialize(in) != null;
-        assert Message.serializer().deserialize(in) != null;
-        assert Message.serializer().deserialize(in) != null;
-        assert Message.serializer().deserialize(in) != null;
-        assert Message.serializer().deserialize(in) != 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 Message.serializer().deserialize(in, getVersion()) != null;
+        assert Message.serializer().deserialize(in, getVersion()) != null;
+        assert Message.serializer().deserialize(in, getVersion()) != null;
+        assert Message.serializer().deserialize(in, getVersion()) != null;
+        assert Message.serializer().deserialize(in, getVersion()) != null;
+        assert Message.serializer().deserialize(in, getVersion()) != null;
         in.close();
     }
     
@@ -236,12 +236,12 @@ public class SerializationsTest extends 
         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);
-        TruncateResponse.serializer().serialize(aff, out);
-        TruncateResponse.serializer().serialize(neg, out);
-        Message.serializer().serialize(tr.getMessage(MessagingService.version_), out);
-        Message.serializer().serialize(TruncateResponse.makeTruncateResponseMessage(tr.getMessage(MessagingService.version_), aff), out);
-        Message.serializer().serialize(TruncateResponse.makeTruncateResponseMessage(tr.getMessage(MessagingService.version_), neg), out);
+        Truncation.serializer().serialize(tr, out, getVersion());
+        TruncateResponse.serializer().serialize(aff, out, getVersion());
+        TruncateResponse.serializer().serialize(neg, out, getVersion());
+        Message.serializer().serialize(tr.getMessage(getVersion()), out, getVersion());
+        Message.serializer().serialize(TruncateResponse.makeTruncateResponseMessage(tr.getMessage(getVersion()), aff), out, getVersion());
+        Message.serializer().serialize(TruncateResponse.makeTruncateResponseMessage(tr.getMessage(getVersion()), neg), out, getVersion());
         // todo: notice how CF names weren't validated.
         out.close();
     }
@@ -253,12 +253,12 @@ public class SerializationsTest extends 
             testTruncateWrite();
         
         DataInputStream in = getInput("db.Truncation.bin");
-        assert Truncation.serializer().deserialize(in) != null;
-        assert TruncateResponse.serializer().deserialize(in) != null;
-        assert TruncateResponse.serializer().deserialize(in) != null;
-        assert Message.serializer().deserialize(in) != null;
-        assert Message.serializer().deserialize(in) != null;
-        assert Message.serializer().deserialize(in) != null;
+        assert Truncation.serializer().deserialize(in, getVersion()) != null;
+        assert TruncateResponse.serializer().deserialize(in, getVersion()) != null;
+        assert TruncateResponse.serializer().deserialize(in, getVersion()) != null;
+        assert Message.serializer().deserialize(in, getVersion()) != null;
+        assert Message.serializer().deserialize(in, getVersion()) != null;
+        assert Message.serializer().deserialize(in, getVersion()) != null;
         in.close();
     }
     
@@ -267,8 +267,8 @@ public class SerializationsTest extends 
         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);
-        WriteResponse.serializer().serialize(neg, out);
+        WriteResponse.serializer().serialize(aff, out, getVersion());
+        WriteResponse.serializer().serialize(neg, out, getVersion());
         out.close();
     }
     
@@ -279,8 +279,8 @@ public class SerializationsTest extends 
             testWriteResponseWrite();
         
         DataInputStream in = getInput("db.WriteResponse.bin");
-        assert WriteResponse.serializer().deserialize(in) != null;
-        assert WriteResponse.serializer().deserialize(in) != null;
+        assert WriteResponse.serializer().deserialize(in, getVersion()) != null;
+        assert WriteResponse.serializer().deserialize(in, getVersion()) != null;
         in.close();
     }
     

Modified: cassandra/trunk/test/unit/org/apache/cassandra/gms/GossipDigestTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/gms/GossipDigestTest.java?rev=1067975&r1=1067974&r2=1067975&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/gms/GossipDigestTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/gms/GossipDigestTest.java Mon Feb  7 15:38:07 2011
@@ -29,6 +29,8 @@ import java.io.IOException;
 
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import java.net.InetAddress;
+
+import org.apache.cassandra.net.MessagingService;
 import org.junit.Test;
 
 public class GossipDigestTest
@@ -48,10 +50,10 @@ public class GossipDigestTest
         
         //test the serialization and equals
         DataOutputBuffer output = new DataOutputBuffer();
-        GossipDigest.serializer().serialize(expected, output);
+        GossipDigest.serializer().serialize(expected, output, MessagingService.version_);
         
         ByteArrayInputStream input = new ByteArrayInputStream(output.getData(), 0, output.getLength());
-        GossipDigest actual = GossipDigest.serializer().deserialize(new DataInputStream(input));
+        GossipDigest actual = GossipDigest.serializer().deserialize(new DataInputStream(input), MessagingService.version_);
         assertEquals(0, expected.compareTo(actual));
     }
 

Modified: cassandra/trunk/test/unit/org/apache/cassandra/gms/SerializationsTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/gms/SerializationsTest.java?rev=1067975&r1=1067974&r2=1067975&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/gms/SerializationsTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/gms/SerializationsTest.java Mon Feb  7 15:38:07 2011
@@ -40,10 +40,10 @@ public class SerializationsTest extends 
     private void testEndpointStateWrite() throws IOException 
     {
         DataOutputStream out = getOutput("gms.EndpointState.bin");
-        HeartBeatState.serializer().serialize(Statics.HeartbeatSt, out);
-        EndpointState.serializer().serialize(Statics.EndpointSt, out);
-        VersionedValue.serializer.serialize(Statics.vv0, out);
-        VersionedValue.serializer.serialize(Statics.vv1, out);
+        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();
     }
     
@@ -54,10 +54,10 @@ public class SerializationsTest extends 
             testEndpointStateWrite();
         
         DataInputStream in = getInput("gms.EndpointState.bin");
-        assert HeartBeatState.serializer().deserialize(in) != null;
-        assert EndpointState.serializer().deserialize(in) != null;
-        assert VersionedValue.serializer.deserialize(in) != null;
-        assert VersionedValue.serializer.deserialize(in) != 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();
     }
      
@@ -72,10 +72,10 @@ public class SerializationsTest extends 
         
         DataOutputStream out = getOutput("gms.Gossip.bin");
         for (GossipDigest gd : Statics.Digests)
-            GossipDigest.serializer().serialize(gd, out);
-        GossipDigestAckMessage.serializer().serialize(ack, out);
-        GossipDigestAck2Message.serializer().serialize(ack2, out);
-        GossipDigestSynMessage.serializer().serialize(syn, out);
+            GossipDigest.serializer().serialize(gd, out, getVersion());
+        GossipDigestAckMessage.serializer().serialize(ack, out, getVersion());
+        GossipDigestAck2Message.serializer().serialize(ack2, out, getVersion());
+        GossipDigestSynMessage.serializer().serialize(syn, out, getVersion());
         out.close();
     }
     
@@ -88,10 +88,10 @@ public class SerializationsTest extends 
         int count = 0;
         DataInputStream in = getInput("gms.Gossip.bin");
         while (count < Statics.Digests.size())
-            assert GossipDigestAck2Message.serializer().deserialize(in) != null;
-        assert GossipDigestAckMessage.serializer().deserialize(in) != null;
-        assert GossipDigestAck2Message.serializer().deserialize(in) != null;
-        assert GossipDigestSynMessage.serializer().deserialize(in) != null;
+            assert GossipDigestAck2Message.serializer().deserialize(in, getVersion()) != null;
+        assert GossipDigestAckMessage.serializer().deserialize(in, getVersion()) != null;
+        assert GossipDigestAck2Message.serializer().deserialize(in, getVersion()) != null;
+        assert GossipDigestSynMessage.serializer().deserialize(in, getVersion()) != null;
         in.close();
     }
     

Modified: cassandra/trunk/test/unit/org/apache/cassandra/io/CompactSerializerTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/io/CompactSerializerTest.java?rev=1067975&r1=1067974&r2=1067975&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/io/CompactSerializerTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/io/CompactSerializerTest.java Mon Feb  7 15:38:07 2011
@@ -67,8 +67,6 @@ public class CompactSerializerTest exten
         expectedClassNames.add("StreamHeaderSerializer");
         expectedClassNames.add("FileStatusSerializer");
         expectedClassNames.add("StreamRequestMessageSerializer");
-        expectedClassNames.add("BloomFilterSerializer");
-        expectedClassNames.add("EstimatedHistogramSerializer");
         expectedClassNames.add("LegacyBloomFilterSerializer");
         expectedClassNames.add("CounterMutationSerializer");
         

Modified: cassandra/trunk/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java?rev=1067975&r1=1067974&r2=1067975&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java Mon Feb  7 15:38:07 2011
@@ -43,6 +43,7 @@ import org.apache.cassandra.io.sstable.I
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.io.util.MappedFileDataInput;
+import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 import org.junit.Test;
@@ -202,7 +203,7 @@ public class LazilyCompactedRowTest exte
             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);
+        RowMutation.serializer().serialize(rm, out, MessagingService.version_);
         assert out.getLength() > DatabaseDescriptor.getColumnIndexSize();
         cfs.forceBlockingFlush();
 

Modified: cassandra/trunk/test/unit/org/apache/cassandra/service/SerializationsTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/service/SerializationsTest.java?rev=1067975&r1=1067974&r2=1067975&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/service/SerializationsTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/service/SerializationsTest.java Mon Feb  7 15:38:07 2011
@@ -42,8 +42,8 @@ public class SerializationsTest extends 
     private void testTreeRequestWrite() throws IOException
     {
         DataOutputStream out = getOutput("service.TreeRequest.bin");
-        AntiEntropyService.TreeRequestVerbHandler.SERIALIZER.serialize(Statics.req, out);
-        Message.serializer().serialize(AntiEntropyService.TreeRequestVerbHandler.makeVerb(Statics.req, MessagingService.version_), out);
+        AntiEntropyService.TreeRequestVerbHandler.SERIALIZER.serialize(Statics.req, out, getVersion());
+        Message.serializer().serialize(AntiEntropyService.TreeRequestVerbHandler.makeVerb(Statics.req, getVersion()), out, getVersion());
         out.close();
     }
     
@@ -54,8 +54,8 @@ public class SerializationsTest extends 
             testTreeRequestWrite();
         
         DataInputStream in = getInput("service.TreeRequest.bin");
-        assert AntiEntropyService.TreeRequestVerbHandler.SERIALIZER.deserialize(in) != null;
-        assert Message.serializer().deserialize(in) != null;
+        assert AntiEntropyService.TreeRequestVerbHandler.SERIALIZER.deserialize(in, getVersion()) != null;
+        assert Message.serializer().deserialize(in, getVersion()) != null;
         in.close();
     }
     
@@ -73,10 +73,10 @@ public class SerializationsTest extends 
         }
         AntiEntropyService.Validator v1 = new AntiEntropyService.Validator(Statics.req, mt);
         DataOutputStream out = getOutput("service.TreeResponse.bin");
-        AntiEntropyService.TreeResponseVerbHandler.SERIALIZER.serialize(v0, out);
-        AntiEntropyService.TreeResponseVerbHandler.SERIALIZER.serialize(v1, out);
-        Message.serializer().serialize(AntiEntropyService.TreeResponseVerbHandler.makeVerb(FBUtilities.getLocalAddress(), v0), out);
-        Message.serializer().serialize(AntiEntropyService.TreeResponseVerbHandler.makeVerb(FBUtilities.getLocalAddress(), v1), out);
+        AntiEntropyService.TreeResponseVerbHandler.SERIALIZER.serialize(v0, out, getVersion());
+        AntiEntropyService.TreeResponseVerbHandler.SERIALIZER.serialize(v1, out, getVersion());
+        Message.serializer().serialize(AntiEntropyService.TreeResponseVerbHandler.makeVerb(FBUtilities.getLocalAddress(), v0), out, getVersion());
+        Message.serializer().serialize(AntiEntropyService.TreeResponseVerbHandler.makeVerb(FBUtilities.getLocalAddress(), v1), out, getVersion());
         out.close();
     }
     
@@ -87,10 +87,10 @@ public class SerializationsTest extends 
             testTreeResponseWrite();
         
         DataInputStream in = getInput("service.TreeResponse.bin");
-        assert AntiEntropyService.TreeResponseVerbHandler.SERIALIZER.deserialize(in) != null;
-        assert AntiEntropyService.TreeResponseVerbHandler.SERIALIZER.deserialize(in) != null;
-        assert Message.serializer().deserialize(in) != null;
-        assert Message.serializer().deserialize(in) != null;
+        assert AntiEntropyService.TreeResponseVerbHandler.SERIALIZER.deserialize(in, getVersion()) != null;
+        assert AntiEntropyService.TreeResponseVerbHandler.SERIALIZER.deserialize(in, getVersion()) != null;
+        assert Message.serializer().deserialize(in, getVersion()) != null;
+        assert Message.serializer().deserialize(in, getVersion()) != null;
         in.close();
     }
     

Modified: cassandra/trunk/test/unit/org/apache/cassandra/streaming/SerializationsTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/streaming/SerializationsTest.java?rev=1067975&r1=1067974&r2=1067975&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/streaming/SerializationsTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/streaming/SerializationsTest.java Mon Feb  7 15:38:07 2011
@@ -55,10 +55,10 @@ public class SerializationsTest extends 
         PendingFile noSST = makePendingFile(false, "also_fake", 100, OperationType.RESTORE_REPLICA_COUNT);
         
         DataOutputStream out = getOutput("streaming.PendingFile.bin");
-        PendingFile.serializer().serialize(normal, out);
-        PendingFile.serializer().serialize(noSections, out);
-        PendingFile.serializer().serialize(noSST, out);
-        PendingFile.serializer().serialize(null, out);
+        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();
     }
     
@@ -69,10 +69,10 @@ public class SerializationsTest extends 
             testPendingFileWrite();
         
         DataInputStream in = getInput("streaming.PendingFile.bin");
-        assert PendingFile.serializer().deserialize(in) != null;
-        assert PendingFile.serializer().deserialize(in) != null;
-        assert PendingFile.serializer().deserialize(in) != null;
-        assert PendingFile.serializer().deserialize(in) == 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();
     }
     
@@ -88,11 +88,11 @@ public class SerializationsTest extends 
         StreamHeader sh4 = new StreamHeader("Keyspace1", 125L, makePendingFile(true, "bb", 100, OperationType.BOOTSTRAP), new ArrayList<PendingFile>());
         
         DataOutputStream out = getOutput("streaming.StreamHeader.bin");
-        StreamHeader.serializer().serialize(sh0, out);
-        StreamHeader.serializer().serialize(sh1, out);
-        StreamHeader.serializer().serialize(sh2, out);
-        StreamHeader.serializer().serialize(sh3, out);
-        StreamHeader.serializer().serialize(sh4, out);
+        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();
     }
     
@@ -103,11 +103,11 @@ public class SerializationsTest extends 
             testStreamHeaderWrite();
         
         DataInputStream in = getInput("streaming.StreamHeader.bin");
-        assert StreamHeader.serializer().deserialize(in) != null;
-        assert StreamHeader.serializer().deserialize(in) != null;
-        assert StreamHeader.serializer().deserialize(in) != null;
-        assert StreamHeader.serializer().deserialize(in) != null;
-        assert StreamHeader.serializer().deserialize(in) != 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();
     }
     
@@ -115,8 +115,8 @@ public class SerializationsTest extends 
     {
         StreamReply rep = new StreamReply("this is a file", 123L, StreamReply.Status.FILE_FINISHED);
         DataOutputStream out = getOutput("streaming.StreamReply.bin");
-        StreamReply.serializer.serialize(rep, out);
-        Message.serializer().serialize(rep.getMessage(MessagingService.version_), out);
+        StreamReply.serializer.serialize(rep, out, getVersion());
+        Message.serializer().serialize(rep.getMessage(getVersion()), out, getVersion());
         out.close();
     }
     
@@ -127,8 +127,8 @@ public class SerializationsTest extends 
             testStreamReplyWrite();
         
         DataInputStream in = getInput("streaming.StreamReply.bin");
-        assert StreamReply.serializer.deserialize(in) != null;
-        assert Message.serializer().deserialize(in) != null;
+        assert StreamReply.serializer.deserialize(in, getVersion()) != null;
+        assert Message.serializer().deserialize(in, getVersion()) != null;
         in.close();
     }
     
@@ -151,12 +151,12 @@ public class SerializationsTest extends 
         StreamRequestMessage msg2 = new StreamRequestMessage(FBUtilities.getLocalAddress(), makePendingFile(false, "aa", 100, OperationType.BOOTSTRAP), 124L);
         
         DataOutputStream out = getOutput("streaming.StreamRequestMessage.bin");
-        StreamRequestMessage.serializer().serialize(msg0, out);
-        StreamRequestMessage.serializer().serialize(msg1, out);
-        StreamRequestMessage.serializer().serialize(msg2, out);
-        Message.serializer().serialize(msg0.getMessage(MessagingService.version_), out);
-        Message.serializer().serialize(msg1.getMessage(MessagingService.version_), out);
-        Message.serializer().serialize(msg2.getMessage(MessagingService.version_), out);
+        StreamRequestMessage.serializer().serialize(msg0, out, getVersion());
+        StreamRequestMessage.serializer().serialize(msg1, out, getVersion());
+        StreamRequestMessage.serializer().serialize(msg2, out, getVersion());
+        Message.serializer().serialize(msg0.getMessage(getVersion()), out, getVersion());
+        Message.serializer().serialize(msg1.getMessage(getVersion()), out, getVersion());
+        Message.serializer().serialize(msg2.getMessage(getVersion()), out, getVersion());
         out.close();
     }
     
@@ -167,12 +167,12 @@ public class SerializationsTest extends 
             testStreamRequestMessageWrite();
         
         DataInputStream in = getInput("streaming.StreamRequestMessage.bin");
-        assert StreamRequestMessage.serializer().deserialize(in) != null;
-        assert StreamRequestMessage.serializer().deserialize(in) != null;
-        assert StreamRequestMessage.serializer().deserialize(in) != null;
-        assert Message.serializer().deserialize(in) != null;
-        assert Message.serializer().deserialize(in) != null;
-        assert Message.serializer().deserialize(in) != null;
+        assert StreamRequestMessage.serializer().deserialize(in, getVersion()) != null;
+        assert StreamRequestMessage.serializer().deserialize(in, getVersion()) != null;
+        assert StreamRequestMessage.serializer().deserialize(in, getVersion()) != null;
+        assert Message.serializer().deserialize(in, getVersion()) != null;
+        assert Message.serializer().deserialize(in, getVersion()) != null;
+        assert Message.serializer().deserialize(in, getVersion()) != null;
         in.close();
     }
     

Modified: cassandra/trunk/test/unit/org/apache/cassandra/streaming/StreamUtil.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/streaming/StreamUtil.java?rev=1067975&r1=1067974&r2=1067975&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/streaming/StreamUtil.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/streaming/StreamUtil.java Mon Feb  7 15:38:07 2011
@@ -41,7 +41,7 @@ public class StreamUtil
 
         try
         {
-            StreamRequestMessage srm = StreamRequestMessage.serializer().deserialize(new DataInputStream(bufIn));
+            StreamRequestMessage srm = StreamRequestMessage.serializer().deserialize(new DataInputStream(bufIn), MessagingService.version_);
             StreamInSession session = StreamInSession.get(to, srm.sessionId);
             session.closeIfFinished();
         }

Modified: cassandra/trunk/test/unit/org/apache/cassandra/utils/LegacyBloomFilterTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/utils/LegacyBloomFilterTest.java?rev=1067975&r1=1067974&r2=1067975&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/utils/LegacyBloomFilterTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/utils/LegacyBloomFilterTest.java Mon Feb  7 15:38:07 2011
@@ -43,11 +43,11 @@ public class LegacyBloomFilterTest
     {
         f.add(ByteBufferUtil.bytes("a"));
         DataOutputBuffer out = new DataOutputBuffer();
-        f.serializer().serialize(f, out);
+        f.serializer().serialize(f, out, 0);
 
         ByteArrayInputStream in = new ByteArrayInputStream(out.getData(), 0, out.getLength());
-        LegacyBloomFilter f2 = f.serializer().deserialize(new DataInputStream(in));
-
+        LegacyBloomFilter f2 = f.serializer().deserialize(new DataInputStream(in), 0);
+        
         assert f2.isPresent(ByteBufferUtil.bytes("a"));
         assert !f2.isPresent(ByteBufferUtil.bytes("b"));
         return f2;
@@ -107,12 +107,6 @@ public class LegacyBloomFilterTest
                                       new KeyGenerator.WordGenerator(1, 2));
     }
 
-    @Test
-    public void testSerialize() throws IOException
-    {
-        LegacyBloomFilterTest.testSerialize(bf);
-    }
-
     public void testManyHashes(Iterator<ByteBuffer> keys)
     {
         int MAX_HASH_COUNT = 128;

Modified: cassandra/trunk/test/unit/org/apache/cassandra/utils/SerializationsTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/utils/SerializationsTest.java?rev=1067975&r1=1067974&r2=1067975&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/utils/SerializationsTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/utils/SerializationsTest.java Mon Feb  7 15:38:07 2011
@@ -65,8 +65,8 @@ public class SerializationsTest extends 
             b.add(key);
         }
         DataOutputStream out = getOutput("utils.LegacyBloomFilter.bin");
-        LegacyBloomFilter.serializer().serialize(a, out);
-        LegacyBloomFilter.serializer().serialize(b, out);
+        LegacyBloomFilter.serializer().serialize(a, out, getVersion());
+        LegacyBloomFilter.serializer().serialize(b, out, getVersion());
         out.close();
     }
     
@@ -77,7 +77,7 @@ public class SerializationsTest extends 
             testLegacyBloomFilterWrite();
         
         DataInputStream in = getInput("utils.LegacyBloomFilter.bin");
-        assert LegacyBloomFilter.serializer().deserialize(in) != null;
+        assert LegacyBloomFilter.serializer().deserialize(in, getVersion()) != null;
         in.close();
     }