You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2013/12/21 22:40:51 UTC

[2/4] Rename RowMutation->Mutation in preparation for Row->Partition

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/long/org/apache/cassandra/db/LongKeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/LongKeyspaceTest.java b/test/long/org/apache/cassandra/db/LongKeyspaceTest.java
index 0dd872c..d610efa 100644
--- a/test/long/org/apache/cassandra/db/LongKeyspaceTest.java
+++ b/test/long/org/apache/cassandra/db/LongKeyspaceTest.java
@@ -21,12 +21,10 @@ package org.apache.cassandra.db;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.utils.WrappedRunnable;
 import static org.apache.cassandra.Util.column;
 
 import org.apache.cassandra.Util;
-import org.apache.cassandra.utils.ByteBufferUtil;
 
 
 public class LongKeyspaceTest extends SchemaLoader
@@ -39,7 +37,7 @@ public class LongKeyspaceTest extends SchemaLoader
 
         for (int i = 1; i < 5000; i += 100)
         {
-            RowMutation rm = new RowMutation("Keyspace1", Util.dk("key" + i).key);
+            Mutation rm = new Mutation("Keyspace1", Util.dk("key" + i).key);
             ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
             for (int j = 0; j < i; j++)
                 cf.addColumn(column("c" + j, "v" + j, 1L));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/long/org/apache/cassandra/db/MeteredFlusherTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/MeteredFlusherTest.java b/test/long/org/apache/cassandra/db/MeteredFlusherTest.java
index 07ddbd6..4bab277 100644
--- a/test/long/org/apache/cassandra/db/MeteredFlusherTest.java
+++ b/test/long/org/apache/cassandra/db/MeteredFlusherTest.java
@@ -50,7 +50,7 @@ public class MeteredFlusherTest extends SchemaLoader
         {
             for (int i = 0; i < 100; i++)
             {
-                RowMutation rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key" + j));
+                Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key" + j));
                 ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "_CF" + i);
                 // don't cheat by allocating this outside of the loop; that defeats the purpose of deliberately using lots of memory
                 ByteBuffer value = ByteBuffer.allocate(100000);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/long/org/apache/cassandra/db/commitlog/ComitLogStress.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/commitlog/ComitLogStress.java b/test/long/org/apache/cassandra/db/commitlog/ComitLogStress.java
index b212a22..d1db2f2 100644
--- a/test/long/org/apache/cassandra/db/commitlog/ComitLogStress.java
+++ b/test/long/org/apache/cassandra/db/commitlog/ComitLogStress.java
@@ -11,7 +11,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
-import org.apache.cassandra.db.RowMutation;
+import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.UUIDGen;
 
@@ -64,7 +64,7 @@ public class ComitLogStress
         public void run() {
             String ks = "Keyspace1";
             ByteBuffer key = ByteBufferUtil.bytes(keyString);
-            RowMutation mutation = new RowMutation(ks, key);
+            Mutation mutation = new Mutation(ks, key);
             mutation.add("Standard1", Util.cellname("name"), ByteBufferUtil.bytes("value"),
                     System.currentTimeMillis());
             CommitLog.instance.add(mutation);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
index 5d68a3b..ce23a47 100644
--- a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
+++ b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
@@ -127,7 +127,7 @@ public class LongCompactionsTest extends SchemaLoader
         for (int j = 0; j < SSTABLES; j++) {
             for (int i = 0; i < ROWS_PER_SSTABLE; i++) {
                 DecoratedKey key = Util.dk(String.valueOf(i % 2));
-                RowMutation rm = new RowMutation(KEYSPACE1, key.key);
+                Mutation rm = new Mutation(KEYSPACE1, key.key);
                 long timestamp = j * ROWS_PER_SSTABLE + i;
                 rm.add("Standard1", Util.cellname(String.valueOf(i / 2)),
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
index 0de0395..54fc22f 100644
--- a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
+++ b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
@@ -27,12 +27,8 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.RowMutation;
-import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.*;
 import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
 public class LongLeveledCompactionStrategyTest extends SchemaLoader
@@ -58,7 +54,7 @@ public class LongLeveledCompactionStrategyTest extends SchemaLoader
         for (int r = 0; r < rows; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            RowMutation rm = new RowMutation(ksname, key.key);
+            Mutation rm = new Mutation(ksname, key.key);
             for (int c = 0; c < columns; c++)
             {
                 rm.add(cfname, Util.cellname("column" + c), value, 0);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/SchemaLoader.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/SchemaLoader.java b/test/unit/org/apache/cassandra/SchemaLoader.java
index df74108..6866402 100644
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra;
 
 import java.io.File;
-import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.*;
 
@@ -399,9 +398,9 @@ public class SchemaLoader
         for (int i = offset; i < offset + numberOfRows; i++)
         {
             ByteBuffer key = ByteBufferUtil.bytes("key" + i);
-            RowMutation rowMutation = new RowMutation(keyspace, key);
-            rowMutation.add(columnFamily, Util.cellname("col" + i), ByteBufferUtil.bytes("val" + i), System.currentTimeMillis());
-            rowMutation.applyUnsafe();
+            Mutation mutation = new Mutation(keyspace, key);
+            mutation.add(columnFamily, Util.cellname("col" + i), ByteBufferUtil.bytes("val" + i), System.currentTimeMillis());
+            mutation.applyUnsafe();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java
index f6e5b51..ec8f9d5 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -141,7 +141,7 @@ public class Util
         return new Bounds<RowPosition>(rp(left), rp(right));
     }
 
-    public static void addMutation(RowMutation rm, String columnFamilyName, String superColumnName, long columnName, String value, long timestamp)
+    public static void addMutation(Mutation rm, String columnFamilyName, String superColumnName, long columnName, String value, long timestamp)
     {
         CellName cname = superColumnName == null
                        ? CellNames.simpleDense(getBytes(columnName))
@@ -185,7 +185,7 @@ public class Util
     /**
      * Writes out a bunch of mutations for a single column family.
      *
-     * @param mutations A group of RowMutations for the same keyspace and column family.
+     * @param mutations A group of Mutations for the same keyspace and column family.
      * @return The ColumnFamilyStore that was used.
      */
     public static ColumnFamilyStore writeColumnFamily(List<IMutation> mutations) throws IOException, ExecutionException, InterruptedException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/CFMetaDataTest.java b/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
index fd8b637..0d2fa95 100644
--- a/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
+++ b/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.config;
 
 import java.util.ArrayList;
-import java.util.Iterator;
 import java.util.List;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -127,7 +126,7 @@ public class CFMetaDataTest extends SchemaLoader
         assert before.equals(after) : String.format("\n%s\n!=\n%s", before, after);
 
         // Test schema conversion
-        RowMutation rm = cfm.toSchema(System.currentTimeMillis());
+        Mutation rm = cfm.toSchema(System.currentTimeMillis());
         ColumnFamily serializedCf = rm.getColumnFamily(Schema.instance.getId(Keyspace.SYSTEM_KS, SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF));
         ColumnFamily serializedCD = rm.getColumnFamily(Schema.instance.getId(Keyspace.SYSTEM_KS, SystemKeyspace.SCHEMA_COLUMNS_CF));
         UntypedResultSet.Row result = QueryProcessor.resultify("SELECT * FROM system.schema_columnfamilies", new Row(k, serializedCf)).one();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/config/DefsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/DefsTest.java b/test/unit/org/apache/cassandra/config/DefsTest.java
index edb2b8d..28d3589 100644
--- a/test/unit/org/apache/cassandra/config/DefsTest.java
+++ b/test/unit/org/apache/cassandra/config/DefsTest.java
@@ -171,7 +171,7 @@ public class DefsTest extends SchemaLoader
         // now read and write to it.
         CellName col0 = cellname("col0");
         DecoratedKey dk = Util.dk("key0");
-        RowMutation rm = new RowMutation(ks, dk.key);
+        Mutation rm = new Mutation(ks, dk.key);
         rm.add(cf, col0, ByteBufferUtil.bytes("value0"), 1L);
         rm.apply();
         ColumnFamilyStore store = Keyspace.open(ks).getColumnFamilyStore(cf);
@@ -195,7 +195,7 @@ public class DefsTest extends SchemaLoader
         assert cfm != null;
 
         // write some data, force a flush, then verify that files exist on disk.
-        RowMutation rm = new RowMutation(ks.name, dk.key);
+        Mutation rm = new Mutation(ks.name, dk.key);
         for (int i = 0; i < 100; i++)
             rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
         rm.apply();
@@ -209,7 +209,7 @@ public class DefsTest extends SchemaLoader
         assert !Schema.instance.getKSMetaData(ks.name).cfMetaData().containsKey(cfm.cfName);
 
         // any write should fail.
-        rm = new RowMutation(ks.name, dk.key);
+        rm = new Mutation(ks.name, dk.key);
         boolean success = true;
         try
         {
@@ -245,7 +245,7 @@ public class DefsTest extends SchemaLoader
 
         // test reads and writes.
         CellName col0 = cellname("col0");
-        RowMutation rm = new RowMutation(newCf.ksName, dk.key);
+        Mutation rm = new Mutation(newCf.ksName, dk.key);
         rm.add(newCf.cfName, col0, ByteBufferUtil.bytes("value0"), 1L);
         rm.apply();
         ColumnFamilyStore store = Keyspace.open(newCf.ksName).getColumnFamilyStore(newCf.cfName);
@@ -269,7 +269,7 @@ public class DefsTest extends SchemaLoader
         assert cfm != null;
 
         // write some data, force a flush, then verify that files exist on disk.
-        RowMutation rm = new RowMutation(ks.name, dk.key);
+        Mutation rm = new Mutation(ks.name, dk.key);
         for (int i = 0; i < 100; i++)
             rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
         rm.apply();
@@ -283,7 +283,7 @@ public class DefsTest extends SchemaLoader
         assert Schema.instance.getKSMetaData(ks.name) == null;
 
         // write should fail.
-        rm = new RowMutation(ks.name, dk.key);
+        rm = new Mutation(ks.name, dk.key);
         boolean success = true;
         try
         {
@@ -320,7 +320,7 @@ public class DefsTest extends SchemaLoader
         assert cfm != null;
 
         // write some data
-        RowMutation rm = new RowMutation(ks.name, dk.key);
+        Mutation rm = new Mutation(ks.name, dk.key);
         for (int i = 0; i < 100; i++)
             rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
         rm.apply();
@@ -354,7 +354,7 @@ public class DefsTest extends SchemaLoader
         // now read and write to it.
         CellName col0 = cellname("col0");
         DecoratedKey dk = Util.dk("key0");
-        RowMutation rm = new RowMutation(newKs.name, dk.key);
+        Mutation rm = new Mutation(newKs.name, dk.key);
         rm.add(newCf.cfName, col0, ByteBufferUtil.bytes("value0"), 1L);
         rm.apply();
         ColumnFamilyStore store = Keyspace.open(newKs.name).getColumnFamilyStore(newCf.cfName);
@@ -503,7 +503,7 @@ public class DefsTest extends SchemaLoader
         ColumnFamilyStore cfs = Keyspace.open("Keyspace6").getColumnFamilyStore("Indexed1");
 
         // insert some data.  save the sstable descriptor so we can make sure it's marked for delete after the drop
-        RowMutation rm = new RowMutation("Keyspace6", ByteBufferUtil.bytes("k1"));
+        Mutation rm = new Mutation("Keyspace6", ByteBufferUtil.bytes("k1"));
         rm.add("Indexed1", cellname("notbirthdate"), ByteBufferUtil.bytes(1L), 0);
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 0);
         rm.apply();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/CleanupTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CleanupTest.java b/test/unit/org/apache/cassandra/db/CleanupTest.java
index 29a538e..a068f1b 100644
--- a/test/unit/org/apache/cassandra/db/CleanupTest.java
+++ b/test/unit/org/apache/cassandra/db/CleanupTest.java
@@ -147,8 +147,8 @@ public class CleanupTest extends SchemaLoader
         {
             String key = String.valueOf(i);
             // create a row and update the birthdate value, test that the index query fetches the new version
-            RowMutation rm;
-            rm = new RowMutation(KEYSPACE1, ByteBufferUtil.bytes(key));
+            Mutation rm;
+            rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes(key));
             rm.add(cfs.name, Util.cellname(COLUMN), VALUE, System.currentTimeMillis());
             rm.applyUnsafe();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/CollationControllerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CollationControllerTest.java b/test/unit/org/apache/cassandra/db/CollationControllerTest.java
index 5b7197b..14325a4 100644
--- a/test/unit/org/apache/cassandra/db/CollationControllerTest.java
+++ b/test/unit/org/apache/cassandra/db/CollationControllerTest.java
@@ -21,7 +21,6 @@ package org.apache.cassandra.db;
 import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.concurrent.ExecutionException;
 
 import org.apache.cassandra.SchemaLoader;
@@ -32,8 +31,6 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.junit.Test;
 
-import org.apache.cassandra.io.sstable.SSTableReader;
-
 public class CollationControllerTest extends SchemaLoader
 {
     @Test
@@ -42,30 +39,30 @@ public class CollationControllerTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open("Keyspace1");
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
-        RowMutation rm;
+        Mutation rm;
         DecoratedKey dk = Util.dk("key1");
         
         // add data
-        rm = new RowMutation(keyspace.getName(), dk.key);
+        rm = new Mutation(keyspace.getName(), dk.key);
         rm.add(cfs.name, Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
         cfs.forceBlockingFlush();
         
         // remove
-        rm = new RowMutation(keyspace.getName(), dk.key);
+        rm = new Mutation(keyspace.getName(), dk.key);
         rm.delete(cfs.name, 10);
         rm.apply();
         
         // add another mutation because sstable maxtimestamp isn't set
         // correctly during flush if the most recent mutation is a row delete
-        rm = new RowMutation(keyspace.getName(), Util.dk("key2").key);
+        rm = new Mutation(keyspace.getName(), Util.dk("key2").key);
         rm.add(cfs.name, Util.cellname("Column1"), ByteBufferUtil.bytes("zxcv"), 20);
         rm.apply();
         
         cfs.forceBlockingFlush();
 
         // add yet one more mutation
-        rm = new RowMutation(keyspace.getName(), dk.key);
+        rm = new Mutation(keyspace.getName(), dk.key);
         rm.add(cfs.name, Util.cellname("Column1"), ByteBufferUtil.bytes("foobar"), 30);
         rm.apply();
         cfs.forceBlockingFlush();
@@ -94,18 +91,18 @@ public class CollationControllerTest extends SchemaLoader
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("StandardGCGS0");
         cfs.disableAutoCompaction();
 
-        RowMutation rm;
+        Mutation rm;
         DecoratedKey dk = Util.dk("key1");
         CellName cellName = Util.cellname("Column1");
 
         // add data
-        rm = new RowMutation(keyspace.getName(), dk.key);
+        rm = new Mutation(keyspace.getName(), dk.key);
         rm.add(cfs.name, cellName, ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
         cfs.forceBlockingFlush();
 
         // remove
-        rm = new RowMutation(keyspace.getName(), dk.key);
+        rm = new Mutation(keyspace.getName(), dk.key);
         rm.delete(cfs.name, cellName, 0);
         rm.apply();
         cfs.forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
index d1a2af7..0a081c1 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@ -82,13 +82,13 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
         cfs.truncateBlocking();
 
-        RowMutation rm;
-        rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
+        Mutation rm;
+        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key1"));
         rm.add("Standard1", cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
         cfs.forceBlockingFlush();
 
-        rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
+        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key1"));
         rm.add("Standard1", cellname("Column1"), ByteBufferUtil.bytes("asdf"), 1);
         rm.apply();
         cfs.forceBlockingFlush();
@@ -106,8 +106,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         cfs.truncateBlocking();
 
         List<IMutation> rms = new LinkedList<IMutation>();
-        RowMutation rm;
-        rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
+        Mutation rm;
+        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key1"));
         rm.add("Standard1", cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.add("Standard1", cellname("Column2"), ByteBufferUtil.bytes("asdf"), 0);
         rms.add(rm);
@@ -125,9 +125,9 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open("Keyspace1");
         final ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
-        RowMutation rm;
+        Mutation rm;
 
-        rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
+        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key1"));
         rm.delete("Standard2", System.currentTimeMillis());
         rm.apply();
 
@@ -168,26 +168,26 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     public void testIndexScan()
     {
         ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1");
-        RowMutation rm;
+        Mutation rm;
         CellName nobirthdate = cellname("notbirthdate");
         CellName birthdate = cellname("birthdate");
 
-        rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("k1"));
+        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("k1"));
         rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(1L), 0);
         rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
-        rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("k2"));
+        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("k2"));
         rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
         rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(2L), 0);
         rm.apply();
 
-        rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("k3"));
+        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("k3"));
         rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
         rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
-        rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("k4aaaa"));
+        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("k4aaaa"));
         rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
         rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(3L), 0);
         rm.apply();
@@ -251,11 +251,11 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testLargeScan()
     {
-        RowMutation rm;
+        Mutation rm;
         ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1");
         for (int i = 0; i < 100; i++)
         {
-            rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key" + i));
+            rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key" + i));
             rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(34L), 0);
             rm.add("Indexed1", cellname("notbirthdate"), ByteBufferUtil.bytes((long) (i % 2)), 0);
             rm.applyUnsafe();
@@ -281,9 +281,9 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     public void testIndexDeletions() throws IOException
     {
         ColumnFamilyStore cfs = Keyspace.open("Keyspace3").getColumnFamilyStore("Indexed1");
-        RowMutation rm;
+        Mutation rm;
 
-        rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
+        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
@@ -297,7 +297,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assert "k1".equals( key );
 
         // delete the column directly
-        rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
+        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
         rm.delete("Indexed1", cellname("birthdate"), 1);
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
@@ -312,7 +312,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assert rows.isEmpty();
 
         // resurrect w/ a newer timestamp
-        rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
+        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 2);
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
@@ -321,7 +321,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assert "k1".equals( key );
 
         // verify that row and delete w/ older timestamp does nothing
-        rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
+        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
         rm.delete("Indexed1", 1);
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
@@ -330,7 +330,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assert "k1".equals( key );
 
         // similarly, column delete w/ older timestamp should do nothing
-        rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
+        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
         rm.delete("Indexed1", cellname("birthdate"), 1);
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
@@ -339,21 +339,21 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assert "k1".equals( key );
 
         // delete the entire row (w/ newer timestamp this time)
-        rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
+        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
         rm.delete("Indexed1", 3);
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.isEmpty() : StringUtils.join(rows, ",");
 
         // make sure obsolete mutations don't generate an index entry
-        rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
+        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 3);
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.isEmpty() : StringUtils.join(rows, ",");
 
         // try insert followed by row delete in the same mutation
-        rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
+        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 1);
         rm.delete("Indexed1", 2);
         rm.apply();
@@ -361,7 +361,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assert rows.isEmpty() : StringUtils.join(rows, ",");
 
         // try row delete followed by insert in the same mutation
-        rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
+        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
         rm.delete("Indexed1", 3);
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 4);
         rm.apply();
@@ -379,11 +379,11 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         CellName birthdate = cellname("birthdate");
 
         // create a row and update the birthdate value, test that the index query fetches the new version
-        RowMutation rm;
-        rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("k1"));
+        Mutation rm;
+        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("k1"));
         rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 1);
         rm.apply();
