Modified: 
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/RowMutation.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/RowMutation.java?rev=809793&r1=809792&r2=809793&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/RowMutation.java 
(original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/RowMutation.java 
Tue Sep  1 01:52:03 2009
@@ -237,10 +237,10 @@
         return new Message(from, StorageService.mutationStage_, 
verbHandlerName, bos.toByteArray());
     }
 
-    public static RowMutation getRowMutation(String table, BatchMutation 
batchMutation)
+    public static RowMutation getRowMutation(String table, String key, 
Map<String, List<ColumnOrSuperColumn>> cfmap)
     {
-        RowMutation rm = new RowMutation(table, batchMutation.key.trim());
-        for (Map.Entry<String, List<ColumnOrSuperColumn>> entry : 
batchMutation.cfmap.entrySet())
+        RowMutation rm = new RowMutation(table, key.trim());
+        for (Map.Entry<String, List<ColumnOrSuperColumn>> entry : 
cfmap.entrySet())
         {
             String cfName = entry.getKey();
             for (ColumnOrSuperColumn cosc : entry.getValue())

Modified: 
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java?rev=809793&r1=809792&r2=809793&view=diff
==============================================================================
--- 
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java
 (original)
+++ 
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java
 Tue Sep  1 01:52:03 2009
@@ -419,31 +419,31 @@
         doInsert(consistency_level, rm);
     }
 
-    public void batch_insert(String table, BatchMutation batch_mutation, int 
consistency_level)
+    public void batch_insert(String keyspace, String key, Map<String, 
List<ColumnOrSuperColumn>> cfmap, int consistency_level)
     throws InvalidRequestException, UnavailableException
     {
         if (logger.isDebugEnabled())
             logger.debug("batch_insert");
 
-        for (String cfName : batch_mutation.cfmap.keySet())
+        for (String cfName : cfmap.keySet())
         {
-            for (ColumnOrSuperColumn cosc : batch_mutation.cfmap.get(cfName))
+            for (ColumnOrSuperColumn cosc : cfmap.get(cfName))
             {
                 if (cosc.column != null)
                 {
-                    ThriftValidation.validateColumnPath(table, new 
ColumnPath(cfName, null, cosc.column.name));
+                    ThriftValidation.validateColumnPath(keyspace, new 
ColumnPath(cfName, null, cosc.column.name));
                 }
                 if (cosc.super_column != null)
                 {
                     for (Column c : cosc.super_column.columns)
                     {
-                        ThriftValidation.validateColumnPath(table, new 
ColumnPath(cfName, cosc.super_column.name, c.name));
+                        ThriftValidation.validateColumnPath(keyspace, new 
ColumnPath(cfName, cosc.super_column.name, c.name));
                     }
                 }
             }
         }
 
-        doInsert(consistency_level, RowMutation.getRowMutation(table, 
batch_mutation));
+        doInsert(consistency_level, RowMutation.getRowMutation(keyspace, key, 
cfmap));
     }
 
     public void remove(String table, String key, ColumnPath column_path, long 
timestamp, int consistency_level)

Modified: incubator/cassandra/trunk/test/system/test_server.py
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/system/test_server.py?rev=809793&r1=809792&r2=809793&view=diff
==============================================================================
--- incubator/cassandra/trunk/test/system/test_server.py (original)
+++ incubator/cassandra/trunk/test/system/test_server.py Tue Sep  1 01:52:03 
2009
@@ -59,7 +59,7 @@
         consistencyLevel = ConsistencyLevel.ZERO
 
     for key in keys:
-        client.batch_insert('Keyspace1', BatchMutation(key=key, cfmap=cfmap), 
consistencyLevel)
+        client.batch_insert('Keyspace1', key, cfmap, consistencyLevel)
 
 def _big_slice(keyspace, key, column_parent):
     p = SlicePredicate(slice_range=SliceRange('', '', False, 1000))
@@ -311,7 +311,7 @@
     def test_batch_insert_super(self):
          cfmap = {'Super1': [ColumnOrSuperColumn(super_column=c) for c in 
_SUPER_COLUMNS],
                   'Super2': [ColumnOrSuperColumn(super_column=c) for c in 
_SUPER_COLUMNS]}
-         client.batch_insert('Keyspace1', BatchMutation(key='key1', 
cfmap=cfmap), ConsistencyLevel.ZERO)
+         client.batch_insert('Keyspace1', 'key1', cfmap, ConsistencyLevel.ZERO)
          time.sleep(0.1)
          _verify_super('Super1')
          _verify_super('Super2')
@@ -319,7 +319,7 @@
     def test_batch_insert_super_blocking(self):
          cfmap = {'Super1': [ColumnOrSuperColumn(super_column=c) for c in 
_SUPER_COLUMNS],
                   'Super2': [ColumnOrSuperColumn(super_column=c) for c in 
_SUPER_COLUMNS]}
-         client.batch_insert('Keyspace1', BatchMutation(key='key1', 
cfmap=cfmap), ConsistencyLevel.ONE)
+         client.batch_insert('Keyspace1', 'key1', cfmap, ConsistencyLevel.ONE)
          _verify_super('Super1')
          _verify_super('Super2')
 


Reply via email to