Modified: 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/net/CompactEndpointSerializationHelper.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/net/CompactEndpointSerializationHelper.java?rev=1182652&r1=1182651&r2=1182652&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/net/CompactEndpointSerializationHelper.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/net/CompactEndpointSerializationHelper.java
 Thu Oct 13 05:08:15 2011
@@ -18,23 +18,21 @@
 
 package org.apache.cassandra.net;
 
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
+import java.io.*;
 import java.net.Inet4Address;
 import java.net.Inet6Address;
 import java.net.InetAddress;
 
 public class CompactEndpointSerializationHelper
 {
-    public static void serialize(InetAddress endpoint, DataOutputStream dos) 
throws IOException
+    public static void serialize(InetAddress endpoint, DataOutput dos) throws 
IOException
     {
         byte[] buf = endpoint.getAddress();
         dos.writeByte(buf.length);
         dos.write(buf);
     }
     
-    public static InetAddress deserialize(DataInputStream dis) throws 
IOException
+    public static InetAddress deserialize(DataInput dis) throws IOException
     {     
         byte[] bytes = new byte[dis.readByte()];
         dis.readFully(bytes, 0, bytes.length);

Modified: 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/net/Header.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/net/Header.java?rev=1182652&r1=1182651&r2=1182652&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/net/Header.java 
(original)
+++ 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/net/Header.java 
Thu Oct 13 05:08:15 2011
@@ -18,28 +18,25 @@
 
 package org.apache.cassandra.net;
 
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
+import java.io.*;
 import java.net.InetAddress;
 import java.util.Hashtable;
 import java.util.Map;
-import java.util.Set;
 
-import org.apache.cassandra.io.ICompactSerializer;
+import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 
 public class Header
 {
-    private static ICompactSerializer<Header> serializer_;
+    private static IVersionedSerializer<Header> serializer_;
 
     static
     {
         serializer_ = new HeaderSerializer();        
     }
     
-    static ICompactSerializer<Header> serializer()
+    static IVersionedSerializer<Header> serializer()
     {
         return serializer_;
     }
@@ -105,9 +102,9 @@ public class Header
     }
 }
 
-class HeaderSerializer implements ICompactSerializer<Header>
+class HeaderSerializer implements IVersionedSerializer<Header>
 {
-    public void serialize(Header t, DataOutputStream dos, int version) throws 
IOException
+    public void serialize(Header t, DataOutput dos, int version) throws 
IOException
     {           
         CompactEndpointSerializationHelper.serialize(t.getFrom(), dos);
         dos.writeInt(t.getVerb().ordinal());
@@ -121,7 +118,7 @@ class HeaderSerializer implements ICompa
         }
     }
 
-    public Header deserialize(DataInputStream dis, int version) throws 
IOException
+    public Header deserialize(DataInput dis, int version) throws IOException
     {
         InetAddress from = CompactEndpointSerializationHelper.deserialize(dis);
         int verbOrdinal = dis.readInt();
@@ -137,6 +134,11 @@ class HeaderSerializer implements ICompa
         }
         return new Header(from, StorageService.VERBS[verbOrdinal], details);
     }
+
+    public long serializedSize(Header header, int version)
+    {
+        throw new UnsupportedOperationException();
+    }
 }
 
 

Modified: 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/net/Message.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/net/Message.java?rev=1182652&r1=1182651&r2=1182652&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/net/Message.java 
(original)
+++ 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/net/Message.java 
Thu Oct 13 05:08:15 2011
@@ -18,13 +18,9 @@
 
 package org.apache.cassandra.net;
 
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
 import java.net.InetAddress;
 
 import org.apache.cassandra.concurrent.Stage;
-import org.apache.cassandra.io.ICompactSerializer;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 

Modified: 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/service/AntiEntropyService.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/service/AntiEntropyService.java?rev=1182652&r1=1182651&r2=1182652&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/service/AntiEntropyService.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/service/AntiEntropyService.java
 Thu Oct 13 05:08:15 2011
@@ -42,7 +42,7 @@ import org.apache.cassandra.dht.Abstract
 import org.apache.cassandra.dht.RandomPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.gms.*;
-import org.apache.cassandra.io.ICompactSerializer;
+import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.FastByteArrayInputStream;
 import org.apache.cassandra.io.util.FastByteArrayOutputStream;
 import org.apache.cassandra.net.CompactEndpointSerializationHelper;
@@ -393,7 +393,7 @@ public class AntiEntropyService
      * Handler for requests from remote nodes to generate a valid tree.
      * The payload is a CFPair representing the columnfamily to validate.
      */
