You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ty...@apache.org on 2014/07/09 22:42:29 UTC

[2/2] git commit: Use unsafe mutations where possible in unit tests

Use unsafe mutations where possible in unit tests

Patch by Lyuben Todorov; reviewed by Tyler Hobbs for CASSANDRA-6969


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/b453f089
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/b453f089
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/b453f089

Branch: refs/heads/trunk
Commit: b453f0897bac7f8e928588d7b288ea3e65082353
Parents: e2bb7d2
Author: lyubent <lt...@dundee.ac.uk>
Authored: Wed Jul 9 15:41:13 2014 -0500
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Wed Jul 9 15:42:11 2014 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 test/unit/org/apache/cassandra/Util.java        |  2 +-
 .../cassandra/cache/AutoSavingCacheTest.java    |  2 +-
 .../org/apache/cassandra/config/DefsTest.java   | 20 ++---
 .../cassandra/db/BatchlogManagerTest.java       |  4 +-
 .../cassandra/db/CollationControllerTest.java   | 12 +--
 .../cassandra/db/ColumnFamilyStoreTest.java     | 78 ++++++++++----------
 .../apache/cassandra/db/HintedHandOffTest.java  |  4 +-
 .../org/apache/cassandra/db/KeyCacheTest.java   |  4 +-
 .../apache/cassandra/db/KeyCollisionTest.java   |  2 +-
 .../org/apache/cassandra/db/KeyspaceTest.java   | 30 ++++----
 .../org/apache/cassandra/db/MultitableTest.java |  4 +-
 .../apache/cassandra/db/RangeTombstoneTest.java | 40 +++++-----
 .../db/RecoveryManagerTruncateTest.java         |  2 +-
 .../org/apache/cassandra/db/RemoveCellTest.java |  4 +-
 .../cassandra/db/RemoveColumnFamilyTest.java    |  4 +-
 .../db/RemoveColumnFamilyWithFlush1Test.java    |  4 +-
 .../db/RemoveColumnFamilyWithFlush2Test.java    |  4 +-
 .../apache/cassandra/db/RemoveSubCellTest.java  | 10 +--
 .../apache/cassandra/db/RowIterationTest.java   |  8 +-
 .../org/apache/cassandra/db/TimeSortTest.java   | 10 +--
 .../db/compaction/AntiCompactionTest.java       |  2 +-
 .../compaction/BlacklistingCompactionsTest.java |  2 +-
 .../db/compaction/CompactionsPurgeTest.java     | 34 ++++-----
 .../db/compaction/CompactionsTest.java          | 18 ++---
 .../LeveledCompactionStrategyTest.java          |  8 +-
 .../db/compaction/OneCompactionTest.java        |  2 +-
 .../SizeTieredCompactionStrategyTest.java       |  4 +-
 .../cassandra/db/compaction/TTLExpiryTest.java  | 16 ++--
 .../db/index/PerRowSecondaryIndexTest.java      |  8 +-
 .../cassandra/db/marshal/CompositeTypeTest.java |  2 +-
 .../db/marshal/DynamicCompositeTypeTest.java    |  2 +-
 .../io/sstable/IndexSummaryManagerTest.java     |  4 +-
 .../io/sstable/SSTableMetadataTest.java         | 26 +++----
 .../cassandra/io/sstable/SSTableReaderTest.java | 20 ++---
 .../io/sstable/SSTableScannerTest.java          |  2 +-
 .../streaming/StreamingTransferTest.java        |  6 +-
 37 files changed, 203 insertions(+), 202 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 7f74520..ff0a1c6 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -10,6 +10,7 @@
  * Enable code coverage thru JaCoCo (CASSANDRA-7226)
  * Switch external naming of 'column families' to 'tables' (CASSANDRA-4369) 
  * Shorten SSTable path (CASSANDRA-6962)
+ * Use unsafe mutations for most unit tests (CASSANDRA-6969)
 
 
 2.1.1

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/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 a8747c4..e06bd95 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -190,7 +190,7 @@ public class Util
         UUID cfid = first.getColumnFamilyIds().iterator().next();
 
         for (Mutation rm : mutations)
-            rm.apply();
+            rm.applyUnsafe();
 
         ColumnFamilyStore store = Keyspace.open(keyspaceName).getColumnFamilyStore(cfid);
         store.forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java b/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java
index 683062c..3da4555 100644
--- a/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java
+++ b/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java
@@ -57,7 +57,7 @@ public class AutoSavingCacheTest
         {
             Mutation rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("key1"));
             rm.add(CF_STANDARD1, Util.cellname("c1"), ByteBufferUtil.bytes(i), 0);
-            rm.apply();
+            rm.applyUnsafe();
             cfs.forceBlockingFlush();
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/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 206a336..6df67bc 100644
--- a/test/unit/org/apache/cassandra/config/DefsTest.java
+++ b/test/unit/org/apache/cassandra/config/DefsTest.java
@@ -202,7 +202,7 @@ public class DefsTest
         DecoratedKey dk = Util.dk("key0");
         Mutation rm = new Mutation(ks, dk.getKey());
         rm.add(cf, col0, ByteBufferUtil.bytes("value0"), 1L);
-        rm.apply();
+        rm.applyUnsafe();
         ColumnFamilyStore store = Keyspace.open(ks).getColumnFamilyStore(cf);
         Assert.assertNotNull(store);
         store.forceBlockingFlush();
@@ -227,7 +227,7 @@ public class DefsTest
         Mutation rm = new Mutation(ks.name, dk.getKey());
         for (int i = 0; i < 100; i++)
             rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
-        rm.apply();
+        rm.applyUnsafe();
         ColumnFamilyStore store = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
         Assert.assertNotNull(store);
         store.forceBlockingFlush();
@@ -243,7 +243,7 @@ public class DefsTest
         try
         {
             rm.add("Standard1", cellname("col0"), ByteBufferUtil.bytes("value0"), 1L);
-            rm.apply();
+            rm.applyUnsafe();
         }
         catch (Throwable th)
         {
@@ -276,7 +276,7 @@ public class DefsTest
         CellName col0 = cellname("col0");
         Mutation rm = new Mutation(newCf.ksName, dk.getKey());
         rm.add(newCf.cfName, col0, ByteBufferUtil.bytes("value0"), 1L);
-        rm.apply();
+        rm.applyUnsafe();
         ColumnFamilyStore store = Keyspace.open(newCf.ksName).getColumnFamilyStore(newCf.cfName);
         Assert.assertNotNull(store);
         store.forceBlockingFlush();
@@ -301,7 +301,7 @@ public class DefsTest
         Mutation rm = new Mutation(ks.name, dk.getKey());
         for (int i = 0; i < 100; i++)
             rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
-        rm.apply();
+        rm.applyUnsafe();
         ColumnFamilyStore store = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
         Assert.assertNotNull(store);
         store.forceBlockingFlush();
@@ -317,7 +317,7 @@ public class DefsTest
         try
         {
             rm.add("Standard1", cellname("col0"), ByteBufferUtil.bytes("value0"), 1L);
-            rm.apply();
+            rm.applyUnsafe();
         }
         catch (Throwable th)
         {
@@ -352,7 +352,7 @@ public class DefsTest
         Mutation rm = new Mutation(ks.name, dk.getKey());
         for (int i = 0; i < 100; i++)
             rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
-        rm.apply();
+        rm.applyUnsafe();
 
         MigrationManager.announceKeyspaceDrop(ks.name);
 
@@ -385,7 +385,7 @@ public class DefsTest
         DecoratedKey dk = Util.dk("key0");
         Mutation rm = new Mutation(newKs.name, dk.getKey());
         rm.add(newCf.cfName, col0, ByteBufferUtil.bytes("value0"), 1L);
-        rm.apply();
+        rm.applyUnsafe();
         ColumnFamilyStore store = Keyspace.open(newKs.name).getColumnFamilyStore(newCf.cfName);
         Assert.assertNotNull(store);
         store.forceBlockingFlush();
@@ -528,14 +528,14 @@ public class DefsTest
     public void testDropIndex() throws ConfigurationException
     {
         // persist keyspace definition in the system keyspace
-        Schema.instance.getKSMetaData(KEYSPACE6).toSchema(System.currentTimeMillis()).apply();
+        Schema.instance.getKSMetaData(KEYSPACE6).toSchema(System.currentTimeMillis()).applyUnsafe();
         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
         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();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
         ColumnFamilyStore indexedCfs = cfs.indexManager.getIndexForColumn(ByteBufferUtil.bytes("birthdate")).getIndexCfs();
         Descriptor desc = indexedCfs.getSSTables().iterator().next().descriptor;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java b/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java
index 6f36fe9..6313ca9 100644
--- a/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java
@@ -97,7 +97,7 @@ public class BatchlogManagerTest
                                                    UUIDGen.getTimeUUID(),
                                                    MessagingService.current_version,
                                                    timestamp)
-                           .apply();
+                           .applyUnsafe();
         }
 
         // Flush the batchlog to disk (see CASSANDRA-6822).
@@ -167,7 +167,7 @@ public class BatchlogManagerTest
                                                    UUIDGen.getTimeUUID(),
                                                    MessagingService.current_version,
                                                    timestamp * 1000)
