You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2016/08/03 15:45:10 UTC

[04/14] cassandra git commit: Fix RTE on mixed-version cluster due to CDC schema changes.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26838063/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
index 8e3961e..7a90dab 100644
--- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
+++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
@@ -266,8 +266,8 @@ public final class SchemaKeyspace
         }
 
         // (+1 to timestamp to make sure we don't get shadowed by the tombstones we just added)
-        makeCreateKeyspaceMutation(system, timestamp + 1).apply();
-        makeCreateKeyspaceMutation(schema, timestamp + 1).apply();
+        makeCreateKeyspaceMutation(system, timestamp + 1).build().apply();
+        makeCreateKeyspaceMutation(schema, timestamp + 1).build().apply();
     }
 
     public static void truncate()
@@ -377,11 +377,6 @@ public final class SchemaKeyspace
         }
     }
 
-    private static ByteBuffer getSchemaKSKey(String ksName)
-    {
-        return AsciiType.instance.fromString(ksName);
-    }
-
     private static boolean isSystemKeyspaceSchemaPartition(DecoratedKey partitionKey)
     {
         return Schema.isSystemKeyspace(UTF8Type.instance.compose(partitionKey.getKey()));
@@ -391,152 +386,152 @@ public final class SchemaKeyspace
      * Schema entities to mutations
      */
 
-    public static Mutation makeCreateKeyspaceMutation(String name, KeyspaceParams params, long timestamp)
+    private static DecoratedKey decorate(CFMetaData metadata, Object value)
     {
-        RowUpdateBuilder adder = new RowUpdateBuilder(Keyspaces, timestamp, name).clustering();
-        return adder.add(KeyspaceParams.Option.DURABLE_WRITES.toString(), params.durableWrites)
-                    .frozenMap(KeyspaceParams.Option.REPLICATION.toString(), params.replication.asMap())
-                    .build();
+        return metadata.decorateKey(((AbstractType)metadata.getKeyValidator()).decompose(value));
     }
 
-    public static Mutation makeCreateKeyspaceMutation(KeyspaceMetadata keyspace, long timestamp)
+    public static Mutation.SimpleBuilder makeCreateKeyspaceMutation(String name, KeyspaceParams params, long timestamp)
     {
-        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
+        Mutation.SimpleBuilder builder = Mutation.simpleBuilder(Keyspaces.ksName, decorate(Keyspaces, name))
+                                                 .timestamp(timestamp);
 
-        keyspace.tables.forEach(table -> addTableToSchemaMutation(table, timestamp, true, mutation));
-        keyspace.views.forEach(view -> addViewToSchemaMutation(view, timestamp, true, mutation));
-        keyspace.types.forEach(type -> addTypeToSchemaMutation(type, timestamp, mutation));
-        keyspace.functions.udfs().forEach(udf -> addFunctionToSchemaMutation(udf, timestamp, mutation));
-        keyspace.functions.udas().forEach(uda -> addAggregateToSchemaMutation(uda, timestamp, mutation));
+        builder.update(Keyspaces)
+               .row()
+               .add(KeyspaceParams.Option.DURABLE_WRITES.toString(), params.durableWrites)
+               .add(KeyspaceParams.Option.REPLICATION.toString(), params.replication.asMap());
 
-        return mutation;
+        return builder;
     }
 
-    public static Mutation makeDropKeyspaceMutation(KeyspaceMetadata keyspace, long timestamp)
+    public static Mutation.SimpleBuilder makeCreateKeyspaceMutation(KeyspaceMetadata keyspace, long timestamp)
     {
-        int nowInSec = FBUtilities.nowInSeconds();
-        Mutation mutation = new Mutation(NAME, Keyspaces.decorateKey(getSchemaKSKey(keyspace.name)));
+        Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 
-        for (CFMetaData schemaTable : ALL_TABLE_METADATA)
-            mutation.add(PartitionUpdate.fullPartitionDelete(schemaTable, mutation.key(), timestamp, nowInSec));
+        keyspace.tables.forEach(table -> addTableToSchemaMutation(table, true, builder));
+        keyspace.views.forEach(view -> addViewToSchemaMutation(view, true, builder));
+        keyspace.types.forEach(type -> addTypeToSchemaMutation(type, builder));
+        keyspace.functions.udfs().forEach(udf -> addFunctionToSchemaMutation(udf, builder));
+        keyspace.functions.udas().forEach(uda -> addAggregateToSchemaMutation(uda, builder));
 
-        return mutation;
+        return builder;
     }
 
-    public static Mutation makeCreateTypeMutation(KeyspaceMetadata keyspace, UserType type, long timestamp)
+    public static Mutation.SimpleBuilder makeDropKeyspaceMutation(KeyspaceMetadata keyspace, long timestamp)
     {
-        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
-        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
-        addTypeToSchemaMutation(type, timestamp, mutation);
-        return mutation;
+        Mutation.SimpleBuilder builder = Mutation.simpleBuilder(NAME, decorate(Keyspaces, keyspace.name))
+                                                 .timestamp(timestamp);
+
+        for (CFMetaData schemaTable : ALL_TABLE_METADATA)
+            builder.update(schemaTable).delete();
+
+        return builder;
     }
 
-    static void addTypeToSchemaMutation(UserType type, long timestamp, Mutation mutation)
+    public static Mutation.SimpleBuilder makeCreateTypeMutation(KeyspaceMetadata keyspace, UserType type, long timestamp)
     {
-        RowUpdateBuilder adder = new RowUpdateBuilder(Types, timestamp, mutation)
-                                 .clustering(type.getNameAsString())
-                                 .frozenList("field_names", type.fieldNames().stream().map(FieldIdentifier::toString).collect(toList()))
-                                 .frozenList("field_types", type.fieldTypes().stream().map(AbstractType::asCQL3Type).map(CQL3Type::toString).collect(toList()));
-
-        adder.build();
+        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+        Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
+        addTypeToSchemaMutation(type, builder);
+        return builder;
     }
 
-    private static String bbToString(ByteBuffer bb)
+    static void addTypeToSchemaMutation(UserType type, Mutation.SimpleBuilder mutation)
     {
-        try
-        {
-            return ByteBufferUtil.string(bb);
-        }
-        catch (CharacterCodingException e)
-        {
-            throw new RuntimeException(e);
-        }
+        mutation.update(Types)
+                .row(type.getNameAsString())
+                .add("field_names", type.fieldNames().stream().map(FieldIdentifier::toString).collect(toList()))
+                .add("field_types", type.fieldTypes().stream().map(AbstractType::asCQL3Type).map(CQL3Type::toString).collect(toList()));
     }
 
-    public static Mutation dropTypeFromSchemaMutation(KeyspaceMetadata keyspace, UserType type, long timestamp)
+    public static Mutation.SimpleBuilder dropTypeFromSchemaMutation(KeyspaceMetadata keyspace, UserType type, long timestamp)
     {
         // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
-        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
-        return RowUpdateBuilder.deleteRow(Types, timestamp, mutation, type.name);
+        Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
+        builder.update(Types).row(type.name).delete();
+        return builder;
     }
 
-    public static Mutation makeCreateTableMutation(KeyspaceMetadata keyspace, CFMetaData table, long timestamp)
+    public static Mutation.SimpleBuilder makeCreateTableMutation(KeyspaceMetadata keyspace, CFMetaData table, long timestamp)
     {
         // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
-        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
-        addTableToSchemaMutation(table, timestamp, true, mutation);
-        return mutation;
+        Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
+        addTableToSchemaMutation(table, true, builder);
+        return builder;
     }
 
-    static void addTableToSchemaMutation(CFMetaData table, long timestamp, boolean withColumnsAndTriggers, Mutation mutation)
+    static void addTableToSchemaMutation(CFMetaData table, boolean withColumnsAndTriggers, Mutation.SimpleBuilder builder)
     {
-        RowUpdateBuilder adder = new RowUpdateBuilder(Tables, timestamp, mutation).clustering(table.cfName);
-
-        addTableParamsToSchemaMutation(table.params, adder);
+        Row.SimpleBuilder rowBuilder = builder.update(Tables)
+                                              .row(table.cfName)
+                                              .add("id", table.cfId)
+                                              .add("flags", CFMetaData.flagsToStrings(table.flags()));
 
-        adder.add("id", table.cfId)
-             .frozenSet("flags", CFMetaData.flagsToStrings(table.flags()))
-             .build();
+        addTableParamsToRowBuilder(table.params, rowBuilder);
 
         if (withColumnsAndTriggers)
         {
             for (ColumnDefinition column : table.allColumns())
-                addColumnToSchemaMutation(table, column, timestamp, mutation);
+                addColumnToSchemaMutation(table, column, builder);
 
             for (CFMetaData.DroppedColumn column : table.getDroppedColumns().values())
-                addDroppedColumnToSchemaMutation(table, column, timestamp, mutation);
+                addDroppedColumnToSchemaMutation(table, column, builder);
 
             for (TriggerMetadata trigger : table.getTriggers())
-                addTriggerToSchemaMutation(table, trigger, timestamp, mutation);
+                addTriggerToSchemaMutation(table, trigger, builder);
 
             for (IndexMetadata index : table.getIndexes())
-                addIndexToSchemaMutation(table, index, timestamp, mutation);
+                addIndexToSchemaMutation(table, index, builder);
         }
     }
 
-    private static void addTableParamsToSchemaMutation(TableParams params, RowUpdateBuilder adder)
+    private static void addTableParamsToRowBuilder(TableParams params, Row.SimpleBuilder builder)
     {
-        adder.add("bloom_filter_fp_chance", params.bloomFilterFpChance)
-             .add("comment", params.comment)
-             .add("dclocal_read_repair_chance", params.dcLocalReadRepairChance)
-             .add("default_time_to_live", params.defaultTimeToLive)
-             .add("gc_grace_seconds", params.gcGraceSeconds)
-             .add("max_index_interval", params.maxIndexInterval)
-             .add("memtable_flush_period_in_ms", params.memtableFlushPeriodInMs)
-             .add("min_index_interval", params.minIndexInterval)
-             .add("read_repair_chance", params.readRepairChance)
-             .add("speculative_retry", params.speculativeRetry.toString())
-             .add("crc_check_chance", params.crcCheckChance)
-             .frozenMap("caching", params.caching.asMap())
-             .frozenMap("compaction", params.compaction.asMap())
-             .frozenMap("compression", params.compression.asMap())
-             .frozenMap("extensions", params.extensions)
-             .add("cdc", params.cdc);
+        builder.add("bloom_filter_fp_chance", params.bloomFilterFpChance)
+               .add("comment", params.comment)
+               .add("dclocal_read_repair_chance", params.dcLocalReadRepairChance)
+               .add("default_time_to_live", params.defaultTimeToLive)
+               .add("gc_grace_seconds", params.gcGraceSeconds)
+               .add("max_index_interval", params.maxIndexInterval)
+               .add("memtable_flush_period_in_ms", params.memtableFlushPeriodInMs)
+               .add("min_index_interval", params.minIndexInterval)
+               .add("read_repair_chance", params.readRepairChance)
+               .add("speculative_retry", params.speculativeRetry.toString())
+               .add("crc_check_chance", params.crcCheckChance)
+               .add("caching", params.caching.asMap())
+               .add("compaction", params.compaction.asMap())
+               .add("compression", params.compression.asMap())
+               .add("extensions", params.extensions);
+
+        // Only add CDC-enabled flag to schema if it's enabled on the node. This is to work around RTE's post-8099 if a 3.8+
+        // node sends table schema to a < 3.8 versioned node with an unknown column.
+        if (DatabaseDescriptor.isCDCEnabled())
+            builder.add("cdc", params.cdc);
     }
 
-    public static Mutation makeUpdateTableMutation(KeyspaceMetadata keyspace,
-                                                   CFMetaData oldTable,
-                                                   CFMetaData newTable,
-                                                   long timestamp)
+    public static Mutation.SimpleBuilder makeUpdateTableMutation(KeyspaceMetadata keyspace,
+                                                                 CFMetaData oldTable,
+                                                                 CFMetaData newTable,
+                                                                 long timestamp)
     {
-        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
+        Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 
-        addTableToSchemaMutation(newTable, timestamp, false, mutation);
+        addTableToSchemaMutation(newTable, false, builder);
 
         MapDifference<ByteBuffer, ColumnDefinition> columnDiff = Maps.difference(oldTable.getColumnMetadata(),
                                                                                  newTable.getColumnMetadata());
 
         // columns that are no longer needed
         for (ColumnDefinition column : columnDiff.entriesOnlyOnLeft().values())
-            dropColumnFromSchemaMutation(oldTable, column, timestamp, mutation);
+            dropColumnFromSchemaMutation(oldTable, column, builder);
 
         // newly added columns
         for (ColumnDefinition column : columnDiff.entriesOnlyOnRight().values())
-            addColumnToSchemaMutation(newTable, column, timestamp, mutation);
+            addColumnToSchemaMutation(newTable, column, builder);
 
         // old columns with updated attributes
         for (ByteBuffer name : columnDiff.entriesDiffering().keySet())
-            addColumnToSchemaMutation(newTable, newTable.getColumnDefinition(name), timestamp, mutation);
+            addColumnToSchemaMutation(newTable, newTable.getColumnDefinition(name), builder);
 
         // dropped columns
         MapDifference<ByteBuffer, CFMetaData.DroppedColumn> droppedColumnDiff =
@@ -544,38 +539,38 @@ public final class SchemaKeyspace
 
         // newly dropped columns
         for (CFMetaData.DroppedColumn column : droppedColumnDiff.entriesOnlyOnRight().values())
-            addDroppedColumnToSchemaMutation(newTable, column, timestamp, mutation);
+            addDroppedColumnToSchemaMutation(newTable, column, builder);
 
         // columns added then dropped again
         for (ByteBuffer name : droppedColumnDiff.entriesDiffering().keySet())
-            addDroppedColumnToSchemaMutation(newTable, newTable.getDroppedColumns().get(name), timestamp, mutation);
+            addDroppedColumnToSchemaMutation(newTable, newTable.getDroppedColumns().get(name), builder);
 
         MapDifference<String, TriggerMetadata> triggerDiff = triggersDiff(oldTable.getTriggers(), newTable.getTriggers());
 
         // dropped triggers
         for (TriggerMetadata trigger : triggerDiff.entriesOnlyOnLeft().values())
-            dropTriggerFromSchemaMutation(oldTable, trigger, timestamp, mutation);
+            dropTriggerFromSchemaMutation(oldTable, trigger, builder);
 
         // newly created triggers
         for (TriggerMetadata trigger : triggerDiff.entriesOnlyOnRight().values())
-            addTriggerToSchemaMutation(newTable, trigger, timestamp, mutation);
+            addTriggerToSchemaMutation(newTable, trigger, builder);
 
         MapDifference<String, IndexMetadata> indexesDiff = indexesDiff(oldTable.getIndexes(),
                                                                        newTable.getIndexes());
 
         // dropped indexes
         for (IndexMetadata index : indexesDiff.entriesOnlyOnLeft().values())
-            dropIndexFromSchemaMutation(oldTable, index, timestamp, mutation);
+            dropIndexFromSchemaMutation(oldTable, index, builder);
 
         // newly created indexes
         for (IndexMetadata index : indexesDiff.entriesOnlyOnRight().values())
-            addIndexToSchemaMutation(newTable, index, timestamp, mutation);
+            addIndexToSchemaMutation(newTable, index, builder);
 
         // updated indexes need to be updated
         for (MapDifference.ValueDifference<IndexMetadata> diff : indexesDiff.entriesDiffering().values())
-            addUpdatedIndexToSchemaMutation(newTable, diff.rightValue(), timestamp, mutation);
+            addUpdatedIndexToSchemaMutation(newTable, diff.rightValue(), builder);
 
-        return mutation;
+        return builder;
     }
 
     private static MapDifference<String, IndexMetadata> indexesDiff(Indexes before, Indexes after)
@@ -600,144 +595,137 @@ public final class SchemaKeyspace
         return Maps.difference(beforeMap, afterMap);
     }
 