-    public static class TreeRequestVerbHandler implements IVerbHandler, 
ICompactSerializer<TreeRequest>
+    public static class TreeRequestVerbHandler implements IVerbHandler
     {
         public static final TreeRequestVerbHandler SERIALIZER = new 
TreeRequestVerbHandler();
         static Message makeVerb(TreeRequest request, int version)
@@ -411,7 +411,7 @@ public class AntiEntropyService
             }
         }
 
-        public void serialize(TreeRequest request, DataOutputStream dos, int 
version) throws IOException
+        public void serialize(TreeRequest request, DataOutput dos, int 
version) throws IOException
         {
             dos.writeUTF(request.sessionid);
             CompactEndpointSerializationHelper.serialize(request.endpoint, 
dos);
@@ -421,7 +421,7 @@ public class AntiEntropyService
                 AbstractBounds.serializer().serialize(request.range, dos);
         }
 
-        public TreeRequest deserialize(DataInputStream dis, int version) 
throws IOException
+        public TreeRequest deserialize(DataInput dis, int version) throws 
IOException
         {
             String sessId = dis.readUTF();
             InetAddress endpoint = 
CompactEndpointSerializationHelper.deserialize(dis);
@@ -465,7 +465,7 @@ public class AntiEntropyService
      * Handler for responses from remote nodes which contain a valid tree.
      * The payload is a completed Validator object from the remote endpoint.
      */
-    public static class TreeResponseVerbHandler implements IVerbHandler, 
ICompactSerializer<Validator>
+    public static class TreeResponseVerbHandler implements IVerbHandler
     {
         public static final TreeResponseVerbHandler SERIALIZER = new 
TreeResponseVerbHandler();
         static Message makeVerb(InetAddress local, Validator validator)

Modified: 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/PendingFile.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/PendingFile.java?rev=1182652&r1=1182651&r2=1182652&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/PendingFile.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/PendingFile.java
 Thu Oct 13 05:08:15 2011
@@ -21,13 +21,11 @@ package org.apache.cassandra.streaming;
  */
 
 
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
+import java.io.*;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.cassandra.io.ICompactSerializer;
+import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.net.MessagingService;
@@ -108,9 +106,9 @@ public class PendingFile
         return getFilename() + " sections=" + sections.size() + " progress=" + 
progress + "/" + size + " - " + progress*100/size + "%";
     }
 
-    public static class PendingFileSerializer implements 
ICompactSerializer<PendingFile>
+    public static class PendingFileSerializer implements 
IVersionedSerializer<PendingFile>
     {
-        public void serialize(PendingFile sc, DataOutputStream dos, int 
version) throws IOException
+        public void serialize(PendingFile sc, DataOutput dos, int version) 
throws IOException
         {
             if (sc == null)
             {
@@ -131,7 +129,7 @@ public class PendingFile
                 dos.writeLong(sc.estimatedKeys);
         }
 
-        public PendingFile deserialize(DataInputStream dis, int version) 
throws IOException
+        public PendingFile deserialize(DataInput dis, int version) throws 
IOException
         {
             String filename = dis.readUTF();
             if (filename.isEmpty())
@@ -152,5 +150,10 @@ public class PendingFile
                 estimatedKeys = dis.readLong();
             return new PendingFile(null, desc, component, sections, type, 
estimatedKeys);
         }
+
+        public long serializedSize(PendingFile pendingFile, int version)
+        {
+            throw new UnsupportedOperationException();
+        }
     }
 }

Modified: 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/StreamHeader.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/StreamHeader.java?rev=1182652&r1=1182651&r2=1182652&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/StreamHeader.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/StreamHeader.java
 Thu Oct 13 05:08:15 2011
@@ -21,26 +21,24 @@ package org.apache.cassandra.streaming;
  *
  */
 
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
+import java.io.*;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.cassandra.io.ICompactSerializer;
+import org.apache.cassandra.io.IVersionedSerializer;
 
 public class StreamHeader
 {
-    private static ICompactSerializer<StreamHeader> serializer;
+    private static IVersionedSerializer<StreamHeader> serializer;
 
     static
     {
         serializer = new StreamHeaderSerializer();
     }
 
-    public static ICompactSerializer<StreamHeader> serializer()
+    public static IVersionedSerializer<StreamHeader> serializer()
     {
         return serializer;
     }
@@ -69,9 +67,9 @@ public class StreamHeader
         this.pendingFiles = pendingFiles;
     }
 
-    private static class StreamHeaderSerializer implements 
ICompactSerializer<StreamHeader>
+    private static class StreamHeaderSerializer implements 
IVersionedSerializer<StreamHeader>
     {
-        public void serialize(StreamHeader sh, DataOutputStream dos, int 
version) throws IOException
+        public void serialize(StreamHeader sh, DataOutput dos, int version) 
throws IOException
         {
             dos.writeUTF(sh.table);
             dos.writeLong(sh.sessionId);
@@ -83,7 +81,7 @@ public class StreamHeader
             }
         }
 
-        public StreamHeader deserialize(DataInputStream dis, int version) 
throws IOException
+        public StreamHeader deserialize(DataInput dis, int version) throws 
IOException
         {
             String table = dis.readUTF();
             long sessionId = dis.readLong();
@@ -98,5 +96,10 @@ public class StreamHeader
 
             return new StreamHeader(table, sessionId, file, pendingFiles);
         }
+
+        public long serializedSize(StreamHeader streamHeader, int version)
+        {
+            throw new UnsupportedOperationException();
+        }
     }
 }
