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

svn commit: r1071467 - in /cassandra/trunk: drivers/py/cql/connection.py drivers/py/cql/errors.py drivers/py/cql/marshal.py drivers/py/test/test_query_preparation.py test/system/test_cql.py

Author: eevans
Date: Thu Feb 17 01:22:09 2011
New Revision: 1071467

URL: http://svn.apache.org/viewvc?rev=1071467&view=rev
Log:
parameter substitution for python CQL driver

Patch by eevans for CASSANDRA-1711

Added:
    cassandra/trunk/drivers/py/cql/marshal.py
    cassandra/trunk/drivers/py/test/test_query_preparation.py
Modified:
    cassandra/trunk/drivers/py/cql/connection.py
    cassandra/trunk/drivers/py/cql/errors.py
    cassandra/trunk/test/system/test_cql.py

Modified: cassandra/trunk/drivers/py/cql/connection.py
URL: http://svn.apache.org/viewvc/cassandra/trunk/drivers/py/cql/connection.py?rev=1071467&r1=1071466&r2=1071467&view=diff
==============================================================================
--- cassandra/trunk/drivers/py/cql/connection.py (original)
+++ cassandra/trunk/drivers/py/cql/connection.py Thu Feb 17 01:22:09 2011
@@ -20,6 +20,7 @@ from thrift.transport import TTransport,
 from thrift.protocol import TBinaryProtocol
 from thrift.Thrift import TApplicationException
 from errors import CQLException, InvalidCompressionScheme
+from marshal import prepare
 import zlib
 
 try:
@@ -71,18 +72,22 @@ class Connection(object):
         if keyspace:
             self.execute('USE %s;' % keyspace)
 
-    def execute(self, query, compression=None):
+    def execute(self, query, *args, **kwargs):
         """
         Execute a CQL query on a remote node.
         
         Params:
         * query .........: CQL query string.
+        * args ..........: Query parameters.
         * compression ...: Query compression type (optional).
         """
-        compress = compression is None and DEFAULT_COMPRESSION \
-                or compression.upper()
+        if kwargs.has_key("compression"):
+            compress = kwargs.get("compression").upper()
+        else:
+            compress = DEFAULT_COMPRESSION
     
-        compressed_query = Connection.compress_query(query, compress)
+        compressed_query = Connection.compress_query(prepare(query, *args),
+                                                     compress)
         request_compression = getattr(Compression, compress)
 
         try:

Modified: cassandra/trunk/drivers/py/cql/errors.py
URL: http://svn.apache.org/viewvc/cassandra/trunk/drivers/py/cql/errors.py?rev=1071467&r1=1071466&r2=1071467&view=diff
==============================================================================
--- cassandra/trunk/drivers/py/cql/errors.py (original)
+++ cassandra/trunk/drivers/py/cql/errors.py Thu Feb 17 01:22:09 2011
@@ -15,7 +15,8 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-__all__ = ['InvalidCompressionScheme', 'CQLException']
+__all__ = ['InvalidCompressionScheme', 'CQLException', 'InvalidQueryFormat']
 
 class InvalidCompressionScheme(Exception): pass
+class InvalidQueryFormat(Exception): pass
 class CQLException(Exception): pass

Added: cassandra/trunk/drivers/py/cql/marshal.py
URL: http://svn.apache.org/viewvc/cassandra/trunk/drivers/py/cql/marshal.py?rev=1071467&view=auto
==============================================================================
--- cassandra/trunk/drivers/py/cql/marshal.py (added)
+++ cassandra/trunk/drivers/py/cql/marshal.py Thu Feb 17 01:22:09 2011
@@ -0,0 +1,62 @@
+
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+from uuid import UUID
+from StringIO import StringIO
+from errors import InvalidQueryFormat
+
+__all__ = ['prepare']
+
+def prepare(query, *args):
+    result = StringIO()
+    index = query.find('?')
+    oldindex = 0
+    count = 0
+    
+    while (index >= 0):
+        result.write(query[oldindex:index])
+        try:
+            result.write(marshal(args[count]))
+        except IndexError:
+            raise InvalidQueryFormat("not enough arguments in substitution")
+        
+        oldindex = index + 1
+        index = query.find('?', index + 1)
+        count += 1
+    result.write(query[oldindex:])
+    
+    if count < len(args):
+        raise InvalidQueryFormat("too many arguments in substitution")
+    
+    return result.getvalue()
+
+def marshal(term):
+    if isinstance(term, long):
+        return "%dL" % term
+    elif isinstance(term, int):
+        return "%d" % term
+    elif isinstance(term, unicode):
+        return "u\"%s\"" % term
+    elif isinstance(term, str):
+        return "\"%s\"" % term
+    elif isinstance(term, UUID):
+        if term.version == 1:
+            return "timeuuid(\"%s\")" % str(term)
+        else:
+            return "uuid(\"%s\")" % str(term)
+    else:
+        return str(term)