-        rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("k1"));
+        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("k1"));
         rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(2L), 2);
         rm.apply();
 
@@ -401,7 +401,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assert "k1".equals( key );
 
         // update the birthdate value with an OLDER timestamp, and test that the index ignores this
-        rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("k1"));
+        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("k1"));
         rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(3L), 0);
         rm.apply();
 
@@ -427,8 +427,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         ByteBuffer val2 = ByteBufferUtil.bytes(2L);
 
         // create a row and update the "birthdate" value, test that the index query fetches this version
-        RowMutation rm;
-        rm = new RowMutation(keySpace, rowKey);
+        Mutation rm;
+        rm = new Mutation(keySpace, rowKey);
         rm.add(cfName, colName, val1, 0);
         rm.apply();
         IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, val1);
@@ -442,7 +442,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         keyspace.getColumnFamilyStore(cfName).forceBlockingFlush();
 
         // now apply another update, but force the index update to be skipped
-        rm = new RowMutation(keySpace, rowKey);
+        rm = new Mutation(keySpace, rowKey);
         rm.add(cfName, colName, val2, 1);
         keyspace.apply(rm, true, false);
 
@@ -462,7 +462,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // now, reset back to the original value, still skipping the index update, to
         // make sure the value was expunged from the index when it was discovered to be inconsistent
