Author: eevans
Date: Fri Dec 31 17:24:51 2010
New Revision: 1054141

URL: http://svn.apache.org/viewvc?rev=1054141&view=rev
Log:
port java driver, avro -> thrift

Patch by eevans; reviewed by jbellis for CASSANDRA-1913

Modified:
    
cassandra/trunk/drivers/java/src/org/apache/cassandra/cql/driver/Connection.java

Modified: 
cassandra/trunk/drivers/java/src/org/apache/cassandra/cql/driver/Connection.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/drivers/java/src/org/apache/cassandra/cql/driver/Connection.java?rev=1054141&r1=1054140&r2=1054141&view=diff
==============================================================================
--- 
cassandra/trunk/drivers/java/src/org/apache/cassandra/cql/driver/Connection.java
 (original)
+++ 
cassandra/trunk/drivers/java/src/org/apache/cassandra/cql/driver/Connection.java
 Fri Dec 31 17:24:51 2010
@@ -21,21 +21,22 @@
 package org.apache.cassandra.cql.driver;
 
 import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.net.URL;
 import java.nio.ByteBuffer;
 import java.util.zip.Deflater;
 
-import org.apache.avro.ipc.AvroRemoteException;
-import org.apache.avro.ipc.HttpTransceiver;
-import org.apache.avro.specific.SpecificRequestor;
-import org.apache.cassandra.avro.Cassandra;
-import org.apache.cassandra.avro.Compression;
-import org.apache.cassandra.avro.CqlResult;
-import org.apache.cassandra.avro.CqlRow;
-import org.apache.cassandra.avro.InvalidRequestException;
-import org.apache.cassandra.avro.TimedOutException;
-import org.apache.cassandra.avro.UnavailableException;
+import org.apache.cassandra.thrift.Cassandra;
+import org.apache.cassandra.thrift.Compression;
+import org.apache.cassandra.thrift.CqlResult;
+import org.apache.cassandra.thrift.CqlRow;
+import org.apache.cassandra.thrift.InvalidRequestException;
+import org.apache.cassandra.thrift.TimedOutException;
+import org.apache.cassandra.thrift.UnavailableException;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -45,10 +46,11 @@ public class Connection
     
     public String hostName;
     public int port;
-    private Cassandra client;
+    private Cassandra.Client client;
+    private TTransport transport;
     private Compression defaultCompression = Compression.GZIP;
     
-    public Connection(String keyspaceName, String...hosts) throws IOException
+    public Connection(String keyspaceName, String...hosts) throws 
InvalidRequestException, TException
     {
         assert hosts.length > 0;
         
@@ -62,8 +64,12 @@ public class Connection
             break;
         }
         
-        HttpTransceiver tr = new HttpTransceiver(new URL("http", hostName, 
port, ""));
-        client = (Cassandra)SpecificRequestor.getClient(Cassandra.class, tr);
+        TSocket socket = new TSocket(hostName, port);
+        transport = new TFramedTransport(socket);
+        TProtocol protocol = new TBinaryProtocol(transport);
+        client = new Cassandra.Client(protocol);
+        socket.open();
+        
         client.set_keyspace(keyspaceName);
     }
     
@@ -91,13 +97,13 @@ public class Connection
     }
     
     public CqlResult execute(String queryStr)
-    throws InvalidRequestException, UnavailableException, TimedOutException, 
AvroRemoteException
+    throws InvalidRequestException, UnavailableException, TimedOutException, 
TException
     {
         return execute(queryStr, getDefaultCompression());
     }
     
     public CqlResult execute(String queryStr, Compression compression)
-    throws InvalidRequestException, UnavailableException, TimedOutException, 
AvroRemoteException
+    throws InvalidRequestException, UnavailableException, TimedOutException, 
TException
     {
         logger.trace("Executing CQL Query: {}", queryStr);
         return client.execute_cql_query(compressQuery(queryStr, compression), 
compression);
@@ -125,7 +131,7 @@ public class Connection
                 for (CqlRow row : result.rows)
                 {
                     System.out.println("KEY: " + new String(row.key.array()));
-                    for (org.apache.cassandra.avro.Column col : row.columns)
+                    for (org.apache.cassandra.thrift.Column col : row.columns)
                     {
                         System.out.println("  COL: " + new 
String(col.name.array()) + ":" + new String(col.value.array()));
                     }


Reply via email to