Added: cassandra/trunk/drivers/py/test/test_query_preparation.py
URL: http://svn.apache.org/viewvc/cassandra/trunk/drivers/py/test/test_query_preparation.py?rev=1071467&view=auto
==============================================================================
--- cassandra/trunk/drivers/py/test/test_query_preparation.py (added)
+++ cassandra/trunk/drivers/py/test/test_query_preparation.py Thu Feb 17 01:22:09 2011
@@ -0,0 +1,43 @@
+
+import unittest
+from cql.marshal import prepare
+from cql.errors import InvalidQueryFormat
+
+# TESTS[i] ARGUMENTS[i] -> STANDARDS[i]
+TESTS = (
+"""
+SELECT ?,?,?,? FROM ColumnFamily WHERE KEY = ? AND "col" = ?;
+""",
+"""
+USE Keyspace;
+""",
+)
+
+ARGUMENTS = (
+    (1, 3, long(1000), long(3000), "key", unicode("val")),
+    tuple(),
+)
+
+STANDARDS = (
+"""
+SELECT 1,3,1000L,3000L FROM ColumnFamily WHERE KEY = "key" AND "col" = u"val";
+""",
+"""
+USE Keyspace;
+""",
+)
+
+class TestPrepare(unittest.TestCase):
+    def test_prepares(self):
+        "test prepared queries against known standards"
+        for (i, test) in enumerate(TESTS):
+            a = prepare(test, *ARGUMENTS[i])
+            b = STANDARDS[i]
+            assert a == b, "\n%s !=\n%s" % (a, b)
+    
+    def test_bad(self):
+        "ensure bad calls raise exceptions"
+        self.assertRaises(InvalidQueryFormat, prepare, "? ?", 1)
+        self.assertRaises(InvalidQueryFormat, prepare, "? ?", 1, 2, 3)
+        self.assertRaises(InvalidQueryFormat, prepare, "none", 1)
+

Modified: cassandra/trunk/test/system/test_cql.py
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/system/test_cql.py?rev=1071467&r1=1071466&r2=1071467&view=diff
==============================================================================
--- cassandra/trunk/test/system/test_cql.py (original)
+++ cassandra/trunk/test/system/test_cql.py Thu Feb 17 01:22:09 2011
@@ -20,19 +20,12 @@ def uuid1bytes_to_millis(uuidbytes):
     return (uuid.UUID(bytes=uuidbytes).get_time() / 10000) - 12219292800000L
 
 def load_sample(dbconn):
-    dbconn.execute("""
-        UPDATE Standard1 SET "ca1" = "va1", "col" = "val" WHERE KEY = "ka"
-    """)
-    dbconn.execute("""
-        UPDATE Standard1 SET "cb1" = "vb1", "col" = "val" WHERE KEY = "kb"
-    """)
-    dbconn.execute("""
-        UPDATE Standard1 SET "cc1" = "vc1", "col" = "val" WHERE KEY = "kc"
-    """)
-    dbconn.execute("""
-        UPDATE Standard1 SET "cd1" = "vd1", "col" = "val" WHERE KEY = "kd"
-    """)
-    
+    query = "UPDATE Standard1 SET ? = ?, ? = ? WHERE KEY = ?"
+    dbconn.execute(query, "ca1", "va1", "col", "val", "ka")
+    dbconn.execute(query, "cb1", "vb1", "col", "val", "kb")
+    dbconn.execute(query, "cc1", "vc1", "col", "val", "kc")
+    dbconn.execute(query, "cd1", "vd1", "col", "val", "kd")
+
     dbconn.execute("""
         UPDATE Standard2 SET u"%s" = "ve1", "col" = "val" WHERE KEY = "kd"
     """ % u'\xa9'.encode('utf8'))