-    public static Mutation makeDropTableMutation(KeyspaceMetadata keyspace, CFMetaData table, long timestamp)
+    public static Mutation.SimpleBuilder makeDropTableMutation(KeyspaceMetadata keyspace, CFMetaData table, long timestamp)
     {
         // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
-        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
+        Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 
-        RowUpdateBuilder.deleteRow(Tables, timestamp, mutation, table.cfName);
+        builder.update(Tables).row(table.cfName).delete();
 
         for (ColumnDefinition column : table.allColumns())
-            dropColumnFromSchemaMutation(table, column, timestamp, mutation);
+            dropColumnFromSchemaMutation(table, column, builder);
 
         for (TriggerMetadata trigger : table.getTriggers())
-            dropTriggerFromSchemaMutation(table, trigger, timestamp, mutation);
+            dropTriggerFromSchemaMutation(table, trigger, builder);
 
         for (IndexMetadata index : table.getIndexes())
-            dropIndexFromSchemaMutation(table, index, timestamp, mutation);
+            dropIndexFromSchemaMutation(table, index, builder);
 
-        return mutation;
+        return builder;
     }
 
-    private static void addColumnToSchemaMutation(CFMetaData table, ColumnDefinition column, long timestamp, Mutation mutation)
+    private static void addColumnToSchemaMutation(CFMetaData table, ColumnDefinition column, Mutation.SimpleBuilder builder)
     {
-        RowUpdateBuilder adder = new RowUpdateBuilder(Columns, timestamp, mutation).clustering(table.cfName, column.name.toString());
-
         AbstractType<?> type = column.type;
         if (type instanceof ReversedType)
             type = ((ReversedType) type).baseType;
 
-        adder.add("column_name_bytes", column.name.bytes)
-             .add("kind", column.kind.toString().toLowerCase())
-             .add("position", column.position())
-             .add("clustering_order", column.clusteringOrder().toString().toLowerCase())
-             .add("type", type.asCQL3Type().toString())
-             .build();
+        builder.update(Columns)
+               .row(table.cfName, column.name.toString())
+               .add("column_name_bytes", column.name.bytes)
+               .add("kind", column.kind.toString().toLowerCase())
+               .add("position", column.position())
+               .add("clustering_order", column.clusteringOrder().toString().toLowerCase())
+               .add("type", type.asCQL3Type().toString());
     }
 
