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:09 UTC

[03/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/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java b/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java
index 2de671c..72441cd 100644
--- a/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java
+++ b/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java
@@ -34,6 +34,7 @@ import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.FieldIdentifier;
 import org.apache.cassandra.cql3.functions.*;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.index.TargetParser;
 import org.apache.cassandra.thrift.ThriftConversion;
@@ -564,35 +565,40 @@ public class LegacySchemaMigratorTest
         setLegacyIndexStatus(keyspace);
     }
 
-    private static Mutation makeLegacyCreateKeyspaceMutation(KeyspaceMetadata keyspace, long timestamp)
+    private static DecoratedKey decorate(CFMetaData metadata, Object value)
     {
-        // Note that because Keyspaces is a COMPACT TABLE, we're really only setting static columns internally and shouldn't set any clustering.
-        RowUpdateBuilder adder = new RowUpdateBuilder(SystemKeyspace.LegacyKeyspaces, timestamp, keyspace.name);
+        return metadata.decorateKey(((AbstractType)metadata.getKeyValidator()).decompose(value));
+    }
 
-        adder.add("durable_writes", keyspace.params.durableWrites)
-             .add("strategy_class", keyspace.params.replication.klass.getName())
-             .add("strategy_options", json(keyspace.params.replication.options));
+    private static Mutation makeLegacyCreateKeyspaceMutation(KeyspaceMetadata keyspace, long timestamp)
+    {
+        Mutation.SimpleBuilder builder = Mutation.simpleBuilder(SystemKeyspace.NAME, decorate(SystemKeyspace.LegacyKeyspaces, keyspace.name))
+                                                 .timestamp(timestamp);
 
-        Mutation mutation = adder.build();
+        builder.update(SystemKeyspace.LegacyKeyspaces)
+               .row()
+               .add("durable_writes", keyspace.params.durableWrites)
+               .add("strategy_class", keyspace.params.replication.klass.getName())
+               .add("strategy_options", json(keyspace.params.replication.options));
 
-        keyspace.tables.forEach(table -> addTableToSchemaMutation(table, 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));
+        keyspace.tables.forEach(table -> addTableToSchemaMutation(table, 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.build();
     }
 
     /*
      * Serializing tables
      */
 
-    private static void addTableToSchemaMutation(CFMetaData table, long timestamp, boolean withColumnsAndTriggers, Mutation mutation)
+    private static void addTableToSchemaMutation(CFMetaData table, boolean withColumnsAndTriggers, Mutation.SimpleBuilder builder)
     {
         // For property that can be null (and can be changed), we insert tombstones, to make sure
         // we don't keep a property the user has removed
-        RowUpdateBuilder adder = new RowUpdateBuilder(SystemKeyspace.LegacyColumnfamilies, timestamp, mutation)
-                                 .clustering(table.cfName);
+        Row.SimpleBuilder adder = builder.update(SystemKeyspace.LegacyColumnfamilies)
+                                         .row(table.cfName);
 
         adder.add("cf_id", table.cfId)
              .add("type", table.isSuper() ? "Super" : "Standard");
@@ -625,12 +631,14 @@ public class LegacySchemaMigratorTest
              .add("read_repair_chance", table.params.readRepairChance)
              .add("speculative_retry", table.params.speculativeRetry.toString());
 
+        Map<String, Long> dropped = new HashMap<>();
         for (Map.Entry<ByteBuffer, CFMetaData.DroppedColumn> entry : table.getDroppedColumns().entrySet())
         {
             String name = UTF8Type.instance.getString(entry.getKey());
             CFMetaData.DroppedColumn column = entry.getValue();
-            adder.addMapEntry("dropped_columns", name, column.droppedTime);
+            dropped.put(name, column.droppedTime);
         }
+        adder.add("dropped_columns", dropped);
 
         adder.add("is_dense", table.isDense());
 
@@ -639,13 +647,11 @@ public class LegacySchemaMigratorTest
         if (withColumnsAndTriggers)
         {
             for (ColumnDefinition column : table.allColumns())
-                addColumnToSchemaMutation(table, column, timestamp, mutation);
+                addColumnToSchemaMutation(table, column, builder);
 
             for (TriggerMetadata trigger : table.getTriggers())
-                addTriggerToSchemaMutation(table, trigger, timestamp, mutation);
+                addTriggerToSchemaMutation(table, trigger, builder);
         }
-
-        adder.build();
     }
 
     private static String cachingToString(CachingParams caching)
@@ -655,14 +661,14 @@ public class LegacySchemaMigratorTest
                       caching.rowsPerPartitionAsString());
     }
 