\ No newline at end of file

Modified: 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/StreamReply.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/StreamReply.java?rev=1182652&r1=1182651&r2=1182652&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/StreamReply.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/StreamReply.java
 Thu Oct 13 05:08:15 2011
@@ -21,11 +21,9 @@ package org.apache.cassandra.streaming;
  */
 
 
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
+import java.io.*;
 
-import org.apache.cassandra.io.ICompactSerializer;
+import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.FastByteArrayOutputStream;
 import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessageProducer;
@@ -41,7 +39,7 @@ class StreamReply implements MessageProd
         SESSION_FINISHED,
     }
 
-    public static final ICompactSerializer<StreamReply> serializer = new 
FileStatusSerializer();
+    public static final IVersionedSerializer<StreamReply> serializer = new 
FileStatusSerializer();
 
     public final long sessionId;
     public final String file;
@@ -72,21 +70,26 @@ class StreamReply implements MessageProd
                ')';
     }
 
-    private static class FileStatusSerializer implements 
ICompactSerializer<StreamReply>
+    private static class FileStatusSerializer implements 
IVersionedSerializer<StreamReply>
     {
-        public void serialize(StreamReply reply, DataOutputStream dos, int 
version) throws IOException
+        public void serialize(StreamReply reply, DataOutput dos, int version) 
throws IOException
         {
             dos.writeLong(reply.sessionId);
             dos.writeUTF(reply.file);
             dos.writeInt(reply.action.ordinal());
         }
 
-        public StreamReply deserialize(DataInputStream dis, int version) 
throws IOException
+        public StreamReply deserialize(DataInput dis, int version) throws 
IOException
         {
             long sessionId = dis.readLong();
             String targetFile = dis.readUTF();
             Status action = Status.values()[dis.readInt()];
             return new StreamReply(targetFile, sessionId, action);
         }
+
+        public long serializedSize(StreamReply streamReply, int version)
+        {
+            throw new UnsupportedOperationException();
+        }
     }
 }

Modified: 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/StreamRequestMessage.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/StreamRequestMessage.java?rev=1182652&r1=1182651&r2=1182652&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/StreamRequestMessage.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/StreamRequestMessage.java
 Thu Oct 13 05:08:15 2011
@@ -27,12 +27,11 @@ import java.util.*;
 
 import com.google.common.collect.Iterables;
 
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Table;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.io.ICompactSerializer;
+import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.FastByteArrayOutputStream;
 import org.apache.cassandra.net.CompactEndpointSerializationHelper;
 import org.apache.cassandra.net.Message;
@@ -50,13 +49,13 @@ import org.apache.cassandra.utils.FBUtil
 */
 class StreamRequestMessage implements MessageProducer
 {
-    private static ICompactSerializer<StreamRequestMessage> serializer_;
+    private static IVersionedSerializer<StreamRequestMessage> serializer_;
     static
     {
         serializer_ = new StreamRequestMessageSerializer();
     }
 
-    protected static ICompactSerializer<StreamRequestMessage> serializer()
+    protected static IVersionedSerializer<StreamRequestMessage> serializer()
     {
         return serializer_;
     }
@@ -135,9 +134,9 @@ class StreamRequestMessage implements Me
         return sb.toString();
     }
 