-    private static void dropColumnFromSchemaMutation(CFMetaData table, ColumnDefinition column, long timestamp, Mutation mutation)
+    private static void dropColumnFromSchemaMutation(CFMetaData table, ColumnDefinition column, Mutation.SimpleBuilder builder)
     {
         // Note: we do want to use name.toString(), not name.bytes directly for backward compatibility (For CQL3, this won't make a difference).
-        RowUpdateBuilder.deleteRow(Columns, timestamp, mutation, table.cfName, column.name.toString());
+        builder.update(Columns).row(table.cfName, column.name.toString()).delete();
     }
 
-    private static void addDroppedColumnToSchemaMutation(CFMetaData table, CFMetaData.DroppedColumn column, long timestamp, Mutation mutation)
+    private static void addDroppedColumnToSchemaMutation(CFMetaData table, CFMetaData.DroppedColumn column, Mutation.SimpleBuilder builder)
     {
-        RowUpdateBuilder adder = new RowUpdateBuilder(DroppedColumns, timestamp, mutation).clustering(table.cfName, column.name);
-
-        adder.add("dropped_time", new Date(TimeUnit.MICROSECONDS.toMillis(column.droppedTime)))
-             .add("type", expandUserTypes(column.type).asCQL3Type().toString())
-             .build();
+        builder.update(DroppedColumns)
+               .row(table.cfName, column.name)
+               .add("dropped_time", new Date(TimeUnit.MICROSECONDS.toMillis(column.droppedTime)))
+               .add("type", expandUserTypes(column.type).asCQL3Type().toString());
     }
 
-    private static void addTriggerToSchemaMutation(CFMetaData table, TriggerMetadata trigger, long timestamp, Mutation mutation)
+    private static void addTriggerToSchemaMutation(CFMetaData table, TriggerMetadata trigger, Mutation.SimpleBuilder builder)
     {
-        new RowUpdateBuilder(Triggers, timestamp, mutation)
-            .clustering(table.cfName, trigger.name)
-            .frozenMap("options", Collections.singletonMap("class", trigger.classOption))
-            .build();
+        builder.update(Triggers)
+               .row(table.cfName, trigger.name)
+               .add("options", Collections.singletonMap("class", trigger.classOption));
     }
 
-    private static void dropTriggerFromSchemaMutation(CFMetaData table, TriggerMetadata trigger, long timestamp, Mutation mutation)
+    private static void dropTriggerFromSchemaMutation(CFMetaData table, TriggerMetadata trigger, Mutation.SimpleBuilder builder)
     {
-        RowUpdateBuilder.deleteRow(Triggers, timestamp, mutation, table.cfName, trigger.name);
+        builder.update(Triggers).row(table.cfName, trigger.name).delete();
     }
 
-    public static Mutation makeCreateViewMutation(KeyspaceMetadata keyspace, ViewDefinition view, long timestamp)
+    public static Mutation.SimpleBuilder makeCreateViewMutation(KeyspaceMetadata keyspace, ViewDefinition view, long timestamp)
     {
         // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
-        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
-        addViewToSchemaMutation(view, timestamp, true, mutation);
-        return mutation;
+        Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
+        addViewToSchemaMutation(view, true, builder);
+        return builder;
     }
 
-    private static void addViewToSchemaMutation(ViewDefinition view, long timestamp, boolean includeColumns, Mutation mutation)
+    private static void addViewToSchemaMutation(ViewDefinition view, boolean includeColumns, Mutation.SimpleBuilder builder)
     {
-        RowUpdateBuilder builder = new RowUpdateBuilder(Views, timestamp, mutation)
-            .clustering(view.viewName);
-
         CFMetaData table = view.metadata;
+        Row.SimpleBuilder rowBuilder = builder.update(Views)
+                                              .row(view.viewName)
+                                              .add("include_all_columns", view.includeAllColumns)
+                                              .add("base_table_id", view.baseTableId)
+                                              .add("base_table_name", view.baseTableMetadata().cfName)
+                                              .add("where_clause", view.whereClause)
+                                              .add("id", table.cfId);
 
-        builder.add("include_all_columns", view.includeAllColumns)
-               .add("base_table_id", view.baseTableId)
-               .add("base_table_name", view.baseTableMetadata().cfName)
-               .add("where_clause", view.whereClause)
-               .add("id", table.cfId);
-
-        addTableParamsToSchemaMutation(table.params, builder);
+        addTableParamsToRowBuilder(table.params, rowBuilder);
 
         if (includeColumns)
         {
             for (ColumnDefinition column : table.allColumns())
-                addColumnToSchemaMutation(table, column, timestamp, mutation);
+                addColumnToSchemaMutation(table, column, builder);
 
             for (CFMetaData.DroppedColumn column : table.getDroppedColumns().values())
-                addDroppedColumnToSchemaMutation(table, column, timestamp, mutation);
+                addDroppedColumnToSchemaMutation(table, column, builder);
         }
-
-        builder.build();
     }
 
-    public static Mutation makeDropViewMutation(KeyspaceMetadata keyspace, ViewDefinition view, long timestamp)
+    public static Mutation.SimpleBuilder makeDropViewMutation(KeyspaceMetadata keyspace, ViewDefinition view, long timestamp)
     {
         // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
-        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
+        Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 
-        RowUpdateBuilder.deleteRow(Views, timestamp, mutation, view.viewName);
+        builder.update(Views).row(view.viewName).delete();
 
         CFMetaData table = view.metadata;
         for (ColumnDefinition column : table.allColumns())
-            dropColumnFromSchemaMutation(table, column, timestamp, mutation);
+            dropColumnFromSchemaMutation(table, column, builder);
 
         for (IndexMetadata index : table.getIndexes())
-            dropIndexFromSchemaMutation(table, index, timestamp, mutation);
+            dropIndexFromSchemaMutation(table, index, builder);
 
-        return mutation;
+        return builder;
     }
 