-                           .apply();
+                           .applyUnsafe();
         }
 
         // Flush the batchlog to disk (see CASSANDRA-6822).

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/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 923cb9e..c227816 100644
--- a/test/unit/org/apache/cassandra/db/CollationControllerTest.java
+++ b/test/unit/org/apache/cassandra/db/CollationControllerTest.java
@@ -61,26 +61,26 @@ public class CollationControllerTest
         // add data
         rm = new Mutation(keyspace.getName(), dk.getKey());
         rm.add(cfs.name, Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
         
         // remove
         rm = new Mutation(keyspace.getName(), dk.getKey());
         rm.delete(cfs.name, 10);
-        rm.apply();
+        rm.applyUnsafe();
         
         // add another mutation because sstable maxtimestamp isn't set
         // correctly during flush if the most recent mutation is a row delete
         rm = new Mutation(keyspace.getName(), Util.dk("key2").getKey());
         rm.add(cfs.name, Util.cellname("Column1"), ByteBufferUtil.bytes("zxcv"), 20);
-        rm.apply();
+        rm.applyUnsafe();
         
         cfs.forceBlockingFlush();
 
         // add yet one more mutation
         rm = new Mutation(keyspace.getName(), dk.getKey());
         rm.add(cfs.name, Util.cellname("Column1"), ByteBufferUtil.bytes("foobar"), 30);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         // A NamesQueryFilter goes down one code path (through collectTimeOrderedData())
@@ -113,13 +113,13 @@ public class CollationControllerTest
         // add data
         rm = new Mutation(keyspace.getName(), dk.getKey());
         rm.add(cfs.name, cellName, ByteBufferUtil.bytes("asdf"), 0);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         // remove
         rm = new Mutation(keyspace.getName(), dk.getKey());
         rm.delete(cfs.name, cellName, 0);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         // use "realistic" query times since we'll compare these numbers to the local deletion time of the tombstone

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/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 467a451..386be01 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@ -172,12 +172,12 @@ public class ColumnFamilyStoreTest
         Mutation rm;
         rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("key1"));
         rm.add(CF_STANDARD1, cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("key1"));
         rm.add(CF_STANDARD1, cellname("Column1"), ByteBufferUtil.bytes("asdf"), 1);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         cfs.getRecentSSTablesPerReadHistogram(); // resets counts
@@ -215,7 +215,7 @@ public class ColumnFamilyStoreTest
 
         rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("key1"));
         rm.delete(CF_STANDARD2, System.currentTimeMillis());
-        rm.apply();
+        rm.applyUnsafe();
 
         Runnable r = new WrappedRunnable()
         {
@@ -261,22 +261,22 @@ public class ColumnFamilyStoreTest
         rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k1"));
         rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(1L), 0);
         rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(1L), 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k2"));
         rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(2L), 0);
         rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(2L), 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k3"));
         rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(2L), 0);
         rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(1L), 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k4aaaa"));
         rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(2L), 0);
         rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(3L), 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         // basic single-expression query
         IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(1L));
@@ -371,7 +371,7 @@ public class ColumnFamilyStoreTest
 
         rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
         rm.add(CF_INDEX1, cellname("birthdate"), ByteBufferUtil.bytes(1L), 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(1L));
         List<IndexExpression> clause = Arrays.asList(expr);
@@ -385,7 +385,7 @@ public class ColumnFamilyStoreTest
         // delete the column directly
         rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
         rm.delete(CF_INDEX1, cellname("birthdate"), 1);
-        rm.apply();
+        rm.applyUnsafe();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.isEmpty();
 
@@ -400,7 +400,7 @@ public class ColumnFamilyStoreTest
         // resurrect w/ a newer timestamp
         rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
         rm.add(CF_INDEX1, cellname("birthdate"), ByteBufferUtil.bytes(1L), 2);
-        rm.apply();
+        rm.applyUnsafe();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
         key = ByteBufferUtil.string(rows.get(0).key.getKey());
@@ -409,7 +409,7 @@ public class ColumnFamilyStoreTest
         // verify that row and delete w/ older timestamp does nothing
         rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
         rm.delete(CF_INDEX1, 1);
-        rm.apply();
+        rm.applyUnsafe();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
         key = ByteBufferUtil.string(rows.get(0).key.getKey());
@@ -418,7 +418,7 @@ public class ColumnFamilyStoreTest
         // similarly, column delete w/ older timestamp should do nothing
         rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
         rm.delete(CF_INDEX1, cellname("birthdate"), 1);
-        rm.apply();
+        rm.applyUnsafe();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
         key = ByteBufferUtil.string(rows.get(0).key.getKey());
@@ -427,14 +427,14 @@ public class ColumnFamilyStoreTest
         // delete the entire row (w/ newer timestamp this time)
         rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
         rm.delete(CF_INDEX1, 3);
-        rm.apply();
+        rm.applyUnsafe();
         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 Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
         rm.add(CF_INDEX1, cellname("birthdate"), ByteBufferUtil.bytes(1L), 3);
-        rm.apply();
+        rm.applyUnsafe();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.isEmpty() : StringUtils.join(rows, ",");
 
@@ -442,7 +442,7 @@ public class ColumnFamilyStoreTest
         rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
         rm.add(CF_INDEX1, cellname("birthdate"), ByteBufferUtil.bytes(1L), 1);
         rm.delete(CF_INDEX1, 2);
-        rm.apply();
+        rm.applyUnsafe();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.isEmpty() : StringUtils.join(rows, ",");
 
@@ -450,7 +450,7 @@ public class ColumnFamilyStoreTest
         rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
         rm.delete(CF_INDEX1, 3);
         rm.add(CF_INDEX1, cellname("birthdate"), ByteBufferUtil.bytes(1L), 4);
-        rm.apply();
+        rm.applyUnsafe();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
         key = ByteBufferUtil.string(rows.get(0).key.getKey());
@@ -468,10 +468,10 @@ public class ColumnFamilyStoreTest
         Mutation rm;
         rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("k1"));
         rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(1L), 1);
-        rm.apply();
+        rm.applyUnsafe();
         rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("k1"));
         rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(2L), 2);
-        rm.apply();
+        rm.applyUnsafe();
 
         IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(1L));
         List<IndexExpression> clause = Arrays.asList(expr);
@@ -489,7 +489,7 @@ public class ColumnFamilyStoreTest
         // update the birthdate value with an OLDER timestamp, and test that the index ignores this
         rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("k1"));
         rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(3L), 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         rows = keyspace.getColumnFamilyStore(CF_INDEX1).search(range, clause, filter, 100);
         key = ByteBufferUtil.string(rows.get(0).key.getKey());
@@ -507,7 +507,7 @@ public class ColumnFamilyStoreTest
         Mutation rm;
         rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("k100"));
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(100L), 1, 1000);
-        rm.apply();
+        rm.applyUnsafe();
 
         IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(100L));
         List<IndexExpression> clause = Arrays.asList(expr);