-        rm = new RowMutation(keySpace, rowKey);
+        rm = new Mutation(keySpace, rowKey);
         rm.add(cfName, colName, ByteBufferUtil.bytes(1L), 3);
         keyspace.apply(rm, true, false);
 
@@ -495,8 +495,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         ByteBuffer val2 = ByteBufferUtil.bytes("v2");
 
         // create a row and update the author value
-        RowMutation rm;
-        rm = new RowMutation(keySpace, rowKey);
+        Mutation rm;
+        rm = new Mutation(keySpace, rowKey);
         rm.add(cfName, compositeName, val1, 0);
         rm.apply();
 
@@ -514,7 +514,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assertEquals(1, rows.size());
 
         // now apply another update, but force the index update to be skipped
-        rm = new RowMutation(keySpace, rowKey);
+        rm = new Mutation(keySpace, rowKey);
         rm.add(cfName, compositeName, val2, 1);
         keyspace.apply(rm, true, false);
 
@@ -534,7 +534,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // now, reset back to the original value, still skipping the index update, to
         // make sure the value was expunged from the index when it was discovered to be inconsistent
-        rm = new RowMutation(keySpace, rowKey);
+        rm = new Mutation(keySpace, rowKey);
         rm.add(cfName, compositeName, val1, 2);
         keyspace.apply(rm, true, false);
 
@@ -567,13 +567,13 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         ByteBuffer val1 = ByteBufferUtil.bytes("v2");
 
         // Insert indexed value.
-        RowMutation rm;
-        rm = new RowMutation(keySpace, rowKey);
+        Mutation rm;
+        rm = new Mutation(keySpace, rowKey);
         rm.add(cfName, compositeName, val1, 0);
         rm.apply();
 
         // Now delete the value and flush too.
-        rm = new RowMutation(keySpace, rowKey);
+        rm = new Mutation(keySpace, rowKey);
         rm.delete(cfName, 1);
         rm.apply();
 
@@ -597,27 +597,27 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     public void testIndexScanWithLimitOne()
     {
         ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1");
-        RowMutation rm;
+        Mutation rm;
 
         CellName nobirthdate = cellname("notbirthdate");
         CellName birthdate = cellname("birthdate");
 
-        rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("kk1"));
+        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("kk1"));
         rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(1L), 0);
         rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
-        rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("kk2"));
+        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("kk2"));
         rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
         rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
-        rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("kk3"));
+        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("kk3"));
         rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
         rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
-        rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("kk4"));
+        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("kk4"));
         rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
         rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
         rm.apply();
@@ -641,8 +641,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Indexed2");
 
         // create a row and update the birthdate value, test that the index query fetches the new version
-        RowMutation rm;
-        rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("k1"));
+        Mutation rm;
+        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("k1"));
         rm.add("Indexed2", cellname("birthdate"), ByteBufferUtil.bytes(1L), 1);
         rm.apply();
 
@@ -722,7 +722,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assertRowAndColCount(1, 6, false, cfs.getRangeSlice(Util.range("f", "g"), null, ThriftValidation.asIFilter(sp, cfs.metadata, scfName), 100));
 
         // delete
-        RowMutation rm = new RowMutation(keyspace.getName(), key.key);
+        Mutation rm = new Mutation(keyspace.getName(), key.key);
         rm.deleteRange(cfName, SuperColumns.startOf(scfName), SuperColumns.endOf(scfName), 2);
         rm.apply();
 
@@ -776,7 +776,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(cfs.keyspace.getName(), cfs.name);
         for (Cell col : cols)
             cf.addColumn(col.withUpdatedName(CellNames.compositeDense(scfName, col.name().toByteBuffer())));
-        RowMutation rm = new RowMutation(cfs.keyspace.getName(), key.key, cf);
+        Mutation rm = new Mutation(cfs.keyspace.getName(), key.key, cf);
         rm.apply();
     }
 
@@ -785,7 +785,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(cfs.keyspace.getName(), cfs.name);
         for (Cell col : cols)
             cf.addColumn(col);
-        RowMutation rm = new RowMutation(cfs.keyspace.getName(), key.key, cf);
+        Mutation rm = new Mutation(cfs.keyspace.getName(), key.key, cf);
         rm.apply();
     }
 
@@ -817,7 +817,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assertRowAndColCount(1, 4, false, cfs.getRangeSlice(Util.range("f", "g"), null, ThriftValidation.asIFilter(sp, cfs.metadata, null), 100));
 
         // delete (from sstable and memtable)
-        RowMutation rm = new RowMutation(keyspace.getName(), key.key);
+        Mutation rm = new Mutation(keyspace.getName(), key.key);
         rm.delete(cfs.name, 2);
         rm.apply();
 