-    public static Mutation makeUpdateViewMutation(KeyspaceMetadata keyspace,
-                                                  ViewDefinition oldView,
-                                                  ViewDefinition newView,
-                                                  long timestamp)
+    public static Mutation.SimpleBuilder makeUpdateViewMutation(KeyspaceMetadata keyspace,
+                                                                ViewDefinition oldView,
+                                                                ViewDefinition newView,
+                                                                long timestamp)
     {
-        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
+        Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 
-        addViewToSchemaMutation(newView, timestamp, false, mutation);
+        addViewToSchemaMutation(newView, false, builder);
 
         MapDifference<ByteBuffer, ColumnDefinition> columnDiff = Maps.difference(oldView.metadata.getColumnMetadata(),
                                                                                  newView.metadata.getColumnMetadata());
 
         // columns that are no longer needed
         for (ColumnDefinition column : columnDiff.entriesOnlyOnLeft().values())
-            dropColumnFromSchemaMutation(oldView.metadata, column, timestamp, mutation);
+            dropColumnFromSchemaMutation(oldView.metadata, column, builder);
 
         // newly added columns
         for (ColumnDefinition column : columnDiff.entriesOnlyOnRight().values())
-            addColumnToSchemaMutation(newView.metadata, column, timestamp, mutation);
+            addColumnToSchemaMutation(newView.metadata, column, builder);
 
         // old columns with updated attributes
         for (ByteBuffer name : columnDiff.entriesDiffering().keySet())
-            addColumnToSchemaMutation(newView.metadata, newView.metadata.getColumnDefinition(name), timestamp, mutation);
+            addColumnToSchemaMutation(newView.metadata, newView.metadata.getColumnDefinition(name), builder);
 
         // dropped columns
         MapDifference<ByteBuffer, CFMetaData.DroppedColumn> droppedColumnDiff =
@@ -745,63 +733,68 @@ public final class SchemaKeyspace
 
         // newly dropped columns
         for (CFMetaData.DroppedColumn column : droppedColumnDiff.entriesOnlyOnRight().values())
-            addDroppedColumnToSchemaMutation(oldView.metadata, column, timestamp, mutation);
+            addDroppedColumnToSchemaMutation(oldView.metadata, column, builder);
 
         // columns added then dropped again
         for (ByteBuffer name : droppedColumnDiff.entriesDiffering().keySet())
-            addDroppedColumnToSchemaMutation(newView.metadata, newView.metadata.getDroppedColumns().get(name), timestamp, mutation);
+            addDroppedColumnToSchemaMutation(newView.metadata, newView.metadata.getDroppedColumns().get(name), builder);
 
-        return mutation;
+        return builder;
     }
 
     private static void addIndexToSchemaMutation(CFMetaData table,
                                                  IndexMetadata index,
-                                                 long timestamp,
-                                                 Mutation mutation)
+                                                 Mutation.SimpleBuilder builder)
     {
-        RowUpdateBuilder builder = new RowUpdateBuilder(Indexes, timestamp, mutation).clustering(table.cfName, index.name);
-
-        builder.add("kind", index.kind.toString());
-        builder.frozenMap("options", index.options);
-        builder.build();
+        builder.update(Indexes)
+               .row(table.cfName, index.name)
+               .add("kind", index.kind.toString())
+               .add("options", index.options);
     }
 
     private static void dropIndexFromSchemaMutation(CFMetaData table,
                                                     IndexMetadata index,
-                                                    long timestamp,
-                                                    Mutation mutation)
+                                                    Mutation.SimpleBuilder builder)
     {
-        RowUpdateBuilder.deleteRow(Indexes, timestamp, mutation, table.cfName, index.name);
+        builder.update(Indexes).row(table.cfName, index.name).delete();
     }
 
     private static void addUpdatedIndexToSchemaMutation(CFMetaData table,
                                                         IndexMetadata index,
-                                                        long timestamp,
-                                                        Mutation mutation)
+                                                        Mutation.SimpleBuilder builder)
     {
-        addIndexToSchemaMutation(table, index, timestamp, mutation);
+        addIndexToSchemaMutation(table, index, builder);
     }
 
-    public static Mutation makeCreateFunctionMutation(KeyspaceMetadata keyspace, UDFunction function, long timestamp)
+    public static Mutation.SimpleBuilder makeCreateFunctionMutation(KeyspaceMetadata keyspace, UDFunction function, long timestamp)
     {
         // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
-        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
-        addFunctionToSchemaMutation(function, timestamp, mutation);
-        return mutation;
+        Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
+        addFunctionToSchemaMutation(function, builder);
+        return builder;
     }
 
-    static void addFunctionToSchemaMutation(UDFunction function, long timestamp, Mutation mutation)
+    static void addFunctionToSchemaMutation(UDFunction function, Mutation.SimpleBuilder builder)
     {
-        RowUpdateBuilder adder =
-            new RowUpdateBuilder(Functions, timestamp, mutation).clustering(function.name().name, functionArgumentsList(function));
-
-        adder.add("body", function.body())
-             .add("language", function.language())
-             .add("return_type", function.returnType().asCQL3Type().toString())
-             .add("called_on_null_input", function.isCalledOnNullInput())
-             .frozenList("argument_names", function.argNames().stream().map((c) -> bbToString(c.bytes)).collect(toList()));
+        builder.update(Functions)
+               .row(function.name().name, functionArgumentsList(function))
+               .add("body", function.body())
+               .add("language", function.language())
+               .add("return_type", function.returnType().asCQL3Type().toString())
+               .add("called_on_null_input", function.isCalledOnNullInput())
+               .add("argument_names", function.argNames().stream().map((c) -> bbToString(c.bytes)).collect(toList()));
+    }
 
-        adder.build();
+    private static String bbToString(ByteBuffer bb)
+    {
+        try
+        {
+            return ByteBufferUtil.string(bb);
+        }
+        catch (CharacterCodingException e)
+        {
+            throw new RuntimeException(e);
+        }
     }
 
     private static List<String> functionArgumentsList(AbstractFunction fun)
@@ -813,42 +806,42 @@ public final class SchemaKeyspace
                   .collect(toList());
     }
 
-    public static Mutation makeDropFunctionMutation(KeyspaceMetadata keyspace, UDFunction function, long timestamp)
+    public static Mutation.SimpleBuilder makeDropFunctionMutation(KeyspaceMetadata keyspace, UDFunction function, long timestamp)
     {
         // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
-        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
-        return RowUpdateBuilder.deleteRow(Functions, timestamp, mutation, function.name().name, functionArgumentsList(function));
+        Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
+        builder.update(Functions).row(function.name().name, functionArgumentsList(function)).delete();
+        return builder;
     }
 
-    public static Mutation makeCreateAggregateMutation(KeyspaceMetadata keyspace, UDAggregate aggregate, long timestamp)
+    public static Mutation.SimpleBuilder makeCreateAggregateMutation(KeyspaceMetadata keyspace, UDAggregate aggregate, long timestamp)
     {
         // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
-        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
-        addAggregateToSchemaMutation(aggregate, timestamp, mutation);
-        return mutation;
+        Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
+        addAggregateToSchemaMutation(aggregate, builder);
+        return builder;
     }
 
-    static void addAggregateToSchemaMutation(UDAggregate aggregate, long timestamp, Mutation mutation)
+    static void addAggregateToSchemaMutation(UDAggregate aggregate, Mutation.SimpleBuilder builder)
     {
-        RowUpdateBuilder adder =
-            new RowUpdateBuilder(Aggregates, timestamp, mutation) .clustering(aggregate.name().name, functionArgumentsList(aggregate));
-
-        adder.add("return_type", aggregate.returnType().asCQL3Type().toString())
-             .add("state_func", aggregate.stateFunction().name().name)
-             .add("state_type", aggregate.stateType().asCQL3Type().toString())
-             .add("final_func", aggregate.finalFunction() != null ? aggregate.finalFunction().name().name : null)
-             .add("initcond", aggregate.initialCondition() != null
-                              // must use the frozen state type here, as 'null' for unfrozen collections may mean 'empty'
-                              ? aggregate.stateType().freeze().asCQL3Type().toCQLLiteral(aggregate.initialCondition(), Server.CURRENT_VERSION)
-                              : null)
-             .build();
+        builder.update(Aggregates)
+               .row(aggregate.name().name, functionArgumentsList(aggregate))
+               .add("return_type", aggregate.returnType().asCQL3Type().toString())
+               .add("state_func", aggregate.stateFunction().name().name)
+               .add("state_type", aggregate.stateType().asCQL3Type().toString())
+               .add("final_func", aggregate.finalFunction() != null ? aggregate.finalFunction().name().name : null)
+               .add("initcond", aggregate.initialCondition() != null
+                                // must use the frozen state type here, as 'null' for unfrozen collections may mean 'empty'
+                                ? aggregate.stateType().freeze().asCQL3Type().toCQLLiteral(aggregate.initialCondition(), Server.CURRENT_VERSION)
+                                : null);
     }
 
-    public static Mutation makeDropAggregateMutation(KeyspaceMetadata keyspace, UDAggregate aggregate, long timestamp)
+    public static Mutation.SimpleBuilder makeDropAggregateMutation(KeyspaceMetadata keyspace, UDAggregate aggregate, long timestamp)
     {
         // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
-        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
-        return RowUpdateBuilder.deleteRow(Aggregates, timestamp, mutation, aggregate.name().name, functionArgumentsList(aggregate));
+        Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
+        builder.update(Aggregates).row(aggregate.name().name, functionArgumentsList(aggregate)).delete();
+        return builder;
     }
 
     /*

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26838063/src/java/org/apache/cassandra/service/MigrationManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/MigrationManager.java b/src/java/org/apache/cassandra/service/MigrationManager.java
index ba239b3..7eac678 100644
--- a/src/java/org/apache/cassandra/service/MigrationManager.java
+++ b/src/java/org/apache/cassandra/service/MigrationManager.java
@@ -506,12 +506,14 @@ public class MigrationManager
      * actively announce a new version to active hosts via rpc
      * @param schema The schema mutation to be applied
      */