@@ -522,7 +522,7 @@ public class ColumnFamilyStoreTest
         // now overwrite with the same name/value/ttl, but the local expiry time will be different
         rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("k100"));
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(100L), 1, 1000);
-        rm.apply();
+        rm.applyUnsafe();
 
         rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
         assertEquals(1, rows.size());
@@ -530,7 +530,7 @@ public class ColumnFamilyStoreTest
         // check that modifying the indexed value using the same timestamp behaves as expected
         rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("k101"));
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(101L), 1, 1000);
-        rm.apply();
+        rm.applyUnsafe();
 
         expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(101L));
         clause = Arrays.asList(expr);
@@ -540,7 +540,7 @@ public class ColumnFamilyStoreTest
         TimeUnit.SECONDS.sleep(1);
         rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("k101"));
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(102L), 1, 1000);
-        rm.apply();
+        rm.applyUnsafe();
         // search for the old value
         rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
         assertEquals(0, rows.size());
@@ -570,7 +570,7 @@ public class ColumnFamilyStoreTest
         Mutation rm;
         rm = new Mutation(keySpace, rowKey);
         rm.add(cfName, colName, val1, 0);
-        rm.apply();
+        rm.applyUnsafe();
         IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, val1);
         List<IndexExpression> clause = Arrays.asList(expr);
         IDiskAtomFilter filter = new IdentityQueryFilter();
@@ -638,7 +638,7 @@ public class ColumnFamilyStoreTest
         Mutation rm;
         rm = new Mutation(keySpace, rowKey);
         rm.add(cfName, compositeName, val1, 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         // test that the index query fetches this version
         IndexExpression expr = new IndexExpression(colName, IndexExpression.Operator.EQ, val1);
@@ -710,12 +710,12 @@ public class ColumnFamilyStoreTest
         Mutation rm;
         rm = new Mutation(keySpace, rowKey);
         rm.add(cfName, compositeName, val1, 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         // Now delete the value and flush too.
         rm = new Mutation(keySpace, rowKey);
         rm.delete(cfName, 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         // We want the data to be gcable, but even if gcGrace == 0, we still need to wait 1 second
         // since we won't gc on a tie.
@@ -745,22 +745,22 @@ public class ColumnFamilyStoreTest
         rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("kk1"));
         rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(1L), 0);
         rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(1L), 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("kk2"));
         rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(2L), 0);
         rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(1L), 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("kk3"));
         rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(2L), 0);
         rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(1L), 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("kk4"));
         rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(2L), 0);
         rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(1L), 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         // basic single-expression query
         IndexExpression expr1 = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(1L));
@@ -784,7 +784,7 @@ public class ColumnFamilyStoreTest
         Mutation rm;
         rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k1"));
         rm.add(CF_INDEX2, cellname("birthdate"), ByteBufferUtil.bytes(1L), 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         ColumnDefinition old = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("birthdate"));
         ColumnDefinition cd = ColumnDefinition.regularDef(cfs.metadata, old.name.bytes, old.type, null).setIndex("birthdate_index", IndexType.KEYS, null);
@@ -828,13 +828,13 @@ public class ColumnFamilyStoreTest
         Mutation rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k1"));
         CellName column1 = cellname(ByteBuffer.wrap(new byte[]{1}));
         rm.add(cfname, column1, ByteBufferUtil.bytes("data1"), 1);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k1"));
         CellName column2 = cellname(ByteBuffer.wrap(new byte[]{0, 0, 1}));
         rm.add(cfname, column2, ByteBufferUtil.bytes("data2"), 2);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         // fetch by the first column name; we should get the second version of the column value
@@ -908,7 +908,7 @@ public class ColumnFamilyStoreTest
         // delete
         Mutation rm = new Mutation(keyspace.getName(), key.getKey());
         rm.deleteRange(cfName, SuperColumns.startOf(scfName), SuperColumns.endOf(scfName), 2);
-        rm.apply();
+        rm.applyUnsafe();
 
         // verify delete.
         assertRowAndColCount(1, 0, false, cfs.getRangeSlice(Util.range("f", "g"), null, ThriftValidation.asIFilter(sp, cfs.metadata, scfName), 100));
@@ -961,7 +961,7 @@ public class ColumnFamilyStoreTest
         for (Cell col : cols)
             cf.addColumn(col.withUpdatedName(CellNames.compositeDense(scfName, col.name().toByteBuffer())));
         Mutation rm = new Mutation(cfs.keyspace.getName(), key.getKey(), cf);
-        rm.apply();
+        rm.applyUnsafe();
     }
 
     private static void putColsStandard(ColumnFamilyStore cfs, DecoratedKey key, Cell... cols) throws Throwable
@@ -970,7 +970,7 @@ public class ColumnFamilyStoreTest
         for (Cell col : cols)
             cf.addColumn(col);
         Mutation rm = new Mutation(cfs.keyspace.getName(), key.getKey(), cf);
-        rm.apply();
+        rm.applyUnsafe();
     }
 
     @Test
@@ -1003,7 +1003,7 @@ public class ColumnFamilyStoreTest
         // delete (from sstable and memtable)
         Mutation rm = new Mutation(keyspace.getName(), key.getKey());
         rm.delete(cfs.name, 2);
-        rm.apply();
+        rm.applyUnsafe();
 
         // verify delete
         assertRowAndColCount(1, 0, true, cfs.getRangeSlice(Util.range("f", "g"), null, ThriftValidation.asIFilter(sp, cfs.metadata, null), 100));
@@ -1458,7 +1458,7 @@ public class ColumnFamilyStoreTest
             ByteBuffer key = ByteBufferUtil.bytes(String.valueOf("k" + i));
             Mutation rm = new Mutation(KEYSPACE1, key);
             rm.add(CF_INDEX1, cellname("birthdate"), LongType.instance.decompose(1L), System.currentTimeMillis());
-            rm.apply();
+            rm.applyUnsafe();
         }
 
         store.forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/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 e0a78a9..c49cc7a 100644
--- a/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
+++ b/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
@@ -83,7 +83,7 @@ public class HintedHandOffTest
                                               System.currentTimeMillis(),
                                               HintedHandOffManager.calculateHintTTL(rm),
                                               UUID.randomUUID())
-                                     .apply();
+                                     .applyUnsafe();
 
         // flush data to disk
         hintStore.forceBlockingFlush();
@@ -126,7 +126,7 @@ public class HintedHandOffTest
                                               System.currentTimeMillis(),
                                               HintedHandOffManager.calculateHintTTL(rm),
                                               UUID.randomUUID())
-                                     .apply();
+                                     .applyUnsafe();
 
         assert getNoOfHints() == 1;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/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 0d3fa63..ad3a6bc 100644