-    private static void addColumnToSchemaMutation(CFMetaData table, ColumnDefinition column, long timestamp, Mutation mutation)
+    private static void addColumnToSchemaMutation(CFMetaData table, ColumnDefinition column, Mutation.SimpleBuilder builder)
     {
         // We need to special case pk-only dense tables. See CASSANDRA-9874.
         String name = table.isDense() && column.kind == ColumnDefinition.Kind.REGULAR && column.type instanceof EmptyType
                     ? ""
                     : column.name.toString();
 
-        final RowUpdateBuilder adder = new RowUpdateBuilder(SystemKeyspace.LegacyColumns, timestamp, mutation).clustering(table.cfName, name);
+        final Row.SimpleBuilder adder = builder.update(SystemKeyspace.LegacyColumns).row(table.cfName, name);
 
         adder.add("validator", column.type.toString())
              .add("type", serializeKind(column.kind, table.isDense()))
@@ -682,8 +688,6 @@ public class LegacySchemaMigratorTest
             adder.add("index_type", null);
             adder.add("index_options", null);
         }
-
-        adder.build();
     }
 
     private static Optional<IndexMetadata> findIndexForColumn(Indexes indexes,
@@ -712,71 +716,67 @@ public class LegacySchemaMigratorTest
         return kind.toString().toLowerCase();
     }
 
-    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(SystemKeyspace.LegacyTriggers, timestamp, mutation)
-            .clustering(table.cfName, trigger.name)
-            .addMapEntry("trigger_options", "class", trigger.classOption)
-            .build();
+        builder.update(SystemKeyspace.LegacyTriggers)
+               .row(table.cfName, trigger.name)
+               .add("trigger_options", Collections.singletonMap("class", trigger.classOption));
     }
 
     /*
      * Serializing types
      */
 
-    private static void addTypeToSchemaMutation(UserType type, long timestamp, Mutation mutation)
+    private static void addTypeToSchemaMutation(UserType type, Mutation.SimpleBuilder builder)
     {
-        RowUpdateBuilder adder = new RowUpdateBuilder(SystemKeyspace.LegacyUsertypes, timestamp, mutation)
-                                 .clustering(type.getNameAsString());
-
-        adder.resetCollection("field_names")
-             .resetCollection("field_types");
+        Row.SimpleBuilder adder = builder.update(SystemKeyspace.LegacyUsertypes)
+                                         .row(type.getNameAsString());
 
+        List<String> names = new ArrayList<>();
+        List<String> types = new ArrayList<>();
         for (int i = 0; i < type.size(); i++)
         {
-            adder.addListEntry("field_names", type.fieldName(i).toString())
-                 .addListEntry("field_types", type.fieldType(i).toString());
+            names.add(type.fieldName(i).toString());
+            types.add(type.fieldType(i).toString());
         }
 
-        adder.build();
+        adder.add("field_names", names)
+             .add("field_types", types);
     }
 
     /*
      * Serializing functions
      */
 
-    private static void addFunctionToSchemaMutation(UDFunction function, long timestamp, Mutation mutation)
+    private static void addFunctionToSchemaMutation(UDFunction function, Mutation.SimpleBuilder builder)
     {
-        RowUpdateBuilder adder = new RowUpdateBuilder(SystemKeyspace.LegacyFunctions, timestamp, mutation)
-                                 .clustering(function.name().name, functionSignatureWithTypes(function));
+        Row.SimpleBuilder adder = builder.update(SystemKeyspace.LegacyFunctions)
+                                         .row(function.name().name, functionSignatureWithTypes(function));
 
         adder.add("body", function.body())
              .add("language", function.language())
              .add("return_type", function.returnType().toString())
              .add("called_on_null_input", function.isCalledOnNullInput());
 
-        adder.resetCollection("argument_names")
-             .resetCollection("argument_types");
-
+        List<ByteBuffer> names = new ArrayList<>();
+        List<String> types = new ArrayList<>();
         for (int i = 0; i < function.argNames().size(); i++)
         {
-            adder.addListEntry("argument_names", function.argNames().get(i).bytes)
-                 .addListEntry("argument_types", function.argTypes().get(i).toString());
+            names.add(function.argNames().get(i).bytes);
+            types.add(function.argTypes().get(i).toString());
         }
-
-        adder.build();
+        adder.add("argument_names", names)
+             .add("argument_types", types);
     }
 
     /*
      * Serializing aggregates
      */
 