-    private static class StreamRequestMessageSerializer implements 
ICompactSerializer<StreamRequestMessage>
+    private static class StreamRequestMessageSerializer implements 
IVersionedSerializer<StreamRequestMessage>
     {
-        public void serialize(StreamRequestMessage srm, DataOutputStream dos, 
int version) throws IOException
+        public void serialize(StreamRequestMessage srm, DataOutput dos, int 
version) throws IOException
         {
             dos.writeLong(srm.sessionId);
             CompactEndpointSerializationHelper.serialize(srm.target, dos);
@@ -168,7 +167,7 @@ class StreamRequestMessage implements Me
             }
         }
 
-        public StreamRequestMessage deserialize(DataInputStream dis, int 
version) throws IOException
+        public StreamRequestMessage deserialize(DataInput dis, int version) 
throws IOException
         {
             long sessionId = dis.readLong();
             InetAddress target = 
CompactEndpointSerializationHelper.deserialize(dis);
@@ -202,5 +201,10 @@ class StreamRequestMessage implements Me
                 return new StreamRequestMessage(target, ranges, table, stores, 
sessionId, type);
             }
         }
+
+        public long serializedSize(StreamRequestMessage streamRequestMessage, 
int version)
+        {
+            throw new UnsupportedOperationException();
+        }
     }
 }

Modified: 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java?rev=1182652&r1=1182651&r2=1182652&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java
 Thu Oct 13 05:08:15 2011
@@ -33,7 +33,7 @@ import org.apache.cassandra.db.Table;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.gms.Gossiper;
-import org.apache.cassandra.io.ICompactSerializer;
+import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.net.*;
 import org.apache.cassandra.service.StorageService;
@@ -267,9 +267,9 @@ public class StreamingRepairTask impleme
         }
     }
 
-    private static class StreamingRepairTaskSerializer implements 
ICompactSerializer<StreamingRepairTask>
+    private static class StreamingRepairTaskSerializer implements 
IVersionedSerializer<StreamingRepairTask>
     {
-        public void serialize(StreamingRepairTask task, DataOutputStream dos, 
int version) throws IOException
+        public void serialize(StreamingRepairTask task, DataOutput dos, int 
version) throws IOException
         {
             UUIDGen.write(task.id, dos);
             CompactEndpointSerializationHelper.serialize(task.owner, dos);
@@ -285,14 +285,14 @@ public class StreamingRepairTask impleme
             // We don't serialize the callback on purpose
         }
 
-        public StreamingRepairTask deserialize(DataInputStream dis, int 
version) throws IOException
+        public StreamingRepairTask deserialize(DataInput dis, int version) 
throws IOException
         {
             UUID id = UUIDGen.read(dis);
             InetAddress owner = 
CompactEndpointSerializationHelper.deserialize(dis);
             InetAddress src = 
CompactEndpointSerializationHelper.deserialize(dis);
             InetAddress dst = 
CompactEndpointSerializationHelper.deserialize(dis);
-            String tableName = dis.readUTF(dis);
-            String cfName = dis.readUTF(dis);
+            String tableName = dis.readUTF();
+            String cfName = dis.readUTF();
             int rangesCount = dis.readInt();
             List<Range> ranges = new ArrayList<Range>(rangesCount);
             for (int i = 0; i < rangesCount; ++i)
@@ -301,5 +301,10 @@ public class StreamingRepairTask impleme
             }
             return new StreamingRepairTask(id, owner, src, dst, tableName, 
cfName, ranges, makeReplyingCallback(owner, id));
         }
+
+        public long serializedSize(StreamingRepairTask task, int version)
+        {
+            throw new UnsupportedOperationException();
+        }
     }
 }

Modified: 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/BloomFilter.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/BloomFilter.java?rev=1182652&r1=1182651&r2=1182652&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/BloomFilter.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/BloomFilter.java
 Thu Oct 13 05:08:15 2011
@@ -20,7 +20,6 @@ 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 +30,7 @@ public class BloomFilter extends Filter
 
     private static final Logger logger = 
LoggerFactory.getLogger(BloomFilter.class);
     private static final int EXCESS = 20;
-    static ICompactSerializer2<BloomFilter> serializer_ = new 
BloomFilterSerializer();
+    static BloomFilterSerializer serializer_ = new BloomFilterSerializer();
 
     public OpenBitSet bitset;
 
@@ -46,7 +45,7 @@ public class BloomFilter extends Filter
         return new BloomFilter(0, bucketsFor(0, 0));
     }
 
-    public static ICompactSerializer2<BloomFilter> serializer()
+    public static BloomFilterSerializer serializer()
     {
         return serializer_;
     }