--- a/test/unit/org/apache/cassandra/db/KeyCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
@@ -139,10 +139,10 @@ public class KeyCacheTest
         // inserts
         rm = new Mutation(KEYSPACE1, key1.getKey());
         rm.add(COLUMN_FAMILY1, Util.cellname("1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
-        rm.apply();
+        rm.applyUnsafe();
         rm = new Mutation(KEYSPACE1, key2.getKey());
         rm.add(COLUMN_FAMILY1, Util.cellname("2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         // to make sure we have SSTable
         cfs.forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/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 23d8abc..883999f 100644
--- a/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
@@ -103,7 +103,7 @@ public class KeyCollisionTest
         Mutation rm;
         rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes(key));
         rm.add(CF, Util.cellname("column"), ByteBufferUtil.bytes("asdf"), 0);
-        rm.apply();
+        rm.applyUnsafe();
     }
 
     public static class LengthPartitioner extends AbstractPartitioner<BigIntegerToken>

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/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 0bd428d..27a4e20 100644
--- a/test/unit/org/apache/cassandra/db/KeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyspaceTest.java
@@ -101,7 +101,7 @@ public class KeyspaceTest
         ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE2, "Standard3");
         cf.addColumn(column("col1","val1", 1L));
         Mutation rm = new Mutation(KEYSPACE2, TEST_KEY.getKey(), cf);
-        rm.apply();
+        rm.applyUnsafe();
 
         Runnable verify = new WrappedRunnable()
         {
@@ -133,7 +133,7 @@ public class KeyspaceTest
         cf.addColumn(column("col2","val2", 1L));
         cf.addColumn(column("col3","val3", 1L));
         Mutation rm = new Mutation(KEYSPACE1, TEST_KEY.getKey(), cf);
-        rm.apply();
+        rm.applyUnsafe();
 
         Runnable verify = new WrappedRunnable()
         {
@@ -163,7 +163,7 @@ public class KeyspaceTest
         cf.addColumn(column("b", "val2", 1L));
         cf.addColumn(column("c", "val3", 1L));
         Mutation rm = new Mutation(KEYSPACE1, key.getKey(), cf);
-        rm.apply();
+        rm.applyUnsafe();
 
         cf = cfStore.getColumnFamily(key, cellname("b"), cellname("c"), false, 100, System.currentTimeMillis());
         assertEquals(2, cf.getColumnCount());
@@ -182,7 +182,7 @@ public class KeyspaceTest
         ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard2");
         cf.addColumn(column("col1", "val1", 1));
         Mutation rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("row1000"), cf);
-        rm.apply();
+        rm.applyUnsafe();
 
         validateGetSliceNoMatch(keyspace);
         keyspace.getColumnFamilyStore("Standard2").forceBlockingFlush();
@@ -209,7 +209,7 @@ public class KeyspaceTest
         for (int i = 0; i < 300; i++)
             cf.addColumn(column("col" + fmt.format(i), "omg!thisisthevalue!"+i, 1L));
         Mutation rm = new Mutation(KEYSPACE1, ROW.getKey(), cf);
-        rm.apply();
+        rm.applyUnsafe();
 
         Runnable verify = new WrappedRunnable()
         {
@@ -264,7 +264,7 @@ public class KeyspaceTest
             ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "StandardLong1");
             cf.addColumn(new BufferCell(cellname((long)i), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0));
             Mutation rm = new Mutation(KEYSPACE1, ROW.getKey(), cf);
-            rm.apply();
+            rm.applyUnsafe();
         }
 
         cfs.forceBlockingFlush();
@@ -274,7 +274,7 @@ public class KeyspaceTest
             ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "StandardLong1");
             cf.addColumn(new BufferCell(cellname((long)i), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0));
             Mutation rm = new Mutation(KEYSPACE1, ROW.getKey(), cf);
-            rm.apply();
+            rm.applyUnsafe();
 
             cf = cfs.getColumnFamily(ROW, Composites.EMPTY, Composites.EMPTY, true, 1, System.currentTimeMillis());
             assertEquals(1, Iterables.size(cf.getColumnNames()));
@@ -312,11 +312,11 @@ public class KeyspaceTest
         cf.addColumn(column("col7", "val7", 1L));
         cf.addColumn(column("col9", "val9", 1L));
         Mutation rm = new Mutation(KEYSPACE1, ROW.getKey(), cf);
-        rm.apply();
+        rm.applyUnsafe();
 
         rm = new Mutation(KEYSPACE1, ROW.getKey());
         rm.delete("Standard1", cellname("col4"), 2L);
-        rm.apply();
+        rm.applyUnsafe();
 
         Runnable verify = new WrappedRunnable()
         {
@@ -364,7 +364,7 @@ public class KeyspaceTest
         cf.addColumn(expiringColumn("col2", "val2", 1L, 60)); // long enough not to be tombstoned
         cf.addColumn(column("col3", "val3", 1L));
         Mutation rm = new Mutation(KEYSPACE1, ROW.getKey(), cf);
-        rm.apply();
+        rm.applyUnsafe();
 
         Runnable verify = new WrappedRunnable()
         {
@@ -401,7 +401,7 @@ public class KeyspaceTest
         cf.addColumn(column("col5", "val5", 1L));
         cf.addColumn(column("col6", "val6", 1L));
         Mutation rm = new Mutation(KEYSPACE1, ROW.getKey(), cf);
-        rm.apply();
+        rm.applyUnsafe();
         cfStore.forceBlockingFlush();
 
         cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
@@ -409,7 +409,7 @@ public class KeyspaceTest
         cf.addColumn(column("col2", "valx", 2L));
         cf.addColumn(column("col3", "valx", 2L));
         rm = new Mutation(KEYSPACE1, ROW.getKey(), cf);
-        rm.apply();
+        rm.applyUnsafe();
 
         Runnable verify = new WrappedRunnable()
         {
@@ -445,7 +445,7 @@ public class KeyspaceTest
         for (int i = 1000; i < 2000; i++)
             cf.addColumn(column("col" + i, ("v" + i), 1L));
         Mutation rm = new Mutation(KEYSPACE1, key.getKey(), cf);
-        rm.apply();
+        rm.applyUnsafe();
         cfStore.forceBlockingFlush();
 
         validateSliceLarge(cfStore);
@@ -478,7 +478,7 @@ public class KeyspaceTest
                 cf.addColumn(column("col" + i, ("v" + i), i));
             }
             Mutation rm = new Mutation(KEYSPACE1, key.getKey(), cf);
-            rm.apply();
+            rm.applyUnsafe();
             cfStore.forceBlockingFlush();
         }
         cfStore.metric.sstablesPerReadHistogram.cf.clear();
@@ -544,7 +544,7 @@ public class KeyspaceTest
                 Mutation rm = new Mutation(KEYSPACE1, key.getKey());
                 CellName colName = type.makeCellName(ByteBufferUtil.bytes("a" + i), ByteBufferUtil.bytes(j*10 + i));
                 rm.add("StandardComposite2", colName, ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
-                rm.apply();
+                rm.applyUnsafe();
             }
             cfs.forceBlockingFlush();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/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 536ba1c..fd04b76 100644
--- a/test/unit/org/apache/cassandra/db/MultitableTest.java
+++ b/test/unit/org/apache/cassandra/db/MultitableTest.java
@@ -64,12 +64,12 @@ public class MultitableTest
         cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf.addColumn(column("col1", "val1", 1L));
         rm = new Mutation(KEYSPACE1, dk.getKey(), cf);
-        rm.apply();
+        rm.applyUnsafe();
 
         cf = ArrayBackedSortedColumns.factory.create(KEYSPACE2, "Standard1");
         cf.addColumn(column("col2", "val2", 1L));
         rm = new Mutation(KEYSPACE2, dk.getKey(), cf);
-        rm.apply();
+        rm.applyUnsafe();
 
         keyspace1.getColumnFamilyStore("Standard1").forceBlockingFlush();
         keyspace2.getColumnFamilyStore("Standard1").forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/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 4c8c8b4..d1d3c46 100644
--- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
+++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
@@ -89,25 +89,25 @@ public class RangeTombstoneTest
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         for (int i = 0; i < 40; i += 2)
             add(rm, i, 0);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         cf = rm.addOrGet(CFNAME);
         delete(cf, 10, 22, 1);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         for (int i = 1; i < 40; i += 2)
             add(rm, i, 2);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         cf = rm.addOrGet(CFNAME);
         delete(cf, 19, 27, 3);
-        rm.apply();
+        rm.applyUnsafe();
         // We don't flush to test with both a range tomsbtone in memtable and in sstable
 
         // Queries by name
@@ -149,17 +149,17 @@ public class RangeTombstoneTest
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         for (int i = 0; i < 40; i += 2)
             add(rm, i, 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         cf = rm.addOrGet(CFNAME);
         delete(cf, 5, 10, 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         cf = rm.addOrGet(CFNAME);
         delete(cf, 15, 20, 2);
-        rm.apply();
+        rm.applyUnsafe();
 
         cf = cfs.getColumnFamily(QueryFilter.getSliceFilter(dk(key), CFNAME, b(11), b(14), false, Integer.MAX_VALUE, System.currentTimeMillis()));
         Collection<RangeTombstone> rt = rangeTombstones(cf);
@@ -250,25 +250,25 @@ public class RangeTombstoneTest
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         for (int i = 0; i < 20; i++)
             add(rm, i, 0);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         cf = rm.addOrGet(CFNAME);
         delete(cf, 5, 15, 1);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         cf = rm.addOrGet(CFNAME);
         delete(cf, 5, 10, 1);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         cf = rm.addOrGet(CFNAME);
         delete(cf, 5, 8, 2);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(key), CFNAME, System.currentTimeMillis()));
@@ -305,14 +305,14 @@ public class RangeTombstoneTest
 
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         add(rm, 2, 0);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         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);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         // Get the last value of the row
@@ -344,14 +344,14 @@ public class RangeTombstoneTest
         Mutation rm = new Mutation(KSNAME, key);
         for (int i = 0; i < 10; i += 2)
             add(rm, i, 0);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         rm = new Mutation(KSNAME, key);
         ColumnFamily cf = rm.addOrGet(CFNAME);
         for (int i = 0; i < 10; i += 2)
             delete(cf, 0, 7, 0);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         // there should be 2 sstables