@@ -850,13 +850,13 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     {
         ColumnFamilyStore cfs = Keyspace.open("Keyspace2").getColumnFamilyStore("Standard1");
         List<IMutation> rms = new LinkedList<IMutation>();
-        RowMutation rm;
-        rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("key1"));
+        Mutation rm;
+        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("key1"));
         rm.add("Standard1", cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rms.add(rm);
         Util.writeColumnFamily(rms);
 
-        rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("key2"));
+        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("key2"));
         rm.add("Standard1", cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rms.add(rm);
         return Util.writeColumnFamily(rms);
@@ -1270,7 +1270,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         for (int i = 0; i < 10; i++)
         {
             ByteBuffer key = ByteBufferUtil.bytes(String.valueOf("k" + i));
-            RowMutation rm = new RowMutation("Keyspace1", key);
+            Mutation rm = new Mutation("Keyspace1", key);
             rm.add("Indexed1", cellname("birthdate"), LongType.instance.decompose(1L), System.currentTimeMillis());
             rm.apply();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/CommitLogTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CommitLogTest.java b/test/unit/org/apache/cassandra/db/CommitLogTest.java
index 587e731..b51c17e 100644
--- a/test/unit/org/apache/cassandra/db/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/CommitLogTest.java
@@ -99,7 +99,7 @@ public class CommitLogTest extends SchemaLoader
     {
         CommitLog.instance.resetUnsafe();
         // Roughly 32 MB mutation
-        RowMutation rm = new RowMutation("Keyspace1", bytes("k"));
+        Mutation rm = new Mutation("Keyspace1", bytes("k"));
         rm.add("Standard1", Util.cellname("c1"), ByteBuffer.allocate(DatabaseDescriptor.getCommitLogSegmentSize()/4), 0);
 
         // Adding it 5 times
@@ -110,7 +110,7 @@ public class CommitLogTest extends SchemaLoader
         CommitLog.instance.add(rm);
 
         // Adding new mutation on another CF
-        RowMutation rm2 = new RowMutation("Keyspace1", bytes("k"));
+        Mutation rm2 = new Mutation("Keyspace1", bytes("k"));
         rm2.add("Standard2", Util.cellname("c1"), ByteBuffer.allocate(4), 0);
         CommitLog.instance.add(rm2);
 
@@ -129,7 +129,7 @@ public class CommitLogTest extends SchemaLoader
         DatabaseDescriptor.getCommitLogSegmentSize();
         CommitLog.instance.resetUnsafe();
         // Roughly 32 MB mutation
-        RowMutation rm = new RowMutation("Keyspace1", bytes("k"));
+        Mutation rm = new Mutation("Keyspace1", bytes("k"));
         rm.add("Standard1", Util.cellname("c1"), ByteBuffer.allocate((DatabaseDescriptor.getCommitLogSegmentSize()/4) - 1), 0);
 
         // Adding it twice (won't change segment)
@@ -146,7 +146,7 @@ public class CommitLogTest extends SchemaLoader
         assert CommitLog.instance.activeSegments() == 1 : "Expecting 1 segment, got " + CommitLog.instance.activeSegments();
 
         // Adding new mutation on another CF, large enough (including CL entry overhead) that a new segment is created
-        RowMutation rm2 = new RowMutation("Keyspace1", bytes("k"));
+        Mutation rm2 = new Mutation("Keyspace1", bytes("k"));
         rm2.add("Standard2", Util.cellname("c1"), ByteBuffer.allocate((DatabaseDescriptor.getCommitLogSegmentSize()/2) - 100), 0);
         CommitLog.instance.add(rm2);
         // also forces a new segment, since each entry-with-overhead is just under half the CL size
@@ -172,7 +172,7 @@ public class CommitLogTest extends SchemaLoader
     {
         CommitLog.instance.resetUnsafe();
 
-        RowMutation rm = new RowMutation("Keyspace1", bytes("k"));
+        Mutation rm = new Mutation("Keyspace1", bytes("k"));
         rm.add("Standard1", Util.cellname("c1"), ByteBuffer.allocate((DatabaseDescriptor.getCommitLogSegmentSize()) - 83), 0);
         CommitLog.instance.add(rm);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/CounterMutationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CounterMutationTest.java b/test/unit/org/apache/cassandra/db/CounterMutationTest.java
index 5cc97e4..dcfb508 100644
--- a/test/unit/org/apache/cassandra/db/CounterMutationTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterMutationTest.java
@@ -36,12 +36,12 @@ public class CounterMutationTest extends SchemaLoader
     @Test
     public void testMergeOldShards() throws IOException
     {
-        RowMutation rm;
+        Mutation rm;
         CounterMutation cm;
 
         CounterId id1 = CounterId.getLocalId();
 
-        rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
+        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key1"));
         rm.addCounter("Counter1", Util.cellname("Column1"), 3);
         cm = new CounterMutation(rm, ConsistencyLevel.ONE);
         cm.apply();
@@ -49,7 +49,7 @@ public class CounterMutationTest extends SchemaLoader
         CounterId.renewLocalId(2L); // faking time of renewal for test
         CounterId id2 = CounterId.getLocalId();
 
-        rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
+        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key1"));
         rm.addCounter("Counter1", Util.cellname("Column1"), 4);
         cm = new CounterMutation(rm, ConsistencyLevel.ONE);
         cm.apply();
@@ -57,7 +57,7 @@ public class CounterMutationTest extends SchemaLoader
         CounterId.renewLocalId(4L); // faking time of renewal for test
         CounterId id3 = CounterId.getLocalId();
 
-        rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
+        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key1"));
         rm.addCounter("Counter1", Util.cellname("Column1"), 5);
         rm.addCounter("Counter1", Util.cellname("Column2"), 1);
         cm = new CounterMutation(rm, ConsistencyLevel.ONE);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/HintedHandOffTest.java b/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
index 4978c57..10a7289 100644
--- a/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
+++ b/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
@@ -63,7 +63,7 @@ public class HintedHandOffTest extends SchemaLoader
         hintStore.disableAutoCompaction();
 
         // insert 1 hint
-        RowMutation rm = new RowMutation(KEYSPACE4, ByteBufferUtil.bytes(1));
+        Mutation rm = new Mutation(KEYSPACE4, ByteBufferUtil.bytes(1));
         rm.add(STANDARD1_CF, Util.cellname(COLUMN1), ByteBufferUtil.EMPTY_BYTE_BUFFER, System.currentTimeMillis());
 
         HintedHandOffManager.instance.hintFor(rm, HintedHandOffManager.calculateHintTTL(rm), UUID.randomUUID()).apply();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/KeyCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/KeyCacheTest.java b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
index dc5b22c..6ca5487 100644
--- a/test/unit/org/apache/cassandra/db/KeyCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.ExecutionException;
@@ -114,13 +113,13 @@ public class KeyCacheTest extends SchemaLoader
 
         DecoratedKey key1 = Util.dk("key1");
         DecoratedKey key2 = Util.dk("key2");
-        RowMutation rm;
+        Mutation rm;
 
         // inserts
-        rm = new RowMutation(KEYSPACE1, key1.key);
+        rm = new Mutation(KEYSPACE1, key1.key);
         rm.add(COLUMN_FAMILY1, Util.cellname("1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
         rm.apply();
-        rm = new RowMutation(KEYSPACE1, key2.key);
+        rm = new Mutation(KEYSPACE1, key2.key);
         rm.add(COLUMN_FAMILY1, Util.cellname("2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
         rm.apply();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/KeyCollisionTest.java b/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
index 0f3e6d9..2c37f03 100644
--- a/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
@@ -88,8 +88,8 @@ public class KeyCollisionTest extends SchemaLoader
 
     private void insert(String key) throws IOException
     {
-        RowMutation rm;
-        rm = new RowMutation(KEYSPACE, ByteBufferUtil.bytes(key));
+        Mutation rm;
+        rm = new Mutation(KEYSPACE, ByteBufferUtil.bytes(key));
         rm.add(CF, Util.cellname("column"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/KeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/KeyspaceTest.java b/test/unit/org/apache/cassandra/db/KeyspaceTest.java
index fa2ddaf..ef69518 100644
--- a/test/unit/org/apache/cassandra/db/KeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyspaceTest.java
@@ -65,7 +65,7 @@ public class KeyspaceTest extends SchemaLoader
 
         ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace2", "Standard3");
         cf.addColumn(column("col1","val1", 1L));
-        RowMutation rm = new RowMutation("Keyspace2", TEST_KEY.key, cf);
+        Mutation rm = new Mutation("Keyspace2", TEST_KEY.key, cf);
         rm.apply();
 
         Runnable verify = new WrappedRunnable()
@@ -97,7 +97,7 @@ public class KeyspaceTest extends SchemaLoader
         cf.addColumn(column("col1","val1", 1L));
         cf.addColumn(column("col2","val2", 1L));
         cf.addColumn(column("col3","val3", 1L));
-        RowMutation rm = new RowMutation("Keyspace1", TEST_KEY.key, cf);
+        Mutation rm = new Mutation("Keyspace1", TEST_KEY.key, cf);
         rm.apply();
 
         Runnable verify = new WrappedRunnable()
@@ -127,7 +127,7 @@ public class KeyspaceTest extends SchemaLoader
         cf.addColumn(column("a", "val1", 1L));
         cf.addColumn(column("b", "val2", 1L));
         cf.addColumn(column("c", "val3", 1L));
-        RowMutation rm = new RowMutation("Keyspace1", key.key, cf);
+        Mutation rm = new Mutation("Keyspace1", key.key, cf);
         rm.apply();
 
         cf = cfStore.getColumnFamily(key, cellname("b"), cellname("c"), false, 100, System.currentTimeMillis());
@@ -146,7 +146,7 @@ public class KeyspaceTest extends SchemaLoader
         Keyspace keyspace = Keyspace.open("Keyspace1");
         ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard2");
         cf.addColumn(column("col1", "val1", 1));
-        RowMutation rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("row1000"), cf);
+        Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("row1000"), cf);
         rm.apply();
 
         validateGetSliceNoMatch(keyspace);
@@ -173,7 +173,7 @@ public class KeyspaceTest extends SchemaLoader
         // so if we go to 300, we'll get at least 4 blocks, which is plenty for testing.
         for (int i = 0; i < 300; i++)
             cf.addColumn(column("col" + fmt.format(i), "omg!thisisthevalue!"+i, 1L));
-        RowMutation rm = new RowMutation("Keyspace1", ROW.key, cf);
+        Mutation rm = new Mutation("Keyspace1", ROW.key, cf);
         rm.apply();
 
         Runnable verify = new WrappedRunnable()
@@ -228,7 +228,7 @@ public class KeyspaceTest extends SchemaLoader
         {
             ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "StandardLong1");
             cf.addColumn(new Cell(cellname((long)i), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0));
-            RowMutation rm = new RowMutation("Keyspace1", ROW.key, cf);
+            Mutation rm = new Mutation("Keyspace1", ROW.key, cf);
             rm.apply();
         }
 
@@ -238,7 +238,7 @@ public class KeyspaceTest extends SchemaLoader
         {
             ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "StandardLong1");
             cf.addColumn(new Cell(cellname((long)i), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0));
-            RowMutation rm = new RowMutation("Keyspace1", ROW.key, cf);
+            Mutation rm = new Mutation("Keyspace1", ROW.key, cf);
             rm.apply();
 
             cf = cfs.getColumnFamily(ROW, Composites.EMPTY, Composites.EMPTY, true, 1, System.currentTimeMillis());
@@ -276,10 +276,10 @@ public class KeyspaceTest extends SchemaLoader
         cf.addColumn(column("col5", "val5", 1L));
         cf.addColumn(column("col7", "val7", 1L));
         cf.addColumn(column("col9", "val9", 1L));
-        RowMutation rm = new RowMutation("Keyspace1", ROW.key, cf);
+        Mutation rm = new Mutation("Keyspace1", ROW.key, cf);
         rm.apply();
 
-        rm = new RowMutation("Keyspace1", ROW.key);
+        rm = new Mutation("Keyspace1", ROW.key);
         rm.delete("Standard1", cellname("col4"), 2L);
         rm.apply();
 
@@ -328,7 +328,7 @@ public class KeyspaceTest extends SchemaLoader
         cf.addColumn(column("col1", "val1", 1L));
         cf.addColumn(expiringColumn("col2", "val2", 1L, 60)); // long enough not to be tombstoned
         cf.addColumn(column("col3", "val3", 1L));
-        RowMutation rm = new RowMutation("Keyspace1", ROW.key, cf);
+        Mutation rm = new Mutation("Keyspace1", ROW.key, cf);
         rm.apply();
 
         Runnable verify = new WrappedRunnable()
@@ -365,7 +365,7 @@ public class KeyspaceTest extends SchemaLoader
         cf.addColumn(column("col4", "val4", 1L));
         cf.addColumn(column("col5", "val5", 1L));
         cf.addColumn(column("col6", "val6", 1L));
-        RowMutation rm = new RowMutation("Keyspace1", ROW.key, cf);
+        Mutation rm = new Mutation("Keyspace1", ROW.key, cf);
         rm.apply();
         cfStore.forceBlockingFlush();
 
@@ -373,7 +373,7 @@ public class KeyspaceTest extends SchemaLoader
         cf.addColumn(column("col1", "valx", 2L));
         cf.addColumn(column("col2", "valx", 2L));
         cf.addColumn(column("col3", "valx", 2L));
-        rm = new RowMutation("Keyspace1", ROW.key, cf);
+        rm = new Mutation("Keyspace1", ROW.key, cf);
         rm.apply();
 
         Runnable verify = new WrappedRunnable()
@@ -409,7 +409,7 @@ public class KeyspaceTest extends SchemaLoader
         ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
         for (int i = 1000; i < 2000; i++)
             cf.addColumn(column("col" + i, ("v" + i), 1L));
-        RowMutation rm = new RowMutation("Keyspace1", key.key, cf);
+        Mutation rm = new Mutation("Keyspace1", key.key, cf);
         rm.apply();
         cfStore.forceBlockingFlush();
 
@@ -442,7 +442,7 @@ public class KeyspaceTest extends SchemaLoader
             {
                 cf.addColumn(column("col" + i, ("v" + i), i));
             }
-            RowMutation rm = new RowMutation("Keyspace1", key.key, cf);
+            Mutation rm = new Mutation("Keyspace1", key.key, cf);
             rm.apply();
             cfStore.forceBlockingFlush();
         }
@@ -506,7 +506,7 @@ public class KeyspaceTest extends SchemaLoader
         {
             for (int i = 0; i < 10; i++)
             {
-                RowMutation rm = new RowMutation("Keyspace1", key.key);
+                Mutation rm = new Mutation("Keyspace1", key.key);
                 CellName colName = type.makeCellName(ByteBufferUtil.bytes("a" + i), ByteBufferUtil.bytes(j*10 + i));
                 rm.add("StandardComposite2", colName, ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
                 rm.apply();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/MultitableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/MultitableTest.java b/test/unit/org/apache/cassandra/db/MultitableTest.java
index 3d0c8ac..dc8208e 100644
--- a/test/unit/org/apache/cassandra/db/MultitableTest.java
+++ b/test/unit/org/apache/cassandra/db/MultitableTest.java
@@ -39,18 +39,18 @@ public class MultitableTest extends SchemaLoader
         Keyspace keyspace1 = Keyspace.open("Keyspace1");
         Keyspace keyspace2 = Keyspace.open("Keyspace2");
 
-        RowMutation rm;
+        Mutation rm;
         DecoratedKey dk = Util.dk("keymulti");
         ColumnFamily cf;
 
         cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
         cf.addColumn(column("col1", "val1", 1L));
-        rm = new RowMutation("Keyspace1", dk.key, cf);
+        rm = new Mutation("Keyspace1", dk.key, cf);
         rm.apply();
 
         cf = TreeMapBackedSortedColumns.factory.create("Keyspace2", "Standard1");
         cf.addColumn(column("col2", "val2", 1L));
-        rm = new RowMutation("Keyspace2", dk.key, cf);
+        rm = new Mutation("Keyspace2", dk.key, cf);
         rm.apply();
 
         keyspace1.getColumnFamilyStore("Standard1").forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/NameSortTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/NameSortTest.java b/test/unit/org/apache/cassandra/db/NameSortTest.java
index 16a1bc0..6bd71c7 100644
--- a/test/unit/org/apache/cassandra/db/NameSortTest.java
+++ b/test/unit/org/apache/cassandra/db/NameSortTest.java
@@ -60,13 +60,13 @@ public class NameSortTest extends SchemaLoader
         for (int i = 0; i < N; ++i)
         {
             ByteBuffer key = ByteBufferUtil.bytes(Integer.toString(i));
-            RowMutation rm;
+            Mutation rm;
 
             // standard
             for (int j = 0; j < 8; ++j)
             {
                 ByteBuffer bytes = j % 2 == 0 ? ByteBufferUtil.bytes("a") : ByteBufferUtil.bytes("b");
-                rm = new RowMutation("Keyspace1", key);
+                rm = new Mutation("Keyspace1", key);
                 rm.add("Standard1", Util.cellname("Cell-" + j), bytes, j);
                 rm.applyUnsafe();
             }
@@ -74,7 +74,7 @@ public class NameSortTest extends SchemaLoader
             // super
             for (int j = 0; j < 8; ++j)
             {
-                rm = new RowMutation("Keyspace1", key);
+                rm = new Mutation("Keyspace1", key);
                 for (int k = 0; k < 4; ++k)
                 {
                     String value = (j + k) % 2 == 0 ? "a" : "b";

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
index 5c37a30..3bbf4b8 100644
--- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
+++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
@@ -56,28 +56,28 @@ public class RangeTombstoneTest extends SchemaLoader
 
         // Inserting data
         String key = "k1";
-        RowMutation rm;
+        Mutation rm;
         ColumnFamily cf;
 
-        rm = new RowMutation(KSNAME, ByteBufferUtil.bytes(key));
+        rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         for (int i = 0; i < 40; i += 2)
             add(rm, i, 0);
         rm.apply();
         cfs.forceBlockingFlush();
 
-        rm = new RowMutation(KSNAME, ByteBufferUtil.bytes(key));
+        rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         cf = rm.addOrGet(CFNAME);
         delete(cf, 10, 22, 1);
         rm.apply();
         cfs.forceBlockingFlush();
 
-        rm = new RowMutation(KSNAME, ByteBufferUtil.bytes(key));
+        rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         for (int i = 1; i < 40; i += 2)
             add(rm, i, 2);
         rm.apply();
         cfs.forceBlockingFlush();
 
-        rm = new RowMutation(KSNAME, ByteBufferUtil.bytes(key));
+        rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         cf = rm.addOrGet(CFNAME);
         delete(cf, 19, 27, 3);
         rm.apply();
@@ -116,28 +116,28 @@ public class RangeTombstoneTest extends SchemaLoader
 
         // Inserting data
         String key = "k2";
-        RowMutation rm;
+        Mutation rm;
         ColumnFamily cf;
 
-        rm = new RowMutation(KSNAME, ByteBufferUtil.bytes(key));
+        rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         for (int i = 0; i < 20; i++)
             add(rm, i, 0);
         rm.apply();
         cfs.forceBlockingFlush();
 
-        rm = new RowMutation(KSNAME, ByteBufferUtil.bytes(key));
+        rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         cf = rm.addOrGet(CFNAME);
         delete(cf, 5, 15, 1);
         rm.apply();
         cfs.forceBlockingFlush();
 
-        rm = new RowMutation(KSNAME, ByteBufferUtil.bytes(key));
+        rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         cf = rm.addOrGet(CFNAME);
         delete(cf, 5, 10, 1);
         rm.apply();
         cfs.forceBlockingFlush();
 
-        rm = new RowMutation(KSNAME, ByteBufferUtil.bytes(key));
+        rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         cf = rm.addOrGet(CFNAME);
         delete(cf, 5, 8, 2);
         rm.apply();
@@ -172,15 +172,15 @@ public class RangeTombstoneTest extends SchemaLoader
 
         // Inserting data
         String key = "k3";
-        RowMutation rm;
+        Mutation rm;
         ColumnFamily cf;
 
-        rm = new RowMutation(KSNAME, ByteBufferUtil.bytes(key));
+        rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         add(rm, 2, 0);
         rm.apply();
         cfs.forceBlockingFlush();
 
-        rm = new RowMutation(KSNAME, ByteBufferUtil.bytes(key));
+        rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         // Deletes everything but without being a row tombstone
         delete(rm.addOrGet(CFNAME), 0, 10, 1);
         add(rm, 1, 2);
@@ -222,13 +222,13 @@ public class RangeTombstoneTest extends SchemaLoader
         cfs.disableAutoCompaction();
         cfs.setCompactionStrategyClass(SizeTieredCompactionStrategy.class.getCanonicalName());
 
-        RowMutation rm = new RowMutation(KSNAME, key);
+        Mutation rm = new Mutation(KSNAME, key);
         for (int i = 0; i < 10; i += 2)
             add(rm, i, 0);
         rm.apply();
         cfs.forceBlockingFlush();
 
-        rm = new RowMutation(KSNAME, key);
+        rm = new Mutation(KSNAME, key);
         ColumnFamily cf = rm.addOrGet(CFNAME);
         for (int i = 0; i < 10; i += 2)
             delete(cf, 0, 7, 0);
@@ -280,7 +280,7 @@ public class RangeTombstoneTest extends SchemaLoader
         TestIndex index = ((TestIndex)cfs.indexManager.getIndexForColumn(indexedColumnName));
         index.resetCounts();
 
-        RowMutation rm = new RowMutation(KSNAME, key);
+        Mutation rm = new Mutation(KSNAME, key);
         for (int i = 0; i < 10; i++)
             add(rm, i, 0);
         rm.apply();
@@ -288,7 +288,7 @@ public class RangeTombstoneTest extends SchemaLoader
         // We should have indexed 1 column
         assertEquals(1, index.inserts.size());
 
-        rm = new RowMutation(KSNAME, key);
+        rm = new Mutation(KSNAME, key);
         ColumnFamily cf = rm.addOrGet(CFNAME);
         for (int i = 0; i < 10; i += 2)
             delete(cf, 0, 7, 0);
@@ -319,13 +319,13 @@ public class RangeTombstoneTest extends SchemaLoader
         TestIndex index = ((TestIndex)cfs.indexManager.getIndexForColumn(indexedColumnName));
         index.resetCounts();
 
-        RowMutation rm = new RowMutation(KSNAME, key);
+        Mutation rm = new Mutation(KSNAME, key);
         for (int i = 0; i < 10; i++)
             add(rm, i, 0);
         rm.apply();
         cfs.forceBlockingFlush();
 
-        rm = new RowMutation(KSNAME, key);
+        rm = new Mutation(KSNAME, key);
         ColumnFamily cf = rm.addOrGet(CFNAME);
         for (int i = 0; i < 10; i += 2)
             delete(cf, 0, 7, 0);
@@ -360,7 +360,7 @@ public class RangeTombstoneTest extends SchemaLoader
         return ByteBufferUtil.toInt(i.toByteBuffer());
     }
 
-    private static void add(RowMutation rm, int value, long timestamp)
+    private static void add(Mutation rm, int value, long timestamp)
     {
         rm.add(CFNAME, b(value), ByteBufferUtil.bytes(value), timestamp);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/ReadMessageTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ReadMessageTest.java b/test/unit/org/apache/cassandra/db/ReadMessageTest.java
index 1f73b35..d54594f 100644
--- a/test/unit/org/apache/cassandra/db/ReadMessageTest.java
+++ b/test/unit/org/apache/cassandra/db/ReadMessageTest.java
@@ -83,11 +83,11 @@ public class ReadMessageTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open("Keyspace1");
         CellNameType type = keyspace.getColumnFamilyStore("Standard1").getComparator();
-        RowMutation rm;
+        Mutation rm;
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new RowMutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.key);
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("abcd"), 0);
         rm.apply();
 
@@ -100,11 +100,11 @@ public class ReadMessageTest extends SchemaLoader
     @Test
     public void testNoCommitLog() throws Exception
     {
-        RowMutation rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("row"));
+        Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("row"));
         rm.add("Standard1", Util.cellname("commit1"), ByteBufferUtil.bytes("abcd"), 0);
         rm.apply();
 
-        rm = new RowMutation("NoCommitlogSpace", ByteBufferUtil.bytes("row"));
+        rm = new Mutation("NoCommitlogSpace", ByteBufferUtil.bytes("row"));
         rm.add("Standard1", Util.cellname("commit2"), ByteBufferUtil.bytes("abcd"), 0);
         rm.apply();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/RecoveryManager2Test.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManager2Test.java b/test/unit/org/apache/cassandra/db/RecoveryManager2Test.java
index ac2d642..8e4145c 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManager2Test.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManager2Test.java
@@ -73,7 +73,7 @@ public class RecoveryManager2Test extends SchemaLoader
     {
         ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", cfname);
         cf.addColumn(column("col1", "val1", 1L));
-        RowMutation rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes(key), cf);
+        Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes(key), cf);
         rm.apply();
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java b/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java
index 8258970..be11967 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java
@@ -44,18 +44,18 @@ public class RecoveryManager3Test extends SchemaLoader
         Keyspace keyspace1 = Keyspace.open("Keyspace1");
         Keyspace keyspace2 = Keyspace.open("Keyspace2");
 
-        RowMutation rm;
+        Mutation rm;
         DecoratedKey dk = Util.dk("keymulti");
         ColumnFamily cf;
 
         cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
         cf.addColumn(column("col1", "val1", 1L));
-        rm = new RowMutation("Keyspace1", dk.key, cf);
+        rm = new Mutation("Keyspace1", dk.key, cf);
         rm.apply();
 
         cf = TreeMapBackedSortedColumns.factory.create("Keyspace2", "Standard3");
         cf.addColumn(column("col2", "val2", 1L));
-        rm = new RowMutation("Keyspace2", dk.key, cf);
+        rm = new Mutation("Keyspace2", dk.key, cf);
         rm.apply();
 
         keyspace1.getColumnFamilyStore("Standard1").clearUnsafe();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java b/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
index 0c3c23a..9f73ff2 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
@@ -48,18 +48,18 @@ public class RecoveryManagerTest extends SchemaLoader
         Keyspace keyspace1 = Keyspace.open("Keyspace1");
         Keyspace keyspace2 = Keyspace.open("Keyspace2");
 
-        RowMutation rm;
+        Mutation rm;
         DecoratedKey dk = Util.dk("keymulti");
         ColumnFamily cf;
 
         cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
         cf.addColumn(column("col1", "val1", 1L));
-        rm = new RowMutation("Keyspace1", dk.key, cf);
+        rm = new Mutation("Keyspace1", dk.key, cf);
         rm.apply();
 
         cf = TreeMapBackedSortedColumns.factory.create("Keyspace2", "Standard3");
         cf.addColumn(column("col2", "val2", 1L));
-        rm = new RowMutation("Keyspace2", dk.key, cf);
+        rm = new Mutation("Keyspace2", dk.key, cf);
         rm.apply();
 
         keyspace1.getColumnFamilyStore("Standard1").clearUnsafe();
@@ -77,7 +77,7 @@ public class RecoveryManagerTest extends SchemaLoader
     {
         Keyspace keyspace1 = Keyspace.open("Keyspace1");
 
-        RowMutation rm;
+        Mutation rm;
         DecoratedKey dk = Util.dk("key");
         ColumnFamily cf;
 
@@ -85,7 +85,7 @@ public class RecoveryManagerTest extends SchemaLoader
         {
             cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Counter1");
             cf.addColumn(new CounterCell(cellname("col"), 1L, 1L));
-            rm = new RowMutation("Keyspace1", dk.key, cf);
+            rm = new Mutation("Keyspace1", dk.key, cf);
             rm.apply();
         }
 
@@ -116,7 +116,7 @@ public class RecoveryManagerTest extends SchemaLoader
             long ts = TimeUnit.MILLISECONDS.toMicros(timeMS + (i * 1000));
             ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
             cf.addColumn(column("name-" + i, "value", ts));
-            RowMutation rm = new RowMutation("Keyspace1", dk.key, cf);
+            Mutation rm = new Mutation("Keyspace1", dk.key, cf);
             rm.apply();
         }
         keyspace1.getColumnFamilyStore("Standard1").clearUnsafe();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java b/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
index a5e3876..16ab04a 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
@@ -42,13 +42,13 @@ public class RecoveryManagerTruncateTest extends SchemaLoader
 		Keyspace keyspace = Keyspace.open("Keyspace1");
 		ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
 
-		RowMutation rm;
+		Mutation rm;
 		ColumnFamily cf;
 
 		// add a single cell
         cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
 		cf.addColumn(column("col1", "val1", 1L));
-        rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("keymulti"), cf);
+        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("keymulti"), cf);
 		rm.apply();
 
 		// Make sure data was written

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/RemoveCellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveCellTest.java b/test/unit/org/apache/cassandra/db/RemoveCellTest.java
index 1b03122..629b40d 100644
--- a/test/unit/org/apache/cassandra/db/RemoveCellTest.java
+++ b/test/unit/org/apache/cassandra/db/RemoveCellTest.java
@@ -37,17 +37,17 @@ public class RemoveCellTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open("Keyspace1");
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
-        RowMutation rm;
+        Mutation rm;
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new RowMutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.key);
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
         store.forceBlockingFlush();
 
         // remove
-        rm = new RowMutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.key);
         rm.delete("Standard1", Util.cellname("Column1"), 1);
         rm.apply();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
index 236ba10..b80b9d1 100644
--- a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
+++ b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
@@ -35,16 +35,16 @@ public class RemoveColumnFamilyTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open("Keyspace1");
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
-        RowMutation rm;
+        Mutation rm;
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new RowMutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.key);
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
 
         // remove
-        rm = new RowMutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.key);
         rm.delete("Standard1", 1);
         rm.apply();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
index 1bfb140..c95ab97 100644
--- a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
+++ b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
@@ -35,18 +35,18 @@ public class RemoveColumnFamilyWithFlush1Test extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open("Keyspace1");
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
-        RowMutation rm;
+        Mutation rm;
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new RowMutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.key);
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.add("Standard1", Util.cellname("Column2"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
         store.forceBlockingFlush();
 
         // remove
-        rm = new RowMutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.key);
         rm.delete("Standard1", 1);
         rm.apply();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
index 9c8aba1..df81d58 100644
--- a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
+++ b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
@@ -35,15 +35,15 @@ public class RemoveColumnFamilyWithFlush2Test extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open("Keyspace1");
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
-        RowMutation rm;
+        Mutation rm;
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new RowMutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.key);
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
         // remove
-        rm = new RowMutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.key);
         rm.delete("Standard1", 1);
         rm.apply();
         store.forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/RemoveSubCellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveSubCellTest.java b/test/unit/org/apache/cassandra/db/RemoveSubCellTest.java
index 9c3d29c..3026897 100644
--- a/test/unit/org/apache/cassandra/db/RemoveSubCellTest.java
+++ b/test/unit/org/apache/cassandra/db/RemoveSubCellTest.java
@@ -26,7 +26,7 @@ import org.junit.Test;
 import static org.junit.Assert.assertNull;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.db.marshal.CompositeType;
+
 import static org.apache.cassandra.Util.getBytes;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.SchemaLoader;
@@ -42,18 +42,18 @@ public class RemoveSubCellTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open("Keyspace1");
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Super1");
-        RowMutation rm;
+        Mutation rm;
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new RowMutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.key);
         Util.addMutation(rm, "Super1", "SC1", 1, "asdf", 0);
         rm.apply();
         store.forceBlockingFlush();
 
         CellName cname = CellNames.compositeDense(ByteBufferUtil.bytes("SC1"), getBytes(1L));
         // remove
-        rm = new RowMutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.key);
         rm.delete("Super1", cname, 1);
         rm.apply();
 
