You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2010/02/02 16:59:55 UTC

svn commit: r905683 - in /incubator/cassandra/trunk/src/java/org/apache/cassandra: db/RangeSliceCommand.java utils/FBUtilities.java

Author: jbellis
Date: Tue Feb  2 15:59:46 2010
New Revision: 905683

URL: http://svn.apache.org/viewvc?rev=905683&view=rev
Log:
add FBUtilities.serialize and deserialize.  patch by jbellis

Modified:
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/RangeSliceCommand.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/utils/FBUtilities.java

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/RangeSliceCommand.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/RangeSliceCommand.java?rev=905683&r1=905682&r2=905683&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/RangeSliceCommand.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/RangeSliceCommand.java Tue Feb  2 15:59:46 2010
@@ -48,7 +48,6 @@
 import org.apache.cassandra.thrift.SlicePredicate;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.thrift.TDeserializer;
-import org.apache.thrift.TException;
 import org.apache.thrift.TSerializer;
 import org.apache.thrift.protocol.TBinaryProtocol;
 
@@ -124,17 +123,7 @@
             dos.write(sliceCommand.super_column);
 
         TSerializer ser = new TSerializer(new TBinaryProtocol.Factory());
-        try
-        {
-            byte[] serPred = ser.serialize(sliceCommand.predicate);
-            dos.writeInt(serPred.length);
-            dos.write(serPred);
-        }
-        catch (TException ex)
-        {
-            throw new IOException(ex);
-        }
-
+        FBUtilities.serialize(ser, sliceCommand.predicate, dos);
         DecoratedKey.serializer().serialize(sliceCommand.startKey, dos);
         DecoratedKey.serializer().serialize(sliceCommand.finishKey, dos);
         dos.writeInt(sliceCommand.max_keys);
@@ -150,18 +139,9 @@
         if (scLength > 0)
             super_column = readBuf(scLength, dis);
 
-        byte[] predBytes = new byte[dis.readInt()];
-        dis.readFully(predBytes);
         TDeserializer dser = new TDeserializer(new TBinaryProtocol.Factory());
-        SlicePredicate pred =  new SlicePredicate();
-        try
-        {
-            dser.deserialize(pred, predBytes);
-        }
-        catch (TException ex)
-        {
-            throw new IOException(ex);
-        }
+        SlicePredicate pred = new SlicePredicate();
+        FBUtilities.deserialize(dser, pred, dis);
 
         DecoratedKey startKey = DecoratedKey.serializer().deserialize(dis);
         DecoratedKey finishKey = DecoratedKey.serializer().deserialize(dis);

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/utils/FBUtilities.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/utils/FBUtilities.java?rev=905683&r1=905682&r2=905683&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/utils/FBUtilities.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/utils/FBUtilities.java Tue Feb  2 15:59:46 2010
@@ -35,6 +35,11 @@
 import org.apache.commons.collections.iterators.CollatingIterator;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.thrift.SlicePredicate;
+import org.apache.thrift.TBase;
+import org.apache.thrift.TDeserializer;
+import org.apache.thrift.TException;
+import org.apache.thrift.TSerializer;
 
 public class FBUtilities
 {
@@ -342,4 +347,41 @@
             i = j;
         }
     }
+
+    public static void serialize(TSerializer serializer, TBase struct, DataOutput out)
+    throws IOException
+    {
+        assert serializer != null;
+        assert struct != null;
+        assert out != null;
+        byte[] bytes;
+        try
+        {
+            bytes = serializer.serialize(struct);
+        }
+        catch (TException e)
+        {
+            throw new RuntimeException(e);
+        }
+        out.writeInt(bytes.length);
+        out.write(bytes);
+    }
+
+    public static void deserialize(TDeserializer deserializer, TBase struct, DataInput in)
+    throws IOException
+    {
+        assert deserializer != null;
+        assert struct != null;
+        assert in != null;
+        byte[] bytes = new byte[in.readInt()];
+        in.readFully(bytes);
+        try
+        {
+            deserializer.deserialize(struct, bytes);
+        }
+        catch (TException ex)
+        {
+            throw new IOException(ex);
+        }
+    }
 }