@@ -401,18 +401,18 @@ public class RangeTombstoneTest
 
         Mutation rm = new Mutation(KSNAME, key);
         add(rm, 1, 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         // add a RT which hides the column we just inserted
         rm = new Mutation(KSNAME, key);
         ColumnFamily cf = rm.addOrGet(CFNAME);
         delete(cf, 0, 1, 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         // now re-insert that column
         rm = new Mutation(KSNAME, key);
         add(rm, 1, 2);
-        rm.apply();
+        rm.applyUnsafe();
 
         cfs.forceBlockingFlush();
 
@@ -445,14 +445,14 @@ public class RangeTombstoneTest
         Mutation rm = new Mutation(KSNAME, key);
         for (int i = 0; i < 10; i++)
             add(rm, i, 0);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         rm = new Mutation(KSNAME, key);
         ColumnFamily cf = rm.addOrGet(CFNAME);
         for (int i = 0; i < 10; i += 2)
             delete(cf, 0, 7, 0);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         // We should have indexed 1 column

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/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 23caaf8..d0f7cff 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
@@ -65,7 +65,7 @@ public class RecoveryManagerTruncateTest
         cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
 		cf.addColumn(column("col1", "val1", 1L));
         rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("keymulti"), cf);
-		rm.apply();
+		rm.applyUnsafe();
 
 		// Make sure data was written
 		assertNotNull(getFromTable(keyspace, "Standard1", "keymulti", "col1"));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/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 ac941b5..1edb964 100644
--- a/test/unit/org/apache/cassandra/db/RemoveCellTest.java
+++ b/test/unit/org/apache/cassandra/db/RemoveCellTest.java
@@ -58,13 +58,13 @@ public class RemoveCellTest
         // add data
         rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
-        rm.apply();
+        rm.applyUnsafe();
         store.forceBlockingFlush();
 
         // remove
         rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.delete("Standard1", Util.cellname("Column1"), 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         ColumnFamily retrieved = store.getColumnFamily(Util.namesQueryFilter(store, dk, "Column1"));
         assertFalse(retrieved.getColumn(Util.cellname("Column1")).isLive());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/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 c8bab47..fec8711 100644
--- a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
+++ b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
@@ -58,12 +58,12 @@ public class RemoveColumnFamilyTest
         // add data
         rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         // remove
         rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.delete("Standard1", 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, "Standard1", System.currentTimeMillis()));
         assert retrieved.isMarkedForDelete();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/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 830eba5..72827d0 100644
--- a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
+++ b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
@@ -59,13 +59,13 @@ public class RemoveColumnFamilyWithFlush1Test
         rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.add("Standard1", Util.cellname("Column2"), ByteBufferUtil.bytes("asdf"), 0);
-        rm.apply();
+        rm.applyUnsafe();
         store.forceBlockingFlush();
 
         // remove
         rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.delete("Standard1", 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, "Standard1", System.currentTimeMillis()));
         assert retrieved.isMarkedForDelete();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/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 d2ba51f..ef7f7f2 100644
--- a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
+++ b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
@@ -58,11 +58,11 @@ public class RemoveColumnFamilyWithFlush2Test
         // add data
         rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
-        rm.apply();
+        rm.applyUnsafe();
         // remove
         rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.delete("Standard1", 1);
-        rm.apply();
+        rm.applyUnsafe();
         store.forceBlockingFlush();
 
         ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, "Standard1", System.currentTimeMillis()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/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 db27bef..3fa5c2f 100644
--- a/test/unit/org/apache/cassandra/db/RemoveSubCellTest.java
+++ b/test/unit/org/apache/cassandra/db/RemoveSubCellTest.java
@@ -67,14 +67,14 @@ public class RemoveSubCellTest
         // add data
         rm = new Mutation(KEYSPACE1, dk.getKey());
         Util.addMutation(rm, "Super1", "SC1", 1, "asdf", 0);
-        rm.apply();
+        rm.applyUnsafe();
         store.forceBlockingFlush();
 
         CellName cname = CellNames.compositeDense(ByteBufferUtil.bytes("SC1"), getBytes(1L));
         // remove
         rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.delete("Super1", cname, 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, "Super1", System.currentTimeMillis()));
         assertFalse(retrieved.getColumn(cname).isLive());
@@ -92,7 +92,7 @@ public class RemoveSubCellTest
         // add data
         rm = new Mutation(KEYSPACE1, dk.getKey());
         Util.addMutation(rm, "Super1", "SC1", 1, "asdf", 0);
-        rm.apply();
+        rm.applyUnsafe();
         store.forceBlockingFlush();
 
         // remove the SC
@@ -100,7 +100,7 @@ public class RemoveSubCellTest
         CellName cname = CellNames.compositeDense(scName, getBytes(1L));
         rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.deleteRange("Super1", SuperColumns.startOf(scName), SuperColumns.endOf(scName), 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         // Mark current time and make sure the next insert happens at least
         // one second after the previous one (since gc resolution is the second)
@@ -110,7 +110,7 @@ public class RemoveSubCellTest
         // remove the column itself
         rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.delete("Super1", cname, 2);
-        rm.apply();
+        rm.applyUnsafe();
 
         ColumnFamily retrieved = store.getColumnFamily(filter);
         assertFalse(retrieved.getColumn(cname).isLive());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/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 10d8903..ee7bf1a 100644
--- a/test/unit/org/apache/cassandra/db/RowIterationTest.java
+++ b/test/unit/org/apache/cassandra/db/RowIterationTest.java
@@ -67,7 +67,7 @@ public class RowIterationTest
             DecoratedKey key = Util.dk(String.valueOf(i));
             Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             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();
+            rm.applyUnsafe();
             inserted.add(key);
         }
         store.forceBlockingFlush();
@@ -86,7 +86,7 @@ public class RowIterationTest
         Mutation rm = new Mutation(KEYSPACE1, key.getKey());
         rm.delete(CF_NAME, 0);
         rm.add(CF_NAME, Util.cellname("c"), ByteBufferUtil.bytes("values"), 0L);
-        rm.apply();
+        rm.applyUnsafe();
         store.forceBlockingFlush();
 
         // Delete row in second sstable with higher timestamp
@@ -95,7 +95,7 @@ public class RowIterationTest
         rm.add(CF_NAME, Util.cellname("c"), ByteBufferUtil.bytes("values"), 1L);
         DeletionInfo delInfo2 = rm.getColumnFamilies().iterator().next().deletionInfo();
         assert delInfo2.getTopLevelDeletion().markedForDeleteAt == 1L;
-        rm.apply();
+        rm.applyUnsafe();
         store.forceBlockingFlush();
 
         ColumnFamily cf = Util.getRangeSlice(store).get(0).cf;
@@ -113,7 +113,7 @@ public class RowIterationTest
         // Delete a row in first sstable
         Mutation rm = new Mutation(KEYSPACE1, key.getKey());
         rm.delete(CF_NAME, 0);
-        rm.apply();
+        rm.applyUnsafe();
         store.forceBlockingFlush();
 
         ColumnFamily cf = Util.getRangeSlice(store).get(0).cf;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/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 77890fb..1d9fb10 100644
--- a/test/unit/org/apache/cassandra/db/TimeSortTest.java
+++ b/test/unit/org/apache/cassandra/db/TimeSortTest.java
@@ -63,12 +63,12 @@ public class TimeSortTest
 
         rm = new Mutation(KEYSPACE1, key.getKey());
         rm.add(CF_STANDARD1, cellname(100), ByteBufferUtil.bytes("a"), 100);