-    private static void addAggregateToSchemaMutation(UDAggregate aggregate, long timestamp, Mutation mutation)
+    private static void addAggregateToSchemaMutation(UDAggregate aggregate, Mutation.SimpleBuilder builder)
     {
-        RowUpdateBuilder adder = new RowUpdateBuilder(SystemKeyspace.LegacyAggregates, timestamp, mutation)
-                                 .clustering(aggregate.name().name, functionSignatureWithTypes(aggregate));
-
-        adder.resetCollection("argument_types");
+        Row.SimpleBuilder adder = builder.update(SystemKeyspace.LegacyAggregates)
+                                 .row(aggregate.name().name, functionSignatureWithTypes(aggregate));
 
         adder.add("return_type", aggregate.returnType().toString())
              .add("state_func", aggregate.stateFunction().name().name);
@@ -788,10 +788,11 @@ public class LegacySchemaMigratorTest
         if (aggregate.initialCondition() != null)
             adder.add("initcond", aggregate.initialCondition());
 
+        List<String> types = new ArrayList<>();
         for (AbstractType<?> argType : aggregate.argTypes())
-            adder.addListEntry("argument_types", argType.toString());
+            types.add(argType.toString());
 
-        adder.build();
+        adder.add("argument_types", types);
     }
 
     // We allow method overloads, so a function is not uniquely identified by its name only, but

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26838063/test/unit/org/apache/cassandra/schema/SchemaKeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/schema/SchemaKeyspaceTest.java b/test/unit/org/apache/cassandra/schema/SchemaKeyspaceTest.java
index b2e3535..d686fdb 100644
--- a/test/unit/org/apache/cassandra/schema/SchemaKeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/schema/SchemaKeyspaceTest.java
@@ -164,7 +164,7 @@ public class SchemaKeyspaceTest
     private static void updateTable(String keyspace, CFMetaData oldTable, CFMetaData newTable)
     {
         KeyspaceMetadata ksm = Schema.instance.getKeyspaceInstance(keyspace).getMetadata();
-        Mutation mutation = SchemaKeyspace.makeUpdateTableMutation(ksm, oldTable, newTable, FBUtilities.timestampMicros());
+        Mutation mutation = SchemaKeyspace.makeUpdateTableMutation(ksm, oldTable, newTable, FBUtilities.timestampMicros()).build();
         SchemaKeyspace.mergeSchema(Collections.singleton(mutation));
     }
 
@@ -173,7 +173,7 @@ public class SchemaKeyspaceTest
         CFMetaData table = CFMetaData.compile(cql, keyspace);
 
         KeyspaceMetadata ksm = KeyspaceMetadata.create(keyspace, KeyspaceParams.simple(1), Tables.of(table));
-        Mutation mutation = SchemaKeyspace.makeCreateTableMutation(ksm, table, FBUtilities.timestampMicros());
+        Mutation mutation = SchemaKeyspace.makeCreateTableMutation(ksm, table, FBUtilities.timestampMicros()).build();
         SchemaKeyspace.mergeSchema(Collections.singleton(mutation));
     }
 
@@ -187,7 +187,7 @@ public class SchemaKeyspaceTest
         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 serializedCf = rm.getPartitionUpdate(Schema.instance.getId(SchemaKeyspace.NAME, SchemaKeyspace.TABLES));
         PartitionUpdate serializedCD = 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/service/DataResolverTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/DataResolverTest.java b/test/unit/org/apache/cassandra/service/DataResolverTest.java
index b20dfc0..b48512f 100644
--- a/test/unit/org/apache/cassandra/service/DataResolverTest.java
+++ b/test/unit/org/apache/cassandra/service/DataResolverTest.java
@@ -245,7 +245,7 @@ public class DataResolverTest
 
         RangeTombstone tombstone1 = tombstone("1", "11", 1, nowInSec);
         RangeTombstone tombstone2 = tombstone("3", "31", 1, nowInSec);
-        PartitionUpdate update =new RowUpdateBuilder(cfm, nowInSec, 1L, dk).addRangeTombstone(tombstone1)
+        PartitionUpdate update = new RowUpdateBuilder(cfm, nowInSec, 1L, dk).addRangeTombstone(tombstone1)
                                                                                   .addRangeTombstone(tombstone2)
                                                                                   .buildUpdate();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26838063/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
index 6be880c..879f44c 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
@@ -363,7 +363,7 @@ public class StreamingTransferTest
 
 
         updates = new RowUpdateBuilder(cfs.metadata, FBUtilities.timestampMicros() + 1, key);
-        updates.addRangeTombstone(Slice.make(comparator.make(5), comparator.make(7)))
+        updates.addRangeTombstone(5, 7)
                 .build()
                 .apply();