@@ -67,11 +67,11 @@ public class RemoveSubCellTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open("Keyspace1");
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Super1");
-        RowMutation rm;
+        Mutation rm;
         DecoratedKey dk = Util.dk("key2");
 
         // add data
-        rm = new RowMutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.key);
         Util.addMutation(rm, "Super1", "SC1", 1, "asdf", 0);
         rm.apply();
         store.forceBlockingFlush();
@@ -79,7 +79,7 @@ public class RemoveSubCellTest extends SchemaLoader
         // remove the SC
         ByteBuffer scName = ByteBufferUtil.bytes("SC1");
         CellName cname = CellNames.compositeDense(scName, getBytes(1L));
-        rm = new RowMutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.key);
         rm.deleteRange("Super1", SuperColumns.startOf(scName), SuperColumns.endOf(scName), 1);
         rm.apply();
 
@@ -89,7 +89,7 @@ public class RemoveSubCellTest extends SchemaLoader
         Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
 
         // remove the column itself
-        rm = new RowMutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.key);
         rm.delete("Super1", cname, 2);
         rm.apply();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/RowIterationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowIterationTest.java b/test/unit/org/apache/cassandra/db/RowIterationTest.java
index 9ec7bec..fcdd2e1 100644
--- a/test/unit/org/apache/cassandra/db/RowIterationTest.java
+++ b/test/unit/org/apache/cassandra/db/RowIterationTest.java
@@ -31,7 +31,6 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.db.composites.*;
-import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.utils.FBUtilities;
 import static org.junit.Assert.assertEquals;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -52,7 +51,7 @@ public class RowIterationTest extends SchemaLoader
         Set<DecoratedKey> inserted = new HashSet<DecoratedKey>();
         for (int i = 0; i < ROWS_PER_SSTABLE; i++) {
             DecoratedKey key = Util.dk(String.valueOf(i));
-            RowMutation rm = new RowMutation(KEYSPACE1, key.key);
+            Mutation rm = new Mutation(KEYSPACE1, key.key);
             rm.add("Super3", CellNames.compositeDense(ByteBufferUtil.bytes("sc"), ByteBufferUtil.bytes(String.valueOf(i))), ByteBuffer.wrap(new byte[ROWS_PER_SSTABLE * 10 - i * 2]), i);
             rm.apply();
             inserted.add(key);
@@ -70,7 +69,7 @@ public class RowIterationTest extends SchemaLoader
         DecoratedKey key = Util.dk("key");
 
         // Delete row in first sstable
-        RowMutation rm = new RowMutation(KEYSPACE1, key.key);
+        Mutation rm = new Mutation(KEYSPACE1, key.key);
         rm.delete(CF_NAME, 0);
         rm.add(CF_NAME, Util.cellname("c"), ByteBufferUtil.bytes("values"), 0L);
         DeletionInfo delInfo1 = rm.getColumnFamilies().iterator().next().deletionInfo();
@@ -78,7 +77,7 @@ public class RowIterationTest extends SchemaLoader
         store.forceBlockingFlush();
 
         // Delete row in second sstable with higher timestamp
-        rm = new RowMutation(KEYSPACE1, key.key);
+        rm = new Mutation(KEYSPACE1, key.key);
         rm.delete(CF_NAME, 1);
         rm.add(CF_NAME, Util.cellname("c"), ByteBufferUtil.bytes("values"), 1L);
         DeletionInfo delInfo2 = rm.getColumnFamilies().iterator().next().deletionInfo();
@@ -99,7 +98,7 @@ public class RowIterationTest extends SchemaLoader
         DecoratedKey key = Util.dk("key");
 
         // Delete a row in first sstable
-        RowMutation rm = new RowMutation(KEYSPACE1, key.key);
+        Mutation rm = new Mutation(KEYSPACE1, key.key);
         rm.delete(CF_NAME, 0);
         rm.apply();
         store.forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ScrubTest.java b/test/unit/org/apache/cassandra/db/ScrubTest.java
index 614858d..38c8b62 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -85,7 +85,7 @@ public class ScrubTest extends SchemaLoader
 
         ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(KEYSPACE, CF3);
         cf.delete(new DeletionInfo(0, 1)); // expired tombstone
-        RowMutation rm = new RowMutation(KEYSPACE, ByteBufferUtil.bytes(1), cf);
+        Mutation rm = new Mutation(KEYSPACE, ByteBufferUtil.bytes(1), cf);
         rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
@@ -201,7 +201,7 @@ public class ScrubTest extends SchemaLoader
             ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(KEYSPACE, CF);
             cf.addColumn(column("c1", "1", 1L));
             cf.addColumn(column("c2", "2", 1L));
-            RowMutation rm = new RowMutation(KEYSPACE, ByteBufferUtil.bytes(key), cf);
+            Mutation rm = new Mutation(KEYSPACE, ByteBufferUtil.bytes(key), cf);
             rm.applyUnsafe();
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/SerializationsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SerializationsTest.java b/test/unit/org/apache/cassandra/db/SerializationsTest.java
index 983a8f7..2bc1493 100644
--- a/test/unit/org/apache/cassandra/db/SerializationsTest.java
+++ b/test/unit/org/apache/cassandra/db/SerializationsTest.java
@@ -210,23 +210,23 @@ public class SerializationsTest extends AbstractSerializationsTester
         in.close();
     }
 