-        rm.apply();
+        rm.applyUnsafe();
         cfStore.forceBlockingFlush();
 
         rm = new Mutation(KEYSPACE1, key.getKey());
         rm.add(CF_STANDARD1, cellname(0), ByteBufferUtil.bytes("b"), 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         ColumnFamily cf = cfStore.getColumnFamily(key, cellname(10), Composites.EMPTY, false, 1000, System.currentTimeMillis());
         Collection<Cell> cells = cf.getSortedColumns();
@@ -88,7 +88,7 @@ public class TimeSortTest
             {
                 rm.add(CF_STANDARD1, cellname(j * 2), ByteBufferUtil.bytes("a"), j * 2);
             }
-            rm.apply();
+            rm.applyUnsafe();
         }
 
         validateTimeSort(keyspace);
@@ -103,12 +103,12 @@ public class TimeSortTest
         {
             rm.add(CF_STANDARD1, cellname(j * 2 + 1), ByteBufferUtil.bytes("b"), j * 2 + 1);
         }
-        rm.apply();
+        rm.applyUnsafe();
         // and some overwrites
         rm = new Mutation(KEYSPACE1, key.getKey());
         rm.add(CF_STANDARD1, cellname(0), ByteBufferUtil.bytes("c"), 100);
         rm.add(CF_STANDARD1, cellname(10), ByteBufferUtil.bytes("c"), 100);
-        rm.apply();
+        rm.applyUnsafe();
 
         // verify
         ColumnFamily cf = cfStore.getColumnFamily(key, cellname(0), Composites.EMPTY, false, 1000, System.currentTimeMillis());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
index ac628ee..e47f0e9 100644
--- a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
@@ -77,7 +77,7 @@ public class AntiCompactionTest
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
                        timestamp,
                        0);
-            rm.apply();
+            rm.applyUnsafe();
         }
         store.forceBlockingFlush();
         Collection<SSTableReader> sstables = store.getUnrepairedSSTables();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
index 8beea7f..85f5330 100644
--- a/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
@@ -107,7 +107,7 @@ public class BlacklistingCompactionsTest
                 long timestamp = j * ROWS_PER_SSTABLE + i;
                 rm.add("Standard1", cellname(i / 2), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp);
                 maxTimestampExpected = Math.max(timestamp, maxTimestampExpected);
-                rm.apply();
+                rm.applyUnsafe();
                 inserted.add(key);
             }
             cfs.forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
index e8656c4..00923b1 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
@@ -107,7 +107,7 @@ public class CompactionsPurgeTest
         {
             rm.add(cfName, cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
         }
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         // deletes
@@ -115,14 +115,14 @@ public class CompactionsPurgeTest
         {
             rm = new Mutation(KEYSPACE1, key.getKey());
             rm.delete(cfName, cellname(String.valueOf(i)), 1);
-            rm.apply();
+            rm.applyUnsafe();
         }
         cfs.forceBlockingFlush();
 
         // resurrect one column
         rm = new Mutation(KEYSPACE1, key.getKey());
         rm.add(cfName, cellname(String.valueOf(5)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 2);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         // major compact and test that all columns but the resurrected one is completely gone
@@ -152,7 +152,7 @@ public class CompactionsPurgeTest
             {
                 rm.add(cfName, cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
             }
-            rm.apply();
+            rm.applyUnsafe();
             cfs.forceBlockingFlush();
 
             // deletes
@@ -160,7 +160,7 @@ public class CompactionsPurgeTest
             {
                 rm = new Mutation(KEYSPACE2, key.getKey());
                 rm.delete(cfName, cellname(String.valueOf(i)), 1);
-                rm.apply();
+                rm.applyUnsafe();
             }
             cfs.forceBlockingFlush();
         }
@@ -174,7 +174,7 @@ public class CompactionsPurgeTest
         Collection<SSTableReader> sstablesIncomplete = cfs.getSSTables();
         rm = new Mutation(KEYSPACE2, key1.getKey());
         rm.add(cfName, cellname(String.valueOf(5)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 2);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
         cfs.getCompactionStrategy().getUserDefinedTask(sstablesIncomplete, Integer.MAX_VALUE).execute(null);
 
@@ -207,19 +207,19 @@ public class CompactionsPurgeTest
         rm = new Mutation(KEYSPACE2, key3.getKey());
         rm.add(cfName, cellname("c1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 8);
         rm.add(cfName, cellname("c2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 8);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
         // delete c1
         rm = new Mutation(KEYSPACE2, key3.getKey());
         rm.delete(cfName, cellname("c1"), 10);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
         Collection<SSTableReader> sstablesIncomplete = cfs.getSSTables();
 
         // delete c2 so we have new delete in a diffrent SSTable
         rm = new Mutation(KEYSPACE2, key3.getKey());
         rm.delete(cfName, cellname("c2"), 9);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         // compact the sstables with the c1/c2 data and the c1 tombstone
@@ -250,14 +250,14 @@ public class CompactionsPurgeTest
         {
             rm.add(cfName, cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
         }
-        rm.apply();
+        rm.applyUnsafe();
 
         // deletes
         for (int i = 0; i < 5; i++)
         {
             rm = new Mutation(KEYSPACE1, key.getKey());
             rm.delete(cfName, cellname(String.valueOf(i)), 1);
-            rm.apply();
+            rm.applyUnsafe();
         }
         cfs.forceBlockingFlush();
         assertEquals(String.valueOf(cfs.getSSTables()), 1, cfs.getSSTables().size()); // inserts & deletes were in the same memtable -> only deletes in sstable
@@ -288,7 +288,7 @@ public class CompactionsPurgeTest
         {
             rm.add(cfName, cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
         }
-        rm.apply();
+        rm.applyUnsafe();
 
         // move the key up in row cache
         cfs.getColumnFamily(QueryFilter.getIdentityFilter(key, cfName, System.currentTimeMillis()));
@@ -296,7 +296,7 @@ public class CompactionsPurgeTest
         // deletes row
         rm = new Mutation(keyspaceName, key.getKey());
         rm.delete(cfName, 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         // flush and major compact
         cfs.forceBlockingFlush();
@@ -308,7 +308,7 @@ public class CompactionsPurgeTest
         {
             rm.add(cfName, cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
         }
-        rm.apply();
+        rm.applyUnsafe();
 
         // Check that the second insert did went in
         ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(key, cfName, System.currentTimeMillis()));
@@ -334,12 +334,12 @@ public class CompactionsPurgeTest
         rm = new Mutation(keyspaceName, key.getKey());
         for (int i = 0; i < 10; i++)
             rm.add(cfName, cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, i);
-        rm.apply();
+        rm.applyUnsafe();
 
         // deletes row with timestamp such that not all columns are deleted
         rm = new Mutation(keyspaceName, key.getKey());
         rm.delete(cfName, 4);
-        rm.apply();
+        rm.applyUnsafe();
         ColumnFamily cf = cfs.getColumnFamily(filter);
         assertTrue(cf.isMarkedForDelete());
 
@@ -352,7 +352,7 @@ public class CompactionsPurgeTest
         rm = new Mutation(keyspaceName, key.getKey());
         for (int i = 0; i < 5; i++)
             rm.add(cfName, cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, i);
-        rm.apply();
+        rm.applyUnsafe();
 
         // Check that the second insert went in
         cf = cfs.getColumnFamily(filter);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
index e784051..5be732e 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
@@ -140,7 +140,7 @@ public class CompactionsTest
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
                        timestamp,
                        j > 0 ? ttl : 0); // let first column never expire, since deleting all columns does not produce sstable
-            rm.apply();
+            rm.applyUnsafe();
         }
         return timestamp;
     }
@@ -178,13 +178,13 @@ public class CompactionsTest
         rm.add("Super1", Util.cellname(scName, ByteBufferUtil.bytes(0)),
                ByteBufferUtil.EMPTY_BYTE_BUFFER,
                FBUtilities.timestampMicros());
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         // shadow the subcolumn with a supercolumn tombstone
         rm = new Mutation(KEYSPACE1, key.getKey());
         rm.deleteRange("Super1", SuperColumns.startOf(scName), SuperColumns.endOf(scName), FBUtilities.timestampMicros());
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         CompactionManager.instance.performMaximal(cfs);
@@ -296,7 +296,7 @@ public class CompactionsTest
             DecoratedKey key = Util.dk(String.valueOf(i));
             Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             rm.add("Standard2", Util.cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, i);
-            rm.apply();
+            rm.applyUnsafe();
 
             if (i % 2 == 0)
                 cfs.forceBlockingFlush();
@@ -311,7 +311,7 @@ public class CompactionsTest
             DecoratedKey key = Util.dk(String.valueOf(i));
             Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             rm.add("Standard2", Util.cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, i);
-            rm.apply();
+            rm.applyUnsafe();
         }
         cfs.forceBlockingFlush();
         SSTableReader tmpSSTable = null;
@@ -358,7 +358,7 @@ public class CompactionsTest
             rm.add(cfname, Util.cellname("col"),
                    ByteBufferUtil.EMPTY_BYTE_BUFFER,
                    System.currentTimeMillis());
-            rm.apply();
+            rm.applyUnsafe();
         }
         cfs.forceBlockingFlush();
         Collection<SSTableReader> sstables = cfs.getSSTables();
@@ -504,7 +504,7 @@ public class CompactionsTest
         DecoratedKey key = Util.dk(k);
         Mutation rm = new Mutation(KEYSPACE1, key.getKey());
         rm.add(cfname, Util.cellname(ByteBufferUtil.bytes("sc"), ByteBufferUtil.bytes("c")), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         cfs.forceBlockingFlush();
 
@@ -516,7 +516,7 @@ public class CompactionsTest
         // Remove key
         rm = new Mutation(KEYSPACE1, key.getKey());
         rm.delete(cfname, 2);
-        rm.apply();
+        rm.applyUnsafe();
 
         ColumnFamily cf = cfs.getColumnFamily(filter);
         assertTrue("should be empty: " + cf, cf == null || !cf.hasColumns());
@@ -558,7 +558,7 @@ public class CompactionsTest
         DecoratedKey decoratedKey = Util.dk(String.format("%03d", key));
         Mutation rm = new Mutation(KEYSPACE1, decoratedKey.getKey());
         rm.add("CF_STANDARD1", Util.cellname("col"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, 1000);
-        rm.apply();
+        rm.applyUnsafe();
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
index 71bf6e5..5f9b72b 100644
--- a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
@@ -105,7 +105,7 @@ public class LeveledCompactionStrategyTest
             {
                 rm.add(CF_STANDARDDLEVELED, Util.cellname("column" + c), value, 0);
             }
-            rm.apply();
+            rm.applyUnsafe();
             cfs.forceBlockingFlush();
         }
 
@@ -150,7 +150,7 @@ public class LeveledCompactionStrategyTest
             {
                 rm.add(CF_STANDARDDLEVELED, Util.cellname("column" + c), value, 0);
             }
-            rm.apply();
+            rm.applyUnsafe();
             cfs.forceBlockingFlush();
         }
 
@@ -189,7 +189,7 @@ public class LeveledCompactionStrategyTest
             {
                 rm.add(CF_STANDARDDLEVELED, Util.cellname("column" + c), value, 0);
             }
-            rm.apply();
+            rm.applyUnsafe();
             cfs.forceBlockingFlush();
         }
         waitForLeveling(cfs);
@@ -235,7 +235,7 @@ public class LeveledCompactionStrategyTest
             {
                 rm.add(CF_STANDARDDLEVELED, Util.cellname("column" + c), value, 0);
             }
-            rm.apply();
+            rm.applyUnsafe();
             cfs.forceBlockingFlush();
         }
         waitForLeveling(cfs);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java b/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java
index 375dfe8..216c9cf 100644
--- a/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java
@@ -71,7 +71,7 @@ public class OneCompactionTest
             DecoratedKey key = Util.dk(String.valueOf(j));
             Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             rm.add(columnFamilyName, Util.cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
-            rm.apply();
+            rm.applyUnsafe();
             inserted.add(key);
             store.forceBlockingFlush();
             assertEquals(inserted.size(), Util.getRangeSlice(store).size());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java
index ef886e7..56fa7a1 100644
--- a/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java
@@ -180,7 +180,7 @@ public class SizeTieredCompactionStrategyTest
             DecoratedKey key = Util.dk(String.valueOf(r));
             Mutation rm = new Mutation(ksname, key.getKey());
             rm.add(cfname, Util.cellname("column"), value, 0);
-            rm.apply();
+            rm.applyUnsafe();
             cfs.forceBlockingFlush();
         }
         cfs.forceBlockingFlush();
@@ -224,7 +224,7 @@ public class SizeTieredCompactionStrategyTest
             DecoratedKey key = Util.dk(String.valueOf(r));
             Mutation rm = new Mutation(ksname, key.getKey());
             rm.add(cfname, Util.cellname("column"), value, 0);
-            rm.apply();
+            rm.applyUnsafe();
             cfs.forceBlockingFlush();
         }
         cfs.forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java b/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
index 3a2bc4a..f3a693c 100644
--- a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
@@ -71,7 +71,7 @@ public class TTLExpiryTest
                timestamp,
                1);
 
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         rm = new Mutation(KEYSPACE1, Util.dk("ttl").getKey());
@@ -79,21 +79,21 @@ public class TTLExpiryTest
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
                        timestamp,
                        1);