-    private static void announce(Mutation schema, boolean announceLocally)
+    private static void announce(Mutation.SimpleBuilder schema, boolean announceLocally)
     {
+        List<Mutation> mutations = Collections.singletonList(schema.build());
+
         if (announceLocally)
-            SchemaKeyspace.mergeSchema(Collections.singletonList(schema));
+            SchemaKeyspace.mergeSchema(mutations);
         else
-            FBUtilities.waitOnFuture(announce(Collections.singletonList(schema)));
+            FBUtilities.waitOnFuture(announce(mutations));
     }
 
     private static void pushSchemaMutation(InetAddress endpoint, Collection<Mutation> schema)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26838063/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tracing/TraceKeyspace.java b/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
index d7b804a..5c400a9 100644
--- a/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
+++ b/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
@@ -23,7 +23,8 @@ import java.util.*;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.db.RowUpdateBuilder;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.schema.Tables;
@@ -86,36 +87,41 @@ public final class TraceKeyspace
                                              String command,
                                              int ttl)
     {
-        RowUpdateBuilder adder = new RowUpdateBuilder(Sessions, FBUtilities.timestampMicros(), ttl, sessionId)
-                                 .clustering()
-                                 .add("client", client)
-                                 .add("coordinator", FBUtilities.getBroadcastAddress())
-                                 .add("request", request)
-                                 .add("started_at", new Date(startedAt))
-                                 .add("command", command);
-
-        for (Map.Entry<String, String> entry : parameters.entrySet())
-            adder.addMapEntry("parameters", entry.getKey(), entry.getValue());
-        return adder.build();
+        PartitionUpdate.SimpleBuilder builder = PartitionUpdate.simpleBuilder(Sessions, sessionId);
+        builder.row()
+               .ttl(ttl)
+               .add("client", client)
+               .add("coordinator", FBUtilities.getBroadcastAddress())
+               .add("request", request)
+               .add("started_at", new Date(startedAt))
+               .add("command", command)
+               .appendAll("parameters", parameters);
+
+        return builder.buildAsMutation();
     }
 
     static Mutation makeStopSessionMutation(ByteBuffer sessionId, int elapsed, int ttl)
     {
-        return new RowUpdateBuilder(Sessions, FBUtilities.timestampMicros(), ttl, sessionId)
-               .clustering()
-               .add("duration", elapsed)
-               .build();
+        PartitionUpdate.SimpleBuilder builder = PartitionUpdate.simpleBuilder(Sessions, sessionId);
+        builder.row()
+               .ttl(ttl)
+               .add("duration", elapsed);
+        return builder.buildAsMutation();
     }
 
     static Mutation makeEventMutation(ByteBuffer sessionId, String message, int elapsed, String threadName, int ttl)
     {
-        RowUpdateBuilder adder = new RowUpdateBuilder(Events, FBUtilities.timestampMicros(), ttl, sessionId)
-                                 .clustering(UUIDGen.getTimeUUID());
-        adder.add("activity", message);
-        adder.add("source", FBUtilities.getBroadcastAddress());
-        adder.add("thread", threadName);
+        PartitionUpdate.SimpleBuilder builder = PartitionUpdate.simpleBuilder(Events, sessionId);
+        Row.SimpleBuilder rowBuilder = builder.row(UUIDGen.getTimeUUID())
+                                              .ttl(ttl);
+
+        rowBuilder.add("activity", message)
+                  .add("source", FBUtilities.getBroadcastAddress())
+                  .add("thread", threadName);
+
         if (elapsed >= 0)
-            adder.add("source_elapsed", elapsed);
-        return adder.build();
+            rowBuilder.add("source_elapsed", elapsed);
+
+        return builder.buildAsMutation();
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26838063/test/unit/org/apache/cassandra/UpdateBuilder.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/UpdateBuilder.java b/test/unit/org/apache/cassandra/UpdateBuilder.java
index 3a5fbe6..19e48f2 100644
--- a/test/unit/org/apache/cassandra/UpdateBuilder.java
+++ b/test/unit/org/apache/cassandra/UpdateBuilder.java
@@ -21,6 +21,7 @@ import java.nio.ByteBuffer;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.service.StorageService;
@@ -34,32 +35,30 @@ import org.apache.cassandra.service.StorageService;
  */
 public class UpdateBuilder
 {
-    private final PartitionUpdate update;
-    private RowUpdateBuilder currentRow;
-    private long timestamp = FBUtilities.timestampMicros();
+    private final PartitionUpdate.SimpleBuilder updateBuilder;
+    private Row.SimpleBuilder currentRow;
 
-    private UpdateBuilder(CFMetaData metadata, DecoratedKey partitionKey)
+    private UpdateBuilder(PartitionUpdate.SimpleBuilder updateBuilder)
     {
-        this.update = new PartitionUpdate(metadata, partitionKey, metadata.partitionColumns(), 4);
+        this.updateBuilder = updateBuilder;
     }
 
     public static UpdateBuilder create(CFMetaData metadata, Object... partitionKey)
     {
-        return new UpdateBuilder(metadata, makeKey(metadata, partitionKey));
+        return new UpdateBuilder(PartitionUpdate.simpleBuilder(metadata, partitionKey));
     }
 
     public UpdateBuilder withTimestamp(long timestamp)
     {
-        this.timestamp = timestamp;
+        updateBuilder.timestamp(timestamp);
+        if (currentRow != null)
+            currentRow.timestamp(timestamp);
         return this;
     }
 
     public UpdateBuilder newRow(Object... clustering)
     {
-        maybeBuildCurrentRow();
-        currentRow = new RowUpdateBuilder(update, timestamp, 0);
-        if (clustering.length > 0)
-            currentRow.clustering(clustering);
+        currentRow = updateBuilder.row(clustering);
         return this;
     }
 
@@ -72,48 +71,25 @@ public class UpdateBuilder
 
     public PartitionUpdate build()
     {
-        maybeBuildCurrentRow();
-        return update;
+        return updateBuilder.build();
     }
 
     public IMutation makeMutation()
     {
-        Mutation m = new Mutation(build());
-        return update.metadata().isCounter()
+        Mutation m = updateBuilder.buildAsMutation();
+        return updateBuilder.metadata().isCounter()
              ? new CounterMutation(m, ConsistencyLevel.ONE)
              : m;
     }
 
     public void apply()
     {
-        Mutation m = new Mutation(build());
-        if (update.metadata().isCounter())
-            new CounterMutation(m, ConsistencyLevel.ONE).apply();
-        else
-            m.apply();
+        makeMutation().apply();
     }
 
     public void applyUnsafe()
     {
-        assert !update.metadata().isCounter() : "Counters have currently no applyUnsafe() option";
-        new Mutation(build()).applyUnsafe();
-    }
-
-    private void maybeBuildCurrentRow()
-    {
-        if (currentRow != null)
-        {
-            currentRow.build();
-            currentRow = null;
-        }
-    }
-
-    private static DecoratedKey makeKey(CFMetaData metadata, Object[] partitionKey)
-    {
-        if (partitionKey.length == 1 && partitionKey[0] instanceof DecoratedKey)
-            return (DecoratedKey)partitionKey[0];
-
-        ByteBuffer key = CFMetaData.serializePartitionKey(metadata.getKeyValidatorAsClusteringComparator().make(partitionKey));
-        return metadata.decorateKey(key);
+        assert !updateBuilder.metadata().isCounter() : "Counters have currently no applyUnsafe() option";
+        updateBuilder.buildAsMutation().applyUnsafe();
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26838063/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 9689628..151d3d4 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -459,15 +459,37 @@ public class Util
     public static boolean equal(UnfilteredRowIterator a, UnfilteredRowIterator b)
     {
         return Objects.equals(a.columns(), b.columns())
-            && Objects.equals(a.metadata(), b.metadata())
+            && Objects.equals(a.stats(), b.stats())
+            && sameContent(a, b);
+    }
+
+    // Test equality of the iterators, but without caring too much about the "metadata" of said iterator. This is often
+    // what we want in tests. In particular, the columns() reported by the iterators will sometimes differ because they
+    // are a superset of what the iterator actually contains, and depending on the method used to get each iterator
+    // tested, one may include a defined column the other don't while there is not actual content for that column.
+    public static boolean sameContent(UnfilteredRowIterator a, UnfilteredRowIterator b)
+    {
+        return Objects.equals(a.metadata(), b.metadata())
             && Objects.equals(a.isReverseOrder(), b.isReverseOrder())
             && Objects.equals(a.partitionKey(), b.partitionKey())
             && Objects.equals(a.partitionLevelDeletion(), b.partitionLevelDeletion())
             && Objects.equals(a.staticRow(), b.staticRow())
-            && Objects.equals(a.stats(), b.stats())
             && Iterators.elementsEqual(a, b);
     }
 
+    public static boolean sameContent(Mutation a, Mutation b)
+    {
+        if (!a.key().equals(b.key()) || !a.getColumnFamilyIds().equals(b.getColumnFamilyIds()))
+            return false;
+
+        for (UUID cfId : a.getColumnFamilyIds())
+        {
+            if (!sameContent(a.getPartitionUpdate(cfId).unfilteredIterator(), b.getPartitionUpdate(cfId).unfilteredIterator()))
+                return false;
+        }
+        return true;
+    }
+
     // moved & refactored from KeyspaceTest in < 3.0
     public static void assertColumns(Row row, String... expectedColumnNames)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26838063/test/unit/org/apache/cassandra/batchlog/BatchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/batchlog/BatchTest.java b/test/unit/org/apache/cassandra/batchlog/BatchTest.java
index b7a4100..4e64ec6 100644
--- a/test/unit/org/apache/cassandra/batchlog/BatchTest.java
+++ b/test/unit/org/apache/cassandra/batchlog/BatchTest.java
@@ -28,6 +28,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.Mutation;
@@ -44,6 +45,7 @@ import org.apache.cassandra.utils.UUIDGen;
 
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 public class BatchTest
 {
@@ -148,6 +150,19 @@ public class BatchTest
         Iterator<Mutation> it1 = batch1.decodedMutations.iterator();
         Iterator<Mutation> it2 = batch2.decodedMutations.iterator();
         while (it1.hasNext())
-            assertEquals(it1.next().toString(), it2.next().toString());
+        {
+            // We can't simply test the equality of both mutation string representation, that is do:
+            //   assertEquals(it1.next().toString(), it2.next().toString());
+            // because when deserializing from the old format, the returned iterator will always have it's 'columns()'
+            // method return all the table columns (no matter what's the actual content), and the table contains a
+            // 'val0' column we're not setting in that test.
+            //
+            // And it's actually not easy to fix legacy deserialization as we'd need to know which columns are actually
+            // set upfront, which would require use to iterate over the whole content first, which would be costly. And
+            // as the result of 'columns()' is only meant as a superset of the columns in the iterator, we don't bother.
+            Mutation mut1 = it1.next();
+            Mutation mut2 = it2.next();
+            assertTrue(mut1 + " != " + mut2, Util.sameContent(mut1, mut2));
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26838063/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 6bfe5c0..8616987 100644
--- a/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
+++ b/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
@@ -150,7 +150,7 @@ public class CFMetaDataTest
         assert before.equals(after) : String.format("%n%s%n!=%n%s", before, after);
 
         // Test schema conversion
-        Mutation rm = SchemaKeyspace.makeCreateTableMutation(keyspace, cfm, FBUtilities.timestampMicros());
+        Mutation rm = SchemaKeyspace.makeCreateTableMutation(keyspace, cfm, FBUtilities.timestampMicros()).build();
         PartitionUpdate cfU = rm.getPartitionUpdate(Schema.instance.getId(SchemaKeyspace.NAME, SchemaKeyspace.TABLES));
         PartitionUpdate cdU = rm.getPartitionUpdate(Schema.instance.getId(SchemaKeyspace.NAME, SchemaKeyspace.COLUMNS));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26838063/test/unit/org/apache/cassandra/cql3/CDCStatementTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CDCStatementTest.java b/test/unit/org/apache/cassandra/cql3/CDCStatementTest.java
index 632c290..0b18eec 100644
--- a/test/unit/org/apache/cassandra/cql3/CDCStatementTest.java
+++ b/test/unit/org/apache/cassandra/cql3/CDCStatementTest.java
@@ -19,10 +19,20 @@
 package org.apache.cassandra.cql3;
 
 import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
+
 public class CDCStatementTest extends CQLTester
 {
+    @BeforeClass
+    public static void checkConfig()
+    {
+        Assume.assumeTrue(DatabaseDescriptor.isCDCEnabled());
+    }
+
     @Test
     public void testEnableOnCreate() throws Throwable
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26838063/test/unit/org/apache/cassandra/cql3/validation/entities/RowUpdateBuilderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/RowUpdateBuilderTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/RowUpdateBuilderTest.java
deleted file mode 100644
index afe2455..0000000
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/RowUpdateBuilderTest.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cql3.validation.entities;
-
-import org.junit.Test;
-
-import org.apache.cassandra.Util;
-import org.apache.cassandra.cql3.CQLTester;
-import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.db.RowUpdateBuilder;
-import org.apache.cassandra.utils.FBUtilities;
-
-// see CASSANDRA-9743, CASSANDRA-9746
-public class RowUpdateBuilderTest extends CQLTester
-{
-    @Test
-    public void testAddListEntryDurable() throws Throwable
-    {
-        testAddListEntry(false);
-    }
-
-    @Test
-    public void testAddListEntryTransient() throws Throwable
-    {
-        testAddListEntry(true);
-    }
-
-    public void testAddListEntry(boolean skipCommitLog) throws Throwable
-    {
-        createTable("CREATE TABLE %s ("
-                    + "pk text,"
-                    + "ck text,"
-                    + "l1 list<int>,"
-                    + "l2 list<int>,"
-                    + "PRIMARY KEY ((pk), ck))");
-
-        long timestamp = FBUtilities.timestampMicros();
-
-        Mutation mutation = new Mutation(keyspace(), Util.dk("test"));
-        addToMutation("row1", timestamp, mutation);
-        addToMutation("row2", timestamp, mutation);
-
-        if (skipCommitLog)
-            mutation.applyUnsafe();
-        else
-            mutation.apply();
-
-        assertRowCount(execute("SELECT ck FROM %s"), 2);
-    }
-
-    private void addToMutation(String typeName, long timestamp, Mutation mutation)
-    {
-        RowUpdateBuilder adder = new RowUpdateBuilder(getCurrentColumnFamilyStore().metadata, timestamp, mutation)
-                                 .clustering(typeName);
-
-        for (int i = 0; i < 2; i++)
-        {
-            adder.addListEntry("l1", i)
-                 .addListEntry("l2", i);
-        }
-
-        adder.build();
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26838063/test/unit/org/apache/cassandra/db/RecoveryManagerMissingHeaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManagerMissingHeaderTest.java b/test/unit/org/apache/cassandra/db/RecoveryManagerMissingHeaderTest.java
index a67e9e5..8897700 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManagerMissingHeaderTest.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManagerMissingHeaderTest.java
@@ -118,7 +118,7 @@ public class RecoveryManagerMissingHeaderTest
 
         CommitLog.instance.resetUnsafe(false);
 
-        Assert.assertTrue(Util.equal(upd1, Util.getOnlyPartitionUnfiltered(Util.cmd(keyspace1.getColumnFamilyStore(CF_STANDARD1), dk).build()).unfilteredIterator()));
-        Assert.assertTrue(Util.equal(upd2, Util.getOnlyPartitionUnfiltered(Util.cmd(keyspace2.getColumnFamilyStore(CF_STANDARD3), dk).build()).unfilteredIterator()));
+        Assert.assertTrue(Util.sameContent(upd1, Util.getOnlyPartitionUnfiltered(Util.cmd(keyspace1.getColumnFamilyStore(CF_STANDARD1), dk).build()).unfilteredIterator()));
+        Assert.assertTrue(Util.sameContent(upd2, Util.getOnlyPartitionUnfiltered(Util.cmd(keyspace2.getColumnFamilyStore(CF_STANDARD3), dk).build()).unfilteredIterator()));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26838063/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 37d719e..cbc412d 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
@@ -188,8 +188,8 @@ public class RecoveryManagerTest
             }
             Assert.assertFalse(t.isAlive());
 
-            Assert.assertTrue(Util.equal(upd1, Util.getOnlyPartitionUnfiltered(Util.cmd(keyspace1.getColumnFamilyStore(CF_STANDARD1), dk).build()).unfilteredIterator()));
-            Assert.assertTrue(Util.equal(upd2, Util.getOnlyPartitionUnfiltered(Util.cmd(keyspace2.getColumnFamilyStore(CF_STANDARD3), dk).build()).unfilteredIterator()));
+            Assert.assertTrue(Util.sameContent(upd1, Util.getOnlyPartitionUnfiltered(Util.cmd(keyspace1.getColumnFamilyStore(CF_STANDARD1), dk).build()).unfilteredIterator()));
+            Assert.assertTrue(Util.sameContent(upd2, Util.getOnlyPartitionUnfiltered(Util.cmd(keyspace2.getColumnFamilyStore(CF_STANDARD3), dk).build()).unfilteredIterator()));
         }
         finally
         {
@@ -220,8 +220,8 @@ public class RecoveryManagerTest
         CommitLog.instance.resetUnsafe(false);
 
         DecoratedKey dk = Util.dk("keymulti");
-        Assert.assertTrue(Util.equal(upd1, Util.getOnlyPartitionUnfiltered(Util.cmd(keyspace1.getColumnFamilyStore(CF_STANDARD1), dk).build()).unfilteredIterator()));
-        Assert.assertTrue(Util.equal(upd2, Util.getOnlyPartitionUnfiltered(Util.cmd(keyspace2.getColumnFamilyStore(CF_STANDARD3), dk).build()).unfilteredIterator()));
+        Assert.assertTrue(Util.sameContent(upd1, Util.getOnlyPartitionUnfiltered(Util.cmd(keyspace1.getColumnFamilyStore(CF_STANDARD1), dk).build()).unfilteredIterator()));
+        Assert.assertTrue(Util.sameContent(upd2, Util.getOnlyPartitionUnfiltered(Util.cmd(keyspace2.getColumnFamilyStore(CF_STANDARD3), dk).build()).unfilteredIterator()));
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26838063/test/unit/org/apache/cassandra/db/RowUpdateBuilder.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowUpdateBuilder.java b/test/unit/org/apache/cassandra/db/RowUpdateBuilder.java
new file mode 100644
index 0000000..8e71d64
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/RowUpdateBuilder.java
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.db.context.CounterContext;
+import org.apache.cassandra.db.partitions.*;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.utils.*;
+
+/**
+ * Convenience object to create single row updates for tests.
+ *
+ * This is a thin wrapper over the builders in SimpleBuilders for historical reasons.
+ * We could modify all the tests using this class to use the simple builders directly
+ * instead, but there is a fair amount of use so the value of such effort is unclear.
+ */
+public class RowUpdateBuilder
+{
+    private final PartitionUpdate.SimpleBuilder updateBuilder;
+    private Row.SimpleBuilder rowBuilder;
+    private boolean noRowMarker;
+
+    private List<RangeTombstone> rts = new ArrayList<>();
+
+    private RowUpdateBuilder(PartitionUpdate.SimpleBuilder updateBuilder)
+    {
+        this.updateBuilder = updateBuilder;
+    }
+
+    public RowUpdateBuilder(CFMetaData metadata, long timestamp, Object partitionKey)
+    {
+        this(metadata, FBUtilities.nowInSeconds(), timestamp, partitionKey);
+    }
+
+    public RowUpdateBuilder(CFMetaData metadata, int localDeletionTime, long timestamp, Object partitionKey)
+    {
+        this(metadata, localDeletionTime, timestamp, metadata.params.defaultTimeToLive, partitionKey);
+    }
+
+    public RowUpdateBuilder(CFMetaData metadata, long timestamp, int ttl, Object partitionKey)
+    {
+        this(metadata, FBUtilities.nowInSeconds(), timestamp, ttl, partitionKey);
+    }
+
+    public RowUpdateBuilder(CFMetaData metadata, int localDeletionTime, long timestamp, int ttl, Object partitionKey)
+    {
+        this(PartitionUpdate.simpleBuilder(metadata, partitionKey));
+
+        this.updateBuilder.timestamp(timestamp);
+        this.updateBuilder.ttl(ttl);
+        this.updateBuilder.nowInSec(localDeletionTime);
+    }
+
+    private Row.SimpleBuilder rowBuilder()
+    {
+        // Normally, rowBuilder is created by the call to clustering(), but we allow skipping that call for an empty
+        // clustering.
+        if (rowBuilder == null)
+        {
+            rowBuilder = updateBuilder.row();
+            if (noRowMarker)
+                rowBuilder.noPrimaryKeyLivenessInfo();
+        }
+
+        return rowBuilder;
+    }
+
+    // This must be called before any addition or deletion if used.
+    public RowUpdateBuilder noRowMarker()
+    {
+        this.noRowMarker = true;
+        if (rowBuilder != null)
+            rowBuilder.noPrimaryKeyLivenessInfo();
+        return this;
+    }
+
+    public RowUpdateBuilder clustering(Object... clusteringValues)
+    {
+        assert rowBuilder == null;
+        rowBuilder = updateBuilder.row(clusteringValues);
+        if (noRowMarker)
+            rowBuilder.noPrimaryKeyLivenessInfo();
+        return this;
+    }
+
+    public Mutation build()
+    {
+        return new Mutation(buildUpdate());
+    }
+
+    public PartitionUpdate buildUpdate()
+    {
+        PartitionUpdate update = updateBuilder.build();
+        for (RangeTombstone rt : rts)
+            update.add(rt);
+        return update;
+    }
+
+    private static void deleteRow(PartitionUpdate update, long timestamp, int localDeletionTime, Object... clusteringValues)
+    {
+        assert clusteringValues.length == update.metadata().comparator.size() || (clusteringValues.length == 0 && !update.columns().statics.isEmpty());
+
+        boolean isStatic = clusteringValues.length != update.metadata().comparator.size();
+        Row.Builder builder = BTreeRow.sortedBuilder();
+
+        if (isStatic)
+            builder.newRow(Clustering.STATIC_CLUSTERING);
+        else
+            builder.newRow(clusteringValues.length == 0 ? Clustering.EMPTY : update.metadata().comparator.make(clusteringValues));
+        builder.addRowDeletion(Row.Deletion.regular(new DeletionTime(timestamp, localDeletionTime)));
+
+        update.add(builder.build());
+    }
+
+    public static Mutation deleteRow(CFMetaData metadata, long timestamp, Object key, Object... clusteringValues)
+    {
+        return deleteRowAt(metadata, timestamp, FBUtilities.nowInSeconds(), key, clusteringValues);
+    }
+
+    public static Mutation deleteRowAt(CFMetaData metadata, long timestamp, int localDeletionTime, Object key, Object... clusteringValues)
+    {
+        PartitionUpdate update = new PartitionUpdate(metadata, makeKey(metadata, key), metadata.partitionColumns(), 0);
+        deleteRow(update, timestamp, localDeletionTime, clusteringValues);
+        // note that the created mutation may get further update later on, so we don't use the ctor that create a singletonMap
+        // underneath (this class if for convenience, not performance)
+        return new Mutation(update.metadata().ksName, update.partitionKey()).add(update);
+    }
+
+    private static DecoratedKey makeKey(CFMetaData metadata, Object... partitionKey)
+    {
+        if (partitionKey.length == 1 && partitionKey[0] instanceof DecoratedKey)
+            return (DecoratedKey)partitionKey[0];
+
+        ByteBuffer key = CFMetaData.serializePartitionKey(metadata.getKeyValidatorAsClusteringComparator().make(partitionKey));
+        return metadata.decorateKey(key);
+    }
+
+    public RowUpdateBuilder addRangeTombstone(RangeTombstone rt)
+    {
+        rts.add(rt);
+        return this;
+    }
+
+    public RowUpdateBuilder addRangeTombstone(Object start, Object end)
+    {
+        updateBuilder.addRangeTombstone().start(start).end(end);
+        return this;
+    }
+
+    public RowUpdateBuilder add(String columnName, Object value)
+    {
+        rowBuilder().add(columnName, value);
+        return this;
+    }
+
+    public RowUpdateBuilder add(ColumnDefinition columnDefinition, Object value)
+    {
+        return add(columnDefinition.name.toString(), value);
+    }
+
+    public RowUpdateBuilder delete(String columnName)
+    {
+        rowBuilder().delete(columnName);
+        return this;
+    }
+
+    public RowUpdateBuilder delete(ColumnDefinition columnDefinition)
+    {
+        return delete(columnDefinition.name.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26838063/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 b264553..3dd798d 100644
--- a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
@@ -172,7 +172,7 @@ public class TTLExpiryTest
 
         new RowUpdateBuilder(cfs.metadata, timestamp, 1, key)
             .add("col2", ByteBufferUtil.EMPTY_BYTE_BUFFER)
-            .addMapEntry("col8", "bar", "foo")
+            .add("col8", Collections.singletonMap("bar", "foo"))
             .delete("col1")
             .build()
             .applyUnsafe();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26838063/test/unit/org/apache/cassandra/db/partition/PartitionUpdateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/partition/PartitionUpdateTest.java b/test/unit/org/apache/cassandra/db/partition/PartitionUpdateTest.java
index a069db1..bfa9796 100644
--- a/test/unit/org/apache/cassandra/db/partition/PartitionUpdateTest.java
+++ b/test/unit/org/apache/cassandra/db/partition/PartitionUpdateTest.java
@@ -17,6 +17,7 @@
  */
 package org.apache.cassandra.db.partition;
 
+import org.apache.cassandra.UpdateBuilder;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.db.RowUpdateBuilder;
@@ -34,21 +35,17 @@ public class PartitionUpdateTest extends CQLTester
         createTable("CREATE TABLE %s (key text, clustering int, a int, s int static, PRIMARY KEY(key, clustering))");
         CFMetaData cfm = currentTableMetadata();
 
-        long timestamp = FBUtilities.timestampMicros();
-        PartitionUpdate update = new RowUpdateBuilder(cfm, timestamp, "key0").clustering(1).add("a", 1).buildUpdate();
-        Assert.assertEquals(1, update.operationCount());
-
-        update = new RowUpdateBuilder(cfm, timestamp, "key0").buildUpdate();
-        Assert.assertEquals(0, update.operationCount());
+        UpdateBuilder builder = UpdateBuilder.create(cfm, "key0");
+        Assert.assertEquals(0, builder.build().operationCount());
+        Assert.assertEquals(1, builder.newRow(1).add("a", 1).build().operationCount());
 
-        update = new RowUpdateBuilder(cfm, timestamp, "key0").add("s", 1).buildUpdate();
-        Assert.assertEquals(1, update.operationCount());
+        builder = UpdateBuilder.create(cfm, "key0");
+        Assert.assertEquals(1, builder.newRow().add("s", 1).build().operationCount());
 
-        update = new RowUpdateBuilder(cfm, timestamp, "key0").add("s", 1).buildUpdate();
-        update = new RowUpdateBuilder(update, timestamp, cfm.params.defaultTimeToLive).clustering(1)
-                                                                                      .add("a", 1)
-                                                                                      .buildUpdate();
-        Assert.assertEquals(2, update.operationCount());
+        builder = UpdateBuilder.create(cfm, "key0");
+        builder.newRow().add("s", 1);
+        builder.newRow(1).add("a", 1);
+        Assert.assertEquals(2, builder.build().operationCount());
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26838063/test/unit/org/apache/cassandra/hints/HintTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/hints/HintTest.java b/test/unit/org/apache/cassandra/hints/HintTest.java
index 658a41c..4cc2188 100644
--- a/test/unit/org/apache/cassandra/hints/HintTest.java
+++ b/test/unit/org/apache/cassandra/hints/HintTest.java
@@ -30,9 +30,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.config.*;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.partitions.FilteredPartition;
 import org.apache.cassandra.db.partitions.PartitionIterator;
@@ -127,7 +125,7 @@ public class HintTest
 
         // assert that we can read the inserted partitions
         for (PartitionUpdate partition : mutation.getPartitionUpdates())
-            assertPartitionsEqual(partition, readPartition(key, partition.metadata().cfName));
+            assertPartitionsEqual(partition, readPartition(key, partition.metadata().cfName, partition.columns()));
     }
 
     @Test
@@ -152,8 +150,10 @@ public class HintTest
         assertNoPartitions(key, TABLE1);
 
         // TABLE0 and TABLE2 updates should have been applied successfully
-        assertPartitionsEqual(mutation.getPartitionUpdate(Schema.instance.getId(KEYSPACE, TABLE0)), readPartition(key, TABLE0));
-        assertPartitionsEqual(mutation.getPartitionUpdate(Schema.instance.getId(KEYSPACE, TABLE2)), readPartition(key, TABLE2));
+        PartitionUpdate upd0 = mutation.getPartitionUpdate(Schema.instance.getId(KEYSPACE, TABLE0));
+        assertPartitionsEqual(upd0, readPartition(key, TABLE0, upd0.columns()));
+        PartitionUpdate upd2 = mutation.getPartitionUpdate(Schema.instance.getId(KEYSPACE, TABLE2));
+        assertPartitionsEqual(upd2, readPartition(key, TABLE2, upd2.columns()));
     }
 
     @Test
@@ -296,40 +296,44 @@ public class HintTest
 
     private static Mutation createMutation(String key, long now)
     {
-        Mutation mutation = new Mutation(KEYSPACE, dk(key));
+        Mutation.SimpleBuilder builder = Mutation.simpleBuilder(KEYSPACE, dk(key));
 
-        new RowUpdateBuilder(Schema.instance.getCFMetaData(KEYSPACE, TABLE0), now, mutation)
-            .clustering("column0")
-            .add("val", "value0")
-            .build();
+        builder.update(Schema.instance.getCFMetaData(KEYSPACE, TABLE0))
+               .timestamp(now)
+               .row("column0")
+               .add("val", "value0");
 
-        new RowUpdateBuilder(Schema.instance.getCFMetaData(KEYSPACE, TABLE1), now + 1, mutation)
-            .clustering("column1")
-            .add("val", "value1")
-            .build();
+        builder.update(Schema.instance.getCFMetaData(KEYSPACE, TABLE1))
+               .timestamp(now + 1)
+               .row("column1")
+               .add("val", "value1");
 
-        new RowUpdateBuilder(Schema.instance.getCFMetaData(KEYSPACE, TABLE2), now + 2, mutation)
-            .clustering("column2")
-            .add("val", "value2")
-            .build();
+        builder.update(Schema.instance.getCFMetaData(KEYSPACE, TABLE2))
+               .timestamp(now + 2)
+               .row("column2")
+               .add("val", "value2");
 
-        return mutation;
+        return builder.build();
     }
 
-    private static SinglePartitionReadCommand cmd(String key, String table)
+    private static ColumnFamilyStore cfs(String table)
     {
-        CFMetaData meta = Schema.instance.getCFMetaData(KEYSPACE, table);
-        return SinglePartitionReadCommand.fullPartitionRead(meta, FBUtilities.nowInSeconds(), bytes(key));
+        return Schema.instance.getColumnFamilyStoreInstance(Schema.instance.getCFMetaData(KEYSPACE, table).cfId);
     }
 
-    private static FilteredPartition readPartition(String key, String table)
+    private static FilteredPartition readPartition(String key, String table, PartitionColumns columns)
     {
-        return Util.getOnlyPartition(cmd(key, table));
+        String[] columnNames = new String[columns.size()];
+        int i = 0;
+        for (ColumnDefinition column : columns)
+            columnNames[i++] = column.name.toString();
+
+        return Util.getOnlyPartition(Util.cmd(cfs(table), key).columns(columnNames).build());
     }
 
     private static void assertNoPartitions(String key, String table)
     {
-        ReadCommand cmd = cmd(key, table);
+        ReadCommand cmd = Util.cmd(cfs(table), key).build();
 
         try (ReadExecutionController executionController = cmd.executionController();
              PartitionIterator iterator = cmd.executeInternal(executionController))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26838063/test/unit/org/apache/cassandra/hints/LegacyHintsMigratorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/hints/LegacyHintsMigratorTest.java b/test/unit/org/apache/cassandra/hints/LegacyHintsMigratorTest.java
index cc97df0..87abdac 100644
--- a/test/unit/org/apache/cassandra/hints/LegacyHintsMigratorTest.java
+++ b/test/unit/org/apache/cassandra/hints/LegacyHintsMigratorTest.java
@@ -27,6 +27,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.*;
@@ -159,7 +160,7 @@ public class LegacyHintsMigratorTest
 
                 assertEquals(timestamp, hint.creationTime);
                 assertEquals(ttl, hint.gcgs);
-                assertMutationsEqual(mutation, hint.mutation);
+                assertTrue(mutation + " != " + hint.mutation, Util.sameContent(mutation, hint.mutation));
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26838063/test/unit/org/apache/cassandra/schema/DefsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/schema/DefsTest.java b/test/unit/org/apache/cassandra/schema/DefsTest.java
index e9980f6..d4ac1dc 100644
--- a/test/unit/org/apache/cassandra/schema/DefsTest.java
+++ b/test/unit/org/apache/cassandra/schema/DefsTest.java
@@ -498,7 +498,7 @@ public class DefsTest
     public void testDropIndex() throws ConfigurationException
     {
         // persist keyspace definition in the system keyspace
-        SchemaKeyspace.makeCreateKeyspaceMutation(Schema.instance.getKSMetaData(KEYSPACE6), FBUtilities.timestampMicros()).applyUnsafe();
+        SchemaKeyspace.makeCreateKeyspaceMutation(Schema.instance.getKSMetaData(KEYSPACE6), FBUtilities.timestampMicros()).build().applyUnsafe();
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE6).getColumnFamilyStore(TABLE1i);
         String indexName = "birthdate_key_index";