-    private void testRowMutationWrite() throws IOException
+    private void testMutationWrite() throws IOException
     {
-        RowMutation standardRowRm = new RowMutation(statics.KS, statics.StandardRow);
-        RowMutation superRowRm = new RowMutation(statics.KS, statics.SuperRow);
-        RowMutation standardRm = new RowMutation(statics.KS, statics.Key, statics.StandardCf);
-        RowMutation superRm = new RowMutation(statics.KS, statics.Key, statics.SuperCf);
+        Mutation standardRowRm = new Mutation(statics.KS, statics.StandardRow);
+        Mutation superRowRm = new Mutation(statics.KS, statics.SuperRow);
+        Mutation standardRm = new Mutation(statics.KS, statics.Key, statics.StandardCf);
+        Mutation superRm = new Mutation(statics.KS, statics.Key, statics.SuperCf);
         Map<UUID, ColumnFamily> mods = new HashMap<UUID, ColumnFamily>();
         mods.put(statics.StandardCf.metadata().cfId, statics.StandardCf);
         mods.put(statics.SuperCf.metadata().cfId, statics.SuperCf);
-        RowMutation mixedRm = new RowMutation(statics.KS, statics.Key, mods);
+        Mutation mixedRm = new Mutation(statics.KS, statics.Key, mods);
 
         DataOutputStream out = getOutput("db.RowMutation.bin");
-        RowMutation.serializer.serialize(standardRowRm, out, getVersion());
-        RowMutation.serializer.serialize(superRowRm, out, getVersion());
-        RowMutation.serializer.serialize(standardRm, out, getVersion());
-        RowMutation.serializer.serialize(superRm, out, getVersion());
-        RowMutation.serializer.serialize(mixedRm, out, getVersion());
+        Mutation.serializer.serialize(standardRowRm, out, getVersion());
+        Mutation.serializer.serialize(superRowRm, out, getVersion());
+        Mutation.serializer.serialize(standardRm, out, getVersion());
+        Mutation.serializer.serialize(superRm, out, getVersion());
+        Mutation.serializer.serialize(mixedRm, out, getVersion());
 
         standardRowRm.createMessage().serialize(out, getVersion());
         superRowRm.createMessage().serialize(out, getVersion());
@@ -237,27 +237,27 @@ public class SerializationsTest extends AbstractSerializationsTester
         out.close();
 
         // test serializedSize
-        testSerializedSize(standardRowRm, RowMutation.serializer);
-        testSerializedSize(superRowRm, RowMutation.serializer);
-        testSerializedSize(standardRm, RowMutation.serializer);
-        testSerializedSize(superRm, RowMutation.serializer);
-        testSerializedSize(mixedRm, RowMutation.serializer);
+        testSerializedSize(standardRowRm, Mutation.serializer);
+        testSerializedSize(superRowRm, Mutation.serializer);
+        testSerializedSize(standardRm, Mutation.serializer);
+        testSerializedSize(superRm, Mutation.serializer);
+        testSerializedSize(mixedRm, Mutation.serializer);
     }
 
     @Test
