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 2010/11/17 17:57:28 UTC

svn commit: r1036111 - in /cassandra/trunk: src/java/org/apache/cassandra/cql/ test/system/

Author: eevans
Date: Wed Nov 17 16:57:28 2010
New Revision: 1036111

URL: http://svn.apache.org/viewvc?rev=1036111&view=rev
Log:
conforming single-statement UPDATE impl

Patch by eevans

Removed:
    cassandra/trunk/src/java/org/apache/cassandra/cql/Column.java
    cassandra/trunk/src/java/org/apache/cassandra/cql/Row.java
Modified:
    cassandra/trunk/src/java/org/apache/cassandra/cql/Cql.g
    cassandra/trunk/src/java/org/apache/cassandra/cql/QueryProcessor.java
    cassandra/trunk/src/java/org/apache/cassandra/cql/UpdateStatement.java
    cassandra/trunk/test/system/test_cql.py

Modified: cassandra/trunk/src/java/org/apache/cassandra/cql/Cql.g
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/cql/Cql.g?rev=1036111&r1=1036110&r2=1036111&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/cql/Cql.g (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/cql/Cql.g Wed Nov 17 16:57:28 2010
@@ -6,7 +6,8 @@ options {
 
 @header {
     package org.apache.cassandra.cql;
-    import java.util.ArrayList;
+    import java.util.Map;
+    import java.util.HashMap;
     import org.apache.cassandra.thrift.ConsistencyLevel;
     import org.apache.cassandra.avro.InvalidRequestException;
 }
@@ -93,17 +94,24 @@ selectStatement returns [SelectStatement
  *     <CF>
  * USING
  *     CONSISTENCY.ONE
- * WITH
- *     ROW("key1", COL("col1", "val1"), ...) AND
- *     ROW("key2", COL("col1", "val1"), ...) AND
- *     ROW("key3", COLUMN("col1", "val1"), ...)
+ * SET
+ *     name1 = value1,
+ *     name2 = value2
+ * WHERE
+ *     KEY = keyname;
  */
 updateStatement returns [UpdateStatement expr]
-    : { ConsistencyLevel cLevel = ConsistencyLevel.ONE; }
-      K_UPDATE IDENT
+    : {
+          ConsistencyLevel cLevel = ConsistencyLevel.ONE;
+          Map<Term, Term> columns = new HashMap<Term, Term>();
+      }
+      K_UPDATE columnFamily=IDENT
           (K_USING K_CONSISTENCY '.' K_LEVEL { cLevel = ConsistencyLevel.valueOf($K_LEVEL.text); })?
-          K_WITH first=rowDef { $expr = new UpdateStatement($IDENT.text, first, cLevel); }
-          (K_AND next=rowDef { $expr.and(next); })* endStmnt
+          K_SET c1=term '=' v1=term { columns.put(c1, v1); } (',' cN=term '=' vN=term { columns.put(cN, vN); })*
+          K_WHERE K_KEY '=' key=term endStmnt
+      {
+          return new UpdateStatement($columnFamily.text, cLevel, columns, key);
+      }
     ;
 
 // TODO: date/time, utf8
@@ -139,15 +147,6 @@ selectExpression returns [SelectExpressi
       | start=term '..' finish=term { $expr = new SelectExpression(start, finish, count, reversed); }
       )
     ;
-
-columnDef returns [Column column]
-    : K_COLUMN '(' n=term ',' v=term ')' { $column = new Column($n.item, $v.item); }
-    ;
-
-rowDef returns [Row row]
-    : K_ROW '(' key=term ',' first=columnDef { $row = new Row($key.item, first); }
-          (',' next=columnDef { $row.and(next); })* ')'
-    ;
     
 endStmnt
     : (EOF | ';')
@@ -177,6 +176,7 @@ K_USE:         U S E;
 K_FIRST:       F I R S T;
 K_REVERSED:    R E V E R S E D;
 K_COUNT:       C O U N T;
+K_SET:         S E T;
 
 // Case-insensitive alpha characters
 fragment A: ('a'|'A');

Modified: cassandra/trunk/src/java/org/apache/cassandra/cql/QueryProcessor.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/cql/QueryProcessor.java?rev=1036111&r1=1036110&r2=1036111&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/cql/QueryProcessor.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/cql/QueryProcessor.java Wed Nov 17 16:57:28 2010
@@ -24,9 +24,11 @@ package org.apache.cassandra.cql;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeoutException;
 
@@ -56,6 +58,7 @@ import org.apache.cassandra.dht.IPartiti
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.StorageProxy;
 import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.thrift.ConsistencyLevel;
 import org.apache.cassandra.thrift.IndexClause;
 import org.apache.cassandra.thrift.IndexExpression;
 import org.apache.cassandra.thrift.IndexOperator;
@@ -208,6 +211,43 @@ public class QueryProcessor
         return rows;
     }
     
+    private static void batchUpdate(String keyspace, List<UpdateStatement> updateStatements)
+    throws InvalidRequestException, UnavailableException, TimedOutException
+    {
+        ConsistencyLevel consistency = updateStatements.get(0).getConsistencyLevel();
+        List<RowMutation> rowMutations = new ArrayList<RowMutation>();
+
+        for (UpdateStatement update : updateStatements)
+        {
+            ByteBuffer key = update.getKey().getByteBuffer();
+            validateKey(key);
+            validateColumnFamily(keyspace, update.getColumnFamily());
+            
+            RowMutation rm = new RowMutation(keyspace, key);
+            for (Map.Entry<Term, Term> column : update.getColumns().entrySet())
+            {
+                rm.add(new QueryPath(update.getColumnFamily(), null, column.getKey().getByteBuffer()),
+                       column.getValue().getByteBuffer(),
+                       System.currentTimeMillis());
+            }
+            
+            rowMutations.add(rm);
+        }
+        
+        try
+        {
+            StorageProxy.mutate(rowMutations, consistency);
+        }
+        catch (org.apache.cassandra.thrift.UnavailableException e)
+        {
+            throw new UnavailableException();
+        }
+        catch (TimeoutException e)
+        {
+            throw new TimedOutException();
+        }
+    }
+    
     private static SlicePredicate slicePredicateFromSelect(SelectStatement select)
     {
         SlicePredicate thriftSlicePredicate = new SlicePredicate();
@@ -352,40 +392,8 @@ public class QueryProcessor
                 
             case UPDATE:
                 UpdateStatement update = (UpdateStatement)statement.statement;
-                validateColumnFamily(keyspace, update.getColumnFamily());
-                
+                batchUpdate(keyspace, Collections.singletonList(update));
                 avroResult.type = CqlResultType.VOID;
-                
-                List<RowMutation> rowMutations = new ArrayList<RowMutation>();
-                
-                for (Row row : update.getRows())
-                {
-                    validateKey(row.getKey().getByteBuffer());
-                    RowMutation rm = new RowMutation(keyspace, row.getKey().getByteBuffer());
-                    
-                    for (org.apache.cassandra.cql.Column col : row.getColumns())
-                    {
-                        rm.add(new QueryPath(update.getColumnFamily(), null, col.getName().getByteBuffer()),
-                               col.getValue().getByteBuffer(),
-                               System.currentTimeMillis());
-                    }
-                    
-                    rowMutations.add(rm);
-                }
-                
-                try
-                {
-                    StorageProxy.mutate(rowMutations, update.getConsistencyLevel());
-                }
-                catch (org.apache.cassandra.thrift.UnavailableException e)
-                {
-                    throw new UnavailableException();
-                }
-                catch (TimeoutException e)
-                {
-                    throw new TimedOutException();
-                }
-                    
                 return avroResult;
                 
             case USE:

Modified: cassandra/trunk/src/java/org/apache/cassandra/cql/UpdateStatement.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/cql/UpdateStatement.java?rev=1036111&r1=1036110&r2=1036111&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/cql/UpdateStatement.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/cql/UpdateStatement.java Wed Nov 17 16:57:28 2010
@@ -20,8 +20,8 @@
  */
 package org.apache.cassandra.cql;
 
-import java.util.ArrayList;
-import java.util.List;
+import java.util.Map;
+
 import org.apache.cassandra.thrift.ConsistencyLevel;
 
 /**
@@ -31,37 +31,25 @@ import org.apache.cassandra.thrift.Consi
 public class UpdateStatement
 {
     private String columnFamily;
-    private List<Row> rows = new ArrayList<Row>();
     private ConsistencyLevel cLevel;
+    private Map<Term, Term> columns;
+    private Term key;
     
     /**
-     * Creates a new UpdateStatement from a column family name, a row definition,
-     * and a consistency level.
+     * Creates a new UpdateStatement from a column family name, columns map, consistency
+     * level, and key term.
      * 
      * @param columnFamily column family name
-     * @param first a row definition instance
      * @param cLevel the thrift consistency level
+     * @param columns a map of column name/values pairs
+     * @param key the key name
      */
-    public UpdateStatement(String columnFamily, Row first, ConsistencyLevel cLevel)
+    public UpdateStatement(String columnFamily, ConsistencyLevel cLevel, Map<Term, Term> columns, Term key)
     {
         this.columnFamily = columnFamily;
         this.cLevel = cLevel;
-        and(first);
-    }
-    
-    /**
-     * Adds a new row definition to this <code>UPDATE</code>.
-     * 
-     * @param row the row definition to add.
-     */
-    public void and(Row row)
-    {
-        rows.add(row);
-    }
-
-    public List<Row> getRows()
-    {
-        return rows;
+        this.columns = columns;
+        this.key = key;
     }
 
     public ConsistencyLevel getConsistencyLevel()
@@ -74,9 +62,22 @@ public class UpdateStatement
         return columnFamily;
     }
     
+    public Term getKey()
+    {
+        return key;
+    }
+    
+    public Map<Term, Term> getColumns()
+    {
+        return columns;
+    }
+    
     public String toString()
     {
-        return "UpdateStatement(columnFamily=" + columnFamily + ", " +
-            "row=" + rows + ", " + "consistency=" + cLevel + ")";
+        return String.format("UpdateStatement(columnFamily=%s, key=%s, columns=%s, consistency=%s)",
+                             columnFamily,
+                             key,
+                             columns,
+                             cLevel);
     }
 }

Modified: cassandra/trunk/test/system/test_cql.py
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/system/test_cql.py?rev=1036111&r1=1036110&r2=1036111&view=diff
==============================================================================
--- cassandra/trunk/test/system/test_cql.py (original)
+++ cassandra/trunk/test/system/test_cql.py Wed Nov 17 16:57:28 2010
@@ -10,35 +10,66 @@ from avro_utils import assert_raises
 
 def load_sample(dbconn):
     dbconn.execute("""
-        UPDATE
-            Standard1
-        WITH
-            ROW("ka", COL("ca1", "va1"), COL("col", "val")) AND
-            ROW("kb", COL("cb1", "vb1"), COL("col", "val")) AND
-            ROW("kc", COL("cc1", "vc1"), COL("col", "val")) AND
-            ROW("kd", COL("cd1", "vd1"), COL("col", "val"));
-    """)
-    dbconn.execute("""
-      UPDATE
-          StandardLong1
-      WITH
-        ROW("aa", COL(1L, "1"), COL(2L, "2"), COL(3L, "3"), COL(4L, "4")) AND
-        ROW("ab", COL(5L, "5"), COL(6L, "6"), COL(7L, "8"), COL(9L, "9")) AND
-        ROW("ac", COL(9L, "9"), COL(8L, "8"), COL(7L, "7"), COL(6L, "6")) AND
-        ROW("ad", COL(5L, "5"), COL(4L, "4"), COL(3L, "3"), COL(2L, "2")) AND
-        ROW("ae", COL(1L, "1"), COL(2L, "2"), COL(3L, "3"), COL(4L, "4")) AND
-        ROW("af", COL(1L, "1"), COL(2L, "2"), COL(3L, "3"), COL(4L, "4")) AND
-        ROW("ag", COL(5L, "5"), COL(6L, "6"), COL(7L, "8"), COL(9L, "9"));
-    """)
-    dbconn.execute("""
-        UPDATE
-            Indexed1
-        WITH
-            ROW("asmith",   COL("birthdate", 100L), COL("unindexed", 250L)) AND
-            ROW("dozer",    COL("birthdate", 100L), COL("unindexed", 200L)) AND
-            ROW("morpheus", COL("birthdate", 175L), COL("unindexed", 200L)) AND
-            ROW("neo",      COL("birthdate", 150L), COL("unindexed", 250L)) AND
-            ROW("trinity",  COL("birthdate", 125L), COL("unindexed", 200L));
+        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"
+    """)
+
+    dbconn.execute("""
+        UPDATE StandardLong1 SET 1L = "1", 2L = "2", 3L = "3", 4L = "4"
+                WHERE KEY = "aa";
+    """)
+    dbconn.execute("""
+        UPDATE StandardLong1 SET 5L = "5", 6L = "6", 7L = "8", 9L = "9"
+                WHERE KEY = "ab";
+    """)
+    dbconn.execute("""
+        UPDATE StandardLong1 SET 9L = "9", 8L = "8", 7L = "7", 6L = "6"
+                WHERE KEY = "ac";
+    """)
+    dbconn.execute("""
+        UPDATE StandardLong1 SET 5L = "5", 4L = "4", 3L = "3", 2L = "2"
+                WHERE KEY = "ad";
+    """)
+    dbconn.execute("""
+        UPDATE StandardLong1 SET 1L = "1", 2L = "2", 3L = "3", 4L = "4"
+                WHERE KEY = "ae";
+    """)
+    dbconn.execute("""
+        UPDATE StandardLong1 SET 1L = "1", 2L = "2", 3L = "3", 4L = "4"
+                WHERE KEY = "af";
+    """)
+    dbconn.execute("""
+        UPDATE StandardLong1 SET 5L = "5", 6L = "6", 7L = "8", 9L = "9"
+                WHERE KEY = "ag";
+    """)
+
+    dbconn.execute("""
+        UPDATE Indexed1 SET "birthdate" = 100L, "unindexed" = 250L
+                WHERE KEY = "asmith";
+    """)
+    dbconn.execute("""
+        UPDATE Indexed1 SET "birthdate" = 100L, "unindexed" = 200L
+                WHERE KEY = "dozer";
+    """)
+    dbconn.execute("""
+        UPDATE Indexed1 SET "birthdate" = 175L, "unindexed" = 200L
+                WHERE KEY = "morpheus";
+    """)
+    dbconn.execute("""
+        UPDATE Indexed1 SET "birthdate" = 150L, "unindexed" = 250L
+                WHERE KEY = "neo";
+    """)
+    dbconn.execute("""
+        UPDATE Indexed1 SET "birthdate" = 125L, "unindexed" = 200L
+                WHERE KEY = "trinity";
     """)
 
 def init(keyspace="Keyspace1"):