-                rm.apply();
+                rm.applyUnsafe();
         cfs.forceBlockingFlush();
         rm = new Mutation(KEYSPACE1, Util.dk("ttl").getKey());
         rm.add("Standard1", Util.cellname("col3"),
                    ByteBufferUtil.EMPTY_BYTE_BUFFER,
                    timestamp,
                    1);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
         rm = new Mutation(KEYSPACE1, Util.dk("ttl").getKey());
         rm.add("Standard1", Util.cellname("col311"),
                    ByteBufferUtil.EMPTY_BYTE_BUFFER,
                    timestamp,
                    1);
-        rm.apply();
+        rm.applyUnsafe();
 
         cfs.forceBlockingFlush();
         Thread.sleep(2000); // wait for ttl to expire
@@ -119,7 +119,7 @@ public class TTLExpiryTest
                timestamp,
                1);
 
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         rm = new Mutation(KEYSPACE1, Util.dk("ttl").getKey());
@@ -127,21 +127,21 @@ public class TTLExpiryTest
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
                        timestamp,
                        1);
-                rm.apply();
+                rm.applyUnsafe();
         cfs.forceBlockingFlush();
         rm = new Mutation(KEYSPACE1, Util.dk("ttl").getKey());
         rm.add("Standard1", Util.cellname("col3"),
                    ByteBufferUtil.EMPTY_BYTE_BUFFER,
                    timestamp,
                    1);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
         DecoratedKey noTTLKey = Util.dk("nottl");
         rm = new Mutation(KEYSPACE1, noTTLKey.getKey());
         rm.add("Standard1", Util.cellname("col311"),
                    ByteBufferUtil.EMPTY_BYTE_BUFFER,
                    timestamp);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
         Thread.sleep(2000); // wait for ttl to expire
         assertEquals(4, cfs.getSSTables().size());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java b/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