@@ -141,8 +140,8 @@ public class BloomFilter extends Filter
         bitset.clear(0, bitset.size());
     }
 
-    public int serializedSize()
+    public long serializedSize()
     {
-        return BloomFilterSerializer.serializedSize(this);
+        return serializer_.serializedSize(this);
     }
 }

Modified: 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java?rev=1182652&r1=1182651&r2=1182652&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java
 Thu Oct 13 05:08:15 2011
@@ -26,10 +26,10 @@ import java.io.DataOutput;
 import java.io.IOException;
 
 import org.apache.cassandra.db.DBConstants;
-import org.apache.cassandra.io.ICompactSerializer2;
+import org.apache.cassandra.io.ISerializer;
 import org.apache.cassandra.utils.obs.OpenBitSet;
 
-class BloomFilterSerializer implements ICompactSerializer2<BloomFilter>
+public class BloomFilterSerializer implements ISerializer<BloomFilter>
 {
     public void serialize(BloomFilter bf, DataOutput dos) throws IOException
     {
@@ -62,7 +62,7 @@ class BloomFilterSerializer implements I
      *
      * @return serialized size of the given bloom filter
      */
-    public static int serializedSize(BloomFilter bf)
+    public long serializedSize(BloomFilter bf)
     {
         return DBConstants.intSize // hash count
                + DBConstants.intSize // length

Modified: 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/EstimatedHistogram.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/EstimatedHistogram.java?rev=1182652&r1=1182651&r2=1182652&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/EstimatedHistogram.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/EstimatedHistogram.java
 Thu Oct 13 05:08:15 2011
@@ -24,7 +24,7 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.concurrent.atomic.AtomicLongArray;
 
-import org.apache.cassandra.io.ICompactSerializer2;
+import org.apache.cassandra.io.ISerializer;
 
 public class EstimatedHistogram
 {
@@ -202,7 +202,7 @@ public class EstimatedHistogram
                Arrays.equals(getBuckets(false), that.getBuckets(false));
     }
 
-    public static class EstimatedHistogramSerializer implements 
ICompactSerializer2<EstimatedHistogram>
+    public static class EstimatedHistogramSerializer implements 
ISerializer<EstimatedHistogram>
     {
         public void serialize(EstimatedHistogram eh, DataOutput dos) throws 
IOException
         {
@@ -228,5 +228,10 @@ public class EstimatedHistogram
             }
             return new EstimatedHistogram(offsets, buckets);
         }
+
+        public long serializedSize(EstimatedHistogram object)
+        {
+            throw new UnsupportedOperationException();
+        }
     }
 }

Modified: 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/LegacyBloomFilter.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/LegacyBloomFilter.java?rev=1182652&r1=1182651&r2=1182652&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/LegacyBloomFilter.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/LegacyBloomFilter.java
 Thu Oct 13 05:08:15 2011
@@ -21,7 +21,7 @@ package org.apache.cassandra.utils;
 import java.nio.ByteBuffer;
 import java.util.BitSet;
 
-import org.apache.cassandra.io.ICompactSerializer;
+import org.apache.cassandra.io.IVersionedSerializer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -29,9 +29,9 @@ public class LegacyBloomFilter extends F
 {
     private static final int EXCESS = 20;
     private static final Logger logger = 
LoggerFactory.getLogger(LegacyBloomFilter.class);
-    static ICompactSerializer<LegacyBloomFilter> serializer_ = new 
LegacyBloomFilterSerializer();
+    static LegacyBloomFilterSerializer serializer_ = new 
LegacyBloomFilterSerializer();
 
-    public static ICompactSerializer<LegacyBloomFilter> serializer()
+    public static LegacyBloomFilterSerializer serializer()
     {
         return serializer_;
     }
@@ -121,11 +121,6 @@ public class LegacyBloomFilter extends F
         return filter_.toString();
     }
 
-    ICompactSerializer<LegacyBloomFilter> tserializer()
-    {
-        return serializer_;
-    }
-
     int emptyBuckets()
     {
         int n = 0;

Modified: 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/LegacyBloomFilterSerializer.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/LegacyBloomFilterSerializer.java?rev=1182652&r1=1182651&r2=1182652&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/LegacyBloomFilterSerializer.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/LegacyBloomFilterSerializer.java
 Thu Oct 13 05:08:15 2011
@@ -21,16 +21,14 @@ package org.apache.cassandra.utils;
  */
 
 
+import java.io.*;
 import java.util.BitSet;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import org.apache.cassandra.io.ICompactSerializer;
 
-class LegacyBloomFilterSerializer implements 
ICompactSerializer<LegacyBloomFilter>
+import org.apache.cassandra.io.IVersionedSerializer;
+
+public class LegacyBloomFilterSerializer
 {
-    public void serialize(LegacyBloomFilter bf, DataOutputStream dos, int 
version)
+    public void serialize(LegacyBloomFilter bf, DataOutput dos)
             throws IOException
     {
         throw new UnsupportedOperationException("Shouldn't be serializing 
legacy bloom filters");
@@ -40,7 +38,7 @@ class LegacyBloomFilterSerializer implem
 //        oos.flush();
     }
 
-    public LegacyBloomFilter deserialize(DataInputStream dis, int version) 
throws IOException
+    public LegacyBloomFilter deserialize(DataInputStream dis) throws 
IOException
     {
         int hashes = dis.readInt();
         ObjectInputStream ois = new ObjectInputStream(dis);
@@ -53,4 +51,9 @@ class LegacyBloomFilterSerializer implem
           throw new RuntimeException(e);
         }
     }
+
+    public long serializedSize(LegacyBloomFilter legacyBloomFilter)
+    {
+        throw new UnsupportedOperationException();
+    }
 }

Modified: 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/MerkleTree.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/MerkleTree.java?rev=1182652&r1=1182651&r2=1182652&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/MerkleTree.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/MerkleTree.java
 Thu Oct 13 05:08:15 2011
@@ -18,12 +18,7 @@
 */
 package org.apache.cassandra.utils;
 
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.io.Serializable;
+import java.io.*;
 import java.util.*;
 
 import com.google.common.collect.AbstractIterator;
@@ -32,7 +27,7 @@ import com.google.common.collect.Peeking
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.io.ICompactSerializer;
+import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.net.MessagingService;
 
 /**
@@ -84,7 +79,7 @@ public class MerkleTree implements Seria
     private long size;
     private Hashable root;
     
-    public static class MerkleTreeSerializer implements 
ICompactSerializer<MerkleTree>
+    public static class MerkleTreeSerializer
     {
         public void serialize(MerkleTree mt, DataOutputStream dos, int 
version) throws IOException
         {
@@ -632,7 +627,7 @@ public class MerkleTree implements Seria
         private Hashable lchild;
         private Hashable rchild;
 
-        private static ICompactSerializer<Inner> serializer = new 
InnerSerializer();
+        private static InnerSerializer serializer = new InnerSerializer();
         
         /**
          * Constructs an Inner with the given token and children, and a null 
hash.
@@ -701,9 +696,9 @@ public class MerkleTree implements Seria
             return buff.toString();
         }
         
-        private static class InnerSerializer implements 
ICompactSerializer<Inner>
+        private static class InnerSerializer
         {
-            public void serialize(Inner inner, DataOutputStream dos, int 
version) throws IOException
+            public void serialize(Inner inner, DataOutput dos, int version) 
throws IOException
             {
                 if (inner.hash == null)
                     dos.writeInt(-1);
@@ -717,7 +712,7 @@ public class MerkleTree implements Seria
                 Hashable.serializer.serialize(inner.rchild, dos, version);
             }
 
-            public Inner deserialize(DataInputStream dis, int version) throws 
IOException
+            public Inner deserialize(DataInput dis, int version) throws 
IOException
             {
                 int hashLen = dis.readInt();
                 byte[] hash = hashLen >= 0 ? new byte[hashLen] : null;
@@ -744,7 +739,7 @@ public class MerkleTree implements Seria
     {
         public static final long serialVersionUID = 1L;
         static final byte IDENT = 1;
-        private static ICompactSerializer<Leaf> serializer = new 
LeafSerializer();
+        private static LeafSerializer serializer = new LeafSerializer();
         
         /**
          * Constructs a null hash.
@@ -775,9 +770,9 @@ public class MerkleTree implements Seria
             return "#<Leaf " + Hashable.toString(hash()) + ">";
         }
 
-        private static class LeafSerializer implements ICompactSerializer<Leaf>
+        private static class LeafSerializer
         {
-            public void serialize(Leaf leaf, DataOutputStream dos, int 
version) throws IOException
+            public void serialize(Leaf leaf, DataOutput dos) throws IOException
             {
                 if (leaf.hash == null)
                     dos.writeInt(-1);
@@ -788,7 +783,7 @@ public class MerkleTree implements Seria
                 }
             }
 
-            public Leaf deserialize(DataInputStream dis, int version) throws 
IOException
+            public Leaf deserialize(DataInput dis) throws IOException
             {
                 int hashLen = dis.readInt();
                 byte[] hash = hashLen < 0 ? null : new byte[hashLen];
@@ -827,7 +822,7 @@ public class MerkleTree implements Seria
     static abstract class Hashable implements Serializable
     {
         private static final long serialVersionUID = 1L;
-        private static ICompactSerializer<Hashable> serializer = new 
HashableSerializer();
+        private static IVersionedSerializer<Hashable> serializer = new 
HashableSerializer();
 
         protected byte[] hash;
 
@@ -886,9 +881,9 @@ public class MerkleTree implements Seria
             return "[" + Hex.bytesToHex(hash) + "]";
         }
         
-        private static class HashableSerializer implements 
ICompactSerializer<Hashable>
+        private static class HashableSerializer implements 
IVersionedSerializer<Hashable>
         {
-            public void serialize(Hashable h, DataOutputStream dos, int 
version) throws IOException
+            public void serialize(Hashable h, DataOutput dos, int version) 
throws IOException
             {
                 if (h instanceof Inner) 
                 {
@@ -898,22 +893,27 @@ public class MerkleTree implements Seria
                 else if (h instanceof Leaf)
                 {
                     dos.writeByte(Leaf.IDENT);
-                    Leaf.serializer.serialize((Leaf)h, dos, version);
+                    Leaf.serializer.serialize((Leaf)h, dos);
                 }
                 else
                     throw new IOException("Unexpected Hashable: " + 
h.getClass().getCanonicalName());
             }
 
-            public Hashable deserialize(DataInputStream dis, int version) 
throws IOException
+            public Hashable deserialize(DataInput dis, int version) throws 
IOException
             {
                 byte ident = dis.readByte();
                 if (Inner.IDENT == ident)
                     return Inner.serializer.deserialize(dis, version);
                 else if (Leaf.IDENT == ident)
-                    return Leaf.serializer.deserialize(dis, version);
+                    return Leaf.serializer.deserialize(dis);
                 else
                     throw new IOException("Unexpected Hashable: " + ident);
             }
+
+            public long serializedSize(Hashable hashable, int version)
+            {
+                throw new UnsupportedOperationException();
+            }
         }
     }
 

Modified: 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/UUIDGen.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/UUIDGen.java?rev=1182652&r1=1182651&r2=1182652&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/UUIDGen.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/utils/UUIDGen.java
 Thu Oct 13 05:08:15 2011
@@ -20,9 +20,7 @@ package org.apache.cassandra.utils;
  * 
  */
 
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
+import java.io.*;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
@@ -71,13 +69,13 @@ public class UUIDGen
     }
 
     /** reads a uuid from an input stream. */
-    public static UUID read(DataInputStream dis) throws IOException
+    public static UUID read(DataInput dis) throws IOException
     {
         return new UUID(dis.readLong(), dis.readLong());
     }
 
     /** writes a uuid to an output stream. */
-    public static void write(UUID uuid, DataOutputStream dos) throws 
IOException
+    public static void write(UUID uuid, DataOutput dos) throws IOException
     {
         dos.writeLong(uuid.getMostSignificantBits());
         dos.writeLong(uuid.getLeastSignificantBits());
@@ -142,7 +140,7 @@ public class UUIDGen
      * 
      * @param uuid a type-1 (time-based) UUID
      * @return the number of milliseconds since the unix epoch
-     * @throws InvalidArgumentException if the UUID is not version 1
+     * @throws IllegalArgumentException if the UUID is not version 1
      */
     public static long getAdjustedTimestamp(UUID uuid)
     {

Modified: 
cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/gms/GossipDigestTest.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/gms/GossipDigestTest.java?rev=1182652&r1=1182651&r2=1182652&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/gms/GossipDigestTest.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/gms/GossipDigestTest.java
 Thu Oct 13 05:08:15 2011
@@ -35,7 +35,6 @@ import org.junit.Test;
 
 public class GossipDigestTest
 {
-
     @Test
     public void test() throws IOException
     {
@@ -56,5 +55,4 @@ public class GossipDigestTest
         GossipDigest actual = GossipDigest.serializer().deserialize(new 
DataInputStream(input), MessagingService.version_);
         assertEquals(0, expected.compareTo(actual));
     }
-
 }

Modified: 
cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/io/CompactSerializerTest.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/io/CompactSerializerTest.java?rev=1182652&r1=1182651&r2=1182652&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/io/CompactSerializerTest.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/io/CompactSerializerTest.java
 Thu Oct 13 05:08:15 2011
@@ -61,18 +61,12 @@ public class CompactSerializerTest exten
         expectedClassNames.add("VersionedValueSerializer");
         expectedClassNames.add("HeaderSerializer");
         expectedClassNames.add("MessageSerializer");
-        expectedClassNames.add("TreeRequestVerbHandler");
-        expectedClassNames.add("TreeResponseVerbHandler");
         expectedClassNames.add("PendingFileSerializer");
         expectedClassNames.add("StreamHeaderSerializer");
         expectedClassNames.add("FileStatusSerializer");
         expectedClassNames.add("StreamRequestMessageSerializer");
-        expectedClassNames.add("LegacyBloomFilterSerializer");
         expectedClassNames.add("CounterMutationSerializer");
         expectedClassNames.add("HashableSerializer");
-        expectedClassNames.add("InnerSerializer");
-        expectedClassNames.add("LeafSerializer");
-        expectedClassNames.add("MerkleTreeSerializer");
         expectedClassNames.add("StreamingRepairTaskSerializer");
         
         discoveredClassNames = new ArrayList<String>();
@@ -107,7 +101,7 @@ public class CompactSerializerTest exten
                                 if(t instanceof ParameterizedType)
                                 {
                                     ParameterizedType pt = 
(ParameterizedType)t;
-                                    if 
(((Class)pt.getRawType()).getSimpleName().equals("ICompactSerializer"))
+                                    if 
(((Class)pt.getRawType()).getSimpleName().equals("IVersionedSerializer"))
                                     {
                                         discoveredClassNames.add(simpleName);
                                         break classTraversal;

Modified: 
cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/net/MessageSerializer.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/net/MessageSerializer.java?rev=1182652&r1=1182651&r2=1182652&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/net/MessageSerializer.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/net/MessageSerializer.java
 Thu Oct 13 05:08:15 2011
@@ -21,15 +21,13 @@ package org.apache.cassandra.net;
  */
 
 
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
+import java.io.*;
 
-import org.apache.cassandra.io.ICompactSerializer;
+import org.apache.cassandra.io.IVersionedSerializer;
 
-public class MessageSerializer implements ICompactSerializer<Message>
+public class MessageSerializer implements IVersionedSerializer<Message>
 {
-    public void serialize(Message t, DataOutputStream dos, int version) throws 
IOException
+    public void serialize(Message t, DataOutput dos, int version) throws 
IOException
     {
         assert t.getVersion() == version : "internode protocol version 
mismatch"; // indicates programmer error.
         Header.serializer().serialize( t.header_, dos, version);
@@ -38,7 +36,7 @@ public class MessageSerializer implement
         dos.write(bytes);
     }
 
-    public Message deserialize(DataInputStream dis, int version) throws 
IOException
+    public Message deserialize(DataInput dis, int version) throws IOException
     {
         Header header = Header.serializer().deserialize(dis, version);
         int size = dis.readInt();
@@ -46,4 +44,9 @@ public class MessageSerializer implement
         dis.readFully(bytes);
         return new Message(header, bytes, version);
     }
+
+    public long serializedSize(Message message, int version)
+    {
+        throw new UnsupportedOperationException();
+    }
 }

Modified: 
cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/utils/LegacyBloomFilterTest.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/utils/LegacyBloomFilterTest.java?rev=1182652&r1=1182651&r2=1182652&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/utils/LegacyBloomFilterTest.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/utils/LegacyBloomFilterTest.java
 Thu Oct 13 05:08:15 2011
@@ -43,10 +43,10 @@ public class LegacyBloomFilterTest
     {
         f.add(ByteBufferUtil.bytes("a"));
         DataOutputBuffer out = new DataOutputBuffer();
-        f.serializer().serialize(f, out, 0);
+        f.serializer().serialize(f, out);
 
         ByteArrayInputStream in = new ByteArrayInputStream(out.getData(), 0, 
out.getLength());
-        LegacyBloomFilter f2 = f.serializer().deserialize(new 
DataInputStream(in), 0);
+        LegacyBloomFilter f2 = f.serializer().deserialize(new 
DataInputStream(in));
         
         assert f2.isPresent(ByteBufferUtil.bytes("a"));
         assert !f2.isPresent(ByteBufferUtil.bytes("b"));

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


Reply via email to