-    public void testRowMutationRead() throws IOException
+    public void testMutationRead() throws IOException
     {
-        // row mutation deserialization requires being able to look up the keyspace in the schema,
+        // mutation deserialization requires being able to look up the keyspace in the schema,
         // so we need to rewrite this each time.  We can go back to testing on-disk data
         // once we pull RM.keyspace field out.
-        testRowMutationWrite();
+        testMutationWrite();
 
         DataInputStream in = getInput("db.RowMutation.bin");
-        assert RowMutation.serializer.deserialize(in, getVersion()) != null;
-        assert RowMutation.serializer.deserialize(in, getVersion()) != null;
-        assert RowMutation.serializer.deserialize(in, getVersion()) != null;
-        assert RowMutation.serializer.deserialize(in, getVersion()) != null;
-        assert RowMutation.serializer.deserialize(in, getVersion()) != null;
+        assert Mutation.serializer.deserialize(in, getVersion()) != null;
+        assert Mutation.serializer.deserialize(in, getVersion()) != null;
+        assert Mutation.serializer.deserialize(in, getVersion()) != null;
+        assert Mutation.serializer.deserialize(in, getVersion()) != null;
+        assert Mutation.serializer.deserialize(in, getVersion()) != null;
         assert MessageIn.read(in, getVersion(), -1) != null;
         assert MessageIn.read(in, getVersion(), -1) != null;
         assert MessageIn.read(in, getVersion(), -1) != null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6bbb13b9/test/unit/org/apache/cassandra/db/TimeSortTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/TimeSortTest.java b/test/unit/org/apache/cassandra/db/TimeSortTest.java
index f8e9dbc..c7a4a57 100644
--- a/test/unit/org/apache/cassandra/db/TimeSortTest.java
+++ b/test/unit/org/apache/cassandra/db/TimeSortTest.java
@@ -42,15 +42,15 @@ public class TimeSortTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open("Keyspace1");
         ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("StandardLong1");
-        RowMutation rm;
+        Mutation rm;
         DecoratedKey key = Util.dk("key0");
 
-        rm = new RowMutation("Keyspace1", key.key);
+        rm = new Mutation("Keyspace1", key.key);
         rm.add("StandardLong1", cellname(100), ByteBufferUtil.bytes("a"), 100);
         rm.apply();
         cfStore.forceBlockingFlush();
 
-        rm = new RowMutation("Keyspace1", key.key);
+        rm = new Mutation("Keyspace1", key.key);
         rm.add("StandardLong1", cellname(0), ByteBufferUtil.bytes("b"), 0);
         rm.apply();
 
@@ -67,7 +67,7 @@ public class TimeSortTest extends SchemaLoader
 
         for (int i = 900; i < 1000; ++i)
         {
-            RowMutation rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes(Integer.toString(i)));
+            Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes(Integer.toString(i)));
             for (int j = 0; j < 8; ++j)
             {
                 rm.add("StandardLong1", cellname(j * 2), ByteBufferUtil.bytes("a"), j * 2);
@@ -82,14 +82,14 @@ public class TimeSortTest extends SchemaLoader
 
         // interleave some new data to test memtable + sstable
         DecoratedKey key = Util.dk("900");
-        RowMutation rm = new RowMutation("Keyspace1", key.key);
+        Mutation rm = new Mutation("Keyspace1", key.key);
         for (int j = 0; j < 4; ++j)
         {
             rm.add("StandardLong1", cellname(j * 2 + 1), ByteBufferUtil.bytes("b"), j * 2 + 1);
         }
         rm.apply();
         // and some overwrites
-        rm = new RowMutation("Keyspace1", key.key);
+        rm = new Mutation("Keyspace1", key.key);
         rm.add("StandardLong1", cellname(0), ByteBufferUtil.bytes("c"), 100);
         rm.add("StandardLong1", cellname(10), ByteBufferUtil.bytes("c"), 100);
         rm.apply();