index e312182..3097099 100644
--- a/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
@@ -79,7 +79,7 @@ public class PerRowSecondaryIndexTest
         Mutation rm;
         rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k1"));
         rm.add("Indexed1", Util.cellname("indexed"), ByteBufferUtil.bytes("foo"), 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         ColumnFamily indexedRow = PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_ROW;
         assertNotNull(indexedRow);
@@ -88,7 +88,7 @@ public class PerRowSecondaryIndexTest
         // update the row and verify what was indexed
         rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k1"));
         rm.add("Indexed1", Util.cellname("indexed"), ByteBufferUtil.bytes("bar"), 2);
-        rm.apply();
+        rm.applyUnsafe();
 
         indexedRow = PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_ROW;
         assertNotNull(indexedRow);
@@ -103,7 +103,7 @@ public class PerRowSecondaryIndexTest
         Mutation rm;
         rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k2"));
         rm.delete("Indexed1", Util.cellname("indexed"), 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         ColumnFamily indexedRow = PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_ROW;
         assertNotNull(indexedRow);
@@ -121,7 +121,7 @@ public class PerRowSecondaryIndexTest
         Mutation rm;
         rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k3"));
         rm.delete("Indexed1", 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         ColumnFamily indexedRow = PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_ROW;
         assertNotNull(indexedRow);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java
index 35be43d..3e75c1a 100644
--- a/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java
+++ b/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java
@@ -193,7 +193,7 @@ public class CompositeTypeTest
         addColumn(rm, cname4);
         addColumn(rm, cname2);
         addColumn(rm, cname3);
-        rm.apply();
+        rm.applyUnsafe();
 
         ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk("k"), CF_STANDARDCOMPOSITE, System.currentTimeMillis()));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
index fa10dbb..7056a40 100644
--- a/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
+++ b/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
@@ -196,7 +196,7 @@ public class DynamicCompositeTypeTest
         addColumn(rm, cname4);
         addColumn(rm, cname2);
         addColumn(rm, cname3);
-        rm.apply();
+        rm.applyUnsafe();
 
         ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk("k"), CF_STANDARDDYNCOMPOSITE, System.currentTimeMillis()));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
index 83c9daa..72cc80c 100644
--- a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
@@ -432,7 +432,7 @@ public class IndexSummaryManagerTest
             DecoratedKey key = Util.dk(String.valueOf(row));
             Mutation rm = new Mutation(ksname, key.getKey());
             rm.add(cfname, Util.cellname("column"), value, 0);
-            rm.apply();
+            rm.applyUnsafe();
         }
         cfs.forceBlockingFlush();
 
@@ -492,7 +492,7 @@ public class IndexSummaryManagerTest
                 DecoratedKey key = Util.dk(String.valueOf(row));
                 Mutation rm = new Mutation(ksname, key.getKey());
                 rm.add(cfname, Util.cellname("column"), value, 0);
-                rm.apply();
+                rm.applyUnsafe();
             }
             cfs.forceBlockingFlush();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b453f089/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
index 19d984a..7af40fd 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
@@ -87,14 +87,14 @@ public class SSTableMetadataTest
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
                        timestamp,
                        10 + j);
-            rm.apply();
+            rm.applyUnsafe();
         }
         Mutation rm = new Mutation(KEYSPACE1, Util.dk("longttl").getKey());
         rm.add("Standard1", cellname("col"),
                ByteBufferUtil.EMPTY_BYTE_BUFFER,
                timestamp,
                10000);
-        rm.apply();
+        rm.applyUnsafe();
         store.forceBlockingFlush();
         assertEquals(1, store.getSSTables().size());
         int ttltimestamp = (int)(System.currentTimeMillis()/1000);
@@ -110,7 +110,7 @@ public class SSTableMetadataTest
                ByteBufferUtil.EMPTY_BYTE_BUFFER,
                timestamp,
                20000);
-        rm.apply();
+        rm.applyUnsafe();
         ttltimestamp = (int) (System.currentTimeMillis()/1000);
         store.forceBlockingFlush();
         assertEquals(2, store.getSSTables().size());
@@ -162,7 +162,7 @@ public class SSTableMetadataTest
                    ByteBufferUtil.EMPTY_BYTE_BUFFER,
                    timestamp,
                    1000);
-        rm.apply();
+        rm.applyUnsafe();
         store.forceBlockingFlush();
         assertEquals(1,store.getSSTables().size());
         int ttltimestamp = (int) (System.currentTimeMillis()/1000);
@@ -174,7 +174,7 @@ public class SSTableMetadataTest
         }
         rm = new Mutation(KEYSPACE1, key.getKey());
         rm.delete("Standard2", cellname("todelete"), timestamp + 1);
-        rm.apply();
+        rm.applyUnsafe();
         store.forceBlockingFlush();
         assertEquals(2,store.getSSTables().size());
         boolean foundDelete = false;
@@ -209,7 +209,7 @@ public class SSTableMetadataTest
             {
                 rm.add("Standard3", cellname(j + "col" + i), ByteBufferUtil.EMPTY_BYTE_BUFFER, System.currentTimeMillis());
             }
-            rm.apply();
+            rm.applyUnsafe();
         }
         store.forceBlockingFlush();
         assertEquals(1, store.getSSTables().size());
@@ -224,7 +224,7 @@ public class SSTableMetadataTest
         {
             rm.add("Standard3", cellname(9 + "col" + i), ByteBufferUtil.EMPTY_BYTE_BUFFER, System.currentTimeMillis());
         }
-        rm.apply();
+        rm.applyUnsafe();
 
         store.forceBlockingFlush();
         store.forceMajorCompaction();
@@ -262,7 +262,7 @@ public class SSTableMetadataTest
             Mutation rm = new Mutation(KEYSPACE1, key);
             CellName colName = type.makeCellName(ByteBufferUtil.bytes("a"+(9-i)), ByteBufferUtil.bytes(i));
             rm.add("StandardComposite2", colName, ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
-            rm.apply();
+            rm.applyUnsafe();
         }
         cfs.forceBlockingFlush();
 
@@ -272,7 +272,7 @@ public class SSTableMetadataTest
             Mutation rm = new Mutation(KEYSPACE1, key);
             CellName colName = type.makeCellName(ByteBufferUtil.bytes("b"+(9-i)), ByteBufferUtil.bytes(i));
             rm.add("StandardComposite2", colName, ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
-            rm.apply();
+            rm.applyUnsafe();
         }
         cfs.forceBlockingFlush();
         cfs.forceMajorCompaction();
@@ -298,7 +298,7 @@ public class SSTableMetadataTest
         state.writeRemote(CounterId.fromInt(3), 1L, 1L);
         ColumnFamily cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addColumn(new BufferCounterCell(cellname("col"), state.context, 1L, Long.MIN_VALUE));
-        new Mutation(Util.dk("k").getKey(), cells).apply();
+        new Mutation(Util.dk("k").getKey(), cells).applyUnsafe();
         cfs.forceBlockingFlush();
         assertTrue(cfs.getSSTables().iterator().next().getSSTableMetadata().hasLegacyCounterShards);
         cfs.truncateBlocking();
@@ -309,7 +309,7 @@ public class SSTableMetadataTest
         state.writeRemote(CounterId.fromInt(3), 1L, 1L);
         cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addColumn(new BufferCounterCell(cellname("col"), state.context, 1L, Long.MIN_VALUE));
-        new Mutation(Util.dk("k").getKey(), cells).apply();
+        new Mutation(Util.dk("k").getKey(), cells).applyUnsafe();
         cfs.forceBlockingFlush();
         assertTrue(cfs.getSSTables().iterator().next().getSSTableMetadata().hasLegacyCounterShards);
         cfs.truncateBlocking();
@@ -320,7 +320,7 @@ public class SSTableMetadataTest
         state.writeLocal(CounterId.fromInt(2), 1L, 1L);
         cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addColumn(new BufferCounterCell(cellname("col"), state.context, 1L, Long.MIN_VALUE));
-        new Mutation(Util.dk("k").getKey(), cells).apply();
+        new Mutation(Util.dk("k").getKey(), cells).applyUnsafe();
         cfs.forceBlockingFlush();
         assertTrue(cfs.getSSTables().iterator().next().getSSTableMetadata().hasLegacyCounterShards);
         cfs.truncateBlocking();
@@ -330,7 +330,7 @@ public class SSTableMetadataTest
         state.writeGlobal(CounterId.fromInt(1), 1L, 1L);
         cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addColumn(new BufferCounterCell(cellname("col"), state.context, 1L, Long.MIN_VALUE));
-        new Mutation(Util.dk("k").getKey(), cells).apply();
+        new Mutation(Util.dk("k").getKey(), cells).applyUnsafe();
         cfs.forceBlockingFlush();
         assertFalse(cfs.getSSTables().iterator().next().getSSTableMetadata().hasLegacyCounterShards);
         cfs.truncateBlocking();