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 2017/03/28 12:38:31 UTC

[1/3] cassandra git commit: cdc column addition strikes again

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.11 41befde22 -> 8ea0afa3d
  refs/heads/trunk 380a614f1 -> 3bd152e6c


cdc column addition strikes again

patch by Sylvain Lebresne; reviewed by Aleksey Yeschenko for CASSANDRA-13382


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

Branch: refs/heads/cassandra-3.11
Commit: 8ea0afa3dbced228dbe9332bae6b5b2a1760bd8a
Parents: 41befde
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Mon Mar 27 11:27:28 2017 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Mar 28 14:35:00 2017 +0200

----------------------------------------------------------------------
 CHANGES.txt                                              |  1 +
 src/java/org/apache/cassandra/schema/SchemaKeyspace.java | 11 +++++++++--
 2 files changed, 10 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8ea0afa3/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 2122227..c7cc26e 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.11.0
+ * cdc column addition strikes again (CASSANDRA-13382)
  * Fix static column indexes (CASSANDRA-13277) 
  * DataOutputBuffer.asNewBuffer broken (CASSANDRA-13298)
  * unittest CipherFactoryTest failed on MacOS (CASSANDRA-13370)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8ea0afa3/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 58580b9..ff7cf04 100644
--- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
+++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
@@ -26,6 +26,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.MapDifference;
 import com.google.common.collect.Maps;
 import org.slf4j.Logger;
@@ -84,6 +85,12 @@ public final class SchemaKeyspace
     public static final List<String> ALL =
         ImmutableList.of(KEYSPACES, TABLES, COLUMNS, DROPPED_COLUMNS, TRIGGERS, VIEWS, TYPES, FUNCTIONS, AGGREGATES, INDEXES);
 
+    /**
+     * The tables to which we added the cdc column. This is used in {@link #makeUpdateForSchema} below to make sure we skip that
+     * column is cdc is disabled as the columns breaks pre-cdc to post-cdc upgrades (typically, 3.0 -> 3.X).
+     */
+    private static final Set<String> TABLES_WITH_CDC_ADDED = ImmutableSet.of(TABLES, VIEWS);
+
     private static final CFMetaData Keyspaces =
         compile(KEYSPACES,
                 "keyspace definitions",
@@ -385,9 +392,9 @@ public final class SchemaKeyspace
     private static PartitionUpdate makeUpdateForSchema(UnfilteredRowIterator partition, ColumnFilter filter)
     {
         // This method is used during schema migration tasks, and if cdc is disabled, we want to force excluding the
-        // 'cdc' column from the TABLES schema table because it is problematic if received by older nodes (see #12236
+        // 'cdc' column from the TABLES/VIEWS schema table because it is problematic if received by older nodes (see #12236
         // and #12697). Otherwise though, we just simply "buffer" the content of the partition into a PartitionUpdate.
-        if (DatabaseDescriptor.isCDCEnabled() || !partition.metadata().cfName.equals(TABLES))
+        if (DatabaseDescriptor.isCDCEnabled() || !TABLES_WITH_CDC_ADDED.contains(partition.metadata().cfName))
             return PartitionUpdate.fromIterator(partition, filter);
 
         // We want to skip the 'cdc' column. A simple solution for that is based on the fact that


[3/3] cassandra git commit: Merge branch 'cassandra-3.11' into trunk

Posted by sl...@apache.org.
Merge branch 'cassandra-3.11' into trunk

* cassandra-3.11:
  cdc column addition strikes again


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

Branch: refs/heads/trunk
Commit: 3bd152e6cc82f5314ec14ea854ebbf6ad602609f
Parents: 380a614 8ea0afa
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Tue Mar 28 14:38:17 2017 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Mar 28 14:38:17 2017 +0200

----------------------------------------------------------------------
 CHANGES.txt                                              |  1 +
 src/java/org/apache/cassandra/schema/SchemaKeyspace.java | 11 +++++++++--
 2 files changed, 10 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/3bd152e6/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index a9f0c06,c7cc26e..7ec782c
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,52 -1,5 +1,53 @@@
 +4.0
 + * Outbound TCP connections ignore internode authenticator (CASSANDRA-13324)
 + * Upgrade junit from 4.6 to 4.12 (CASSANDRA-13360)
 + * Cleanup ParentRepairSession after repairs (CASSANDRA-13359)
 + * Incremental repair not streaming correct sstables (CASSANDRA-13328)
 + * Upgrade the jna version to 4.3.0 (CASSANDRA-13300)
 + * Add the currentTimestamp, currentDate, currentTime and currentTimeUUID functions (CASSANDRA-13132)
 + * Remove config option index_interval (CASSANDRA-10671)
 + * Reduce lock contention for collection types and serializers (CASSANDRA-13271)
 + * Make it possible to override MessagingService.Verb ids (CASSANDRA-13283)
 + * Avoid synchronized on prepareForRepair in ActiveRepairService (CASSANDRA-9292)
 + * Adds the ability to use uncompressed chunks in compressed files (CASSANDRA-10520)
 + * Don't flush sstables when streaming for incremental repair (CASSANDRA-13226)
 + * Remove unused method (CASSANDRA-13227)
 + * Fix minor bugs related to #9143 (CASSANDRA-13217)
 + * Output warning if user increases RF (CASSANDRA-13079)
 + * Remove pre-3.0 streaming compatibility code for 4.0 (CASSANDRA-13081)
 + * Add support for + and - operations on dates (CASSANDRA-11936)
 + * Fix consistency of incrementally repaired data (CASSANDRA-9143)
 + * Increase commitlog version (CASSANDRA-13161)
 + * Make TableMetadata immutable, optimize Schema (CASSANDRA-9425)
 + * Refactor ColumnCondition (CASSANDRA-12981)
 + * Parallelize streaming of different keyspaces (CASSANDRA-4663)
 + * Improved compactions metrics (CASSANDRA-13015)
 + * Speed-up start-up sequence by avoiding un-needed flushes (CASSANDRA-13031)
 + * Use Caffeine (W-TinyLFU) for on-heap caches (CASSANDRA-10855)
 + * Thrift removal (CASSANDRA-11115)
 + * Remove pre-3.0 compatibility code for 4.0 (CASSANDRA-12716)
 + * Add column definition kind to dropped columns in schema (CASSANDRA-12705)
 + * Add (automate) Nodetool Documentation (CASSANDRA-12672)
 + * Update bundled cqlsh python driver to 3.7.0 (CASSANDRA-12736)
 + * Reject invalid replication settings when creating or altering a keyspace (CASSANDRA-12681)
 + * Clean up the SSTableReader#getScanner API wrt removal of RateLimiter (CASSANDRA-12422)
 + * Use new token allocation for non bootstrap case as well (CASSANDRA-13080)
 + * Avoid byte-array copy when key cache is disabled (CASSANDRA-13084)
 + * Require forceful decommission if number of nodes is less than replication factor (CASSANDRA-12510)
 + * Allow IN restrictions on column families with collections (CASSANDRA-12654)
 + * Log message size in trace message in OutboundTcpConnection (CASSANDRA-13028)
 + * Add timeUnit Days for cassandra-stress (CASSANDRA-13029)
 + * Add mutation size and batch metrics (CASSANDRA-12649)
 + * Add method to get size of endpoints to TokenMetadata (CASSANDRA-12999)
 + * Expose time spent waiting in thread pool queue (CASSANDRA-8398)
 + * Conditionally update index built status to avoid unnecessary flushes (CASSANDRA-12969)
 + * cqlsh auto completion: refactor definition of compaction strategy options (CASSANDRA-12946)
 + * Add support for arithmetic operators (CASSANDRA-11935)
 + * Add histogram for delay to deliver hints (CASSANDRA-13234)
 +
 +
  3.11.0
+  * cdc column addition strikes again (CASSANDRA-13382)
   * Fix static column indexes (CASSANDRA-13277) 
   * DataOutputBuffer.asNewBuffer broken (CASSANDRA-13298)
   * unittest CipherFactoryTest failed on MacOS (CASSANDRA-13370)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3bd152e6/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/schema/SchemaKeyspace.java
index 5e5ba34,ff7cf04..47684ba
--- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
+++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
@@@ -23,8 -23,10 +23,9 @@@ import java.security.MessageDigest
  import java.security.NoSuchAlgorithmException;
  import java.util.*;
  import java.util.concurrent.TimeUnit;
 -import java.util.stream.Collectors;
  
  import com.google.common.collect.ImmutableList;
+ import com.google.common.collect.ImmutableSet;
  import com.google.common.collect.MapDifference;
  import com.google.common.collect.Maps;
  import org.slf4j.Logger;
@@@ -82,167 -85,168 +83,173 @@@ public final class SchemaKeyspac
      public static final List<String> ALL =
          ImmutableList.of(KEYSPACES, TABLES, COLUMNS, DROPPED_COLUMNS, TRIGGERS, VIEWS, TYPES, FUNCTIONS, AGGREGATES, INDEXES);
  
+     /**
+      * The tables to which we added the cdc column. This is used in {@link #makeUpdateForSchema} below to make sure we skip that
+      * column is cdc is disabled as the columns breaks pre-cdc to post-cdc upgrades (typically, 3.0 -> 3.X).
+      */
+     private static final Set<String> TABLES_WITH_CDC_ADDED = ImmutableSet.of(TABLES, VIEWS);
+ 
 -    private static final CFMetaData Keyspaces =
 -        compile(KEYSPACES,
 -                "keyspace definitions",
 -                "CREATE TABLE %s ("
 -                + "keyspace_name text,"
 -                + "durable_writes boolean,"
 -                + "replication frozen<map<text, text>>,"
 -                + "PRIMARY KEY ((keyspace_name)))");
 -
 -    private static final CFMetaData Tables =
 -        compile(TABLES,
 -                "table definitions",
 -                "CREATE TABLE %s ("
 -                + "keyspace_name text,"
 -                + "table_name text,"
 -                + "bloom_filter_fp_chance double,"
 -                + "caching frozen<map<text, text>>,"
 -                + "comment text,"
 -                + "compaction frozen<map<text, text>>,"
 -                + "compression frozen<map<text, text>>,"
 -                + "crc_check_chance double,"
 -                + "dclocal_read_repair_chance double,"
 -                + "default_time_to_live int,"
 -                + "extensions frozen<map<text, blob>>,"
 -                + "flags frozen<set<text>>," // SUPER, COUNTER, DENSE, COMPOUND
 -                + "gc_grace_seconds int,"
 -                + "id uuid,"
 -                + "max_index_interval int,"
 -                + "memtable_flush_period_in_ms int,"
 -                + "min_index_interval int,"
 -                + "read_repair_chance double,"
 -                + "speculative_retry text,"
 -                + "cdc boolean,"
 -                + "PRIMARY KEY ((keyspace_name), table_name))");
 -
 -    private static final CFMetaData Columns =
 -        compile(COLUMNS,
 -                "column definitions",
 -                "CREATE TABLE %s ("
 -                + "keyspace_name text,"
 -                + "table_name text,"
 -                + "column_name text,"
 -                + "clustering_order text,"
 -                + "column_name_bytes blob,"
 -                + "kind text,"
 -                + "position int,"
 -                + "type text,"
 -                + "PRIMARY KEY ((keyspace_name), table_name, column_name))");
 -
 -    private static final CFMetaData DroppedColumns =
 -        compile(DROPPED_COLUMNS,
 -                "dropped column registry",
 -                "CREATE TABLE %s ("
 -                + "keyspace_name text,"
 -                + "table_name text,"
 -                + "column_name text,"
 -                + "dropped_time timestamp,"
 -                + "type text,"
 -                + "PRIMARY KEY ((keyspace_name), table_name, column_name))");
 -
 -    private static final CFMetaData Triggers =
 -        compile(TRIGGERS,
 -                "trigger definitions",
 -                "CREATE TABLE %s ("
 -                + "keyspace_name text,"
 -                + "table_name text,"
 -                + "trigger_name text,"
 -                + "options frozen<map<text, text>>,"
 -                + "PRIMARY KEY ((keyspace_name), table_name, trigger_name))");
 -
 -    private static final CFMetaData Views =
 -        compile(VIEWS,
 -                "view definitions",
 -                "CREATE TABLE %s ("
 -                + "keyspace_name text,"
 -                + "view_name text,"
 -                + "base_table_id uuid,"
 -                + "base_table_name text,"
 -                + "where_clause text,"
 -                + "bloom_filter_fp_chance double,"
 -                + "caching frozen<map<text, text>>,"
 -                + "comment text,"
 -                + "compaction frozen<map<text, text>>,"
 -                + "compression frozen<map<text, text>>,"
 -                + "crc_check_chance double,"
 -                + "dclocal_read_repair_chance double,"
 -                + "default_time_to_live int,"
 -                + "extensions frozen<map<text, blob>>,"
 -                + "gc_grace_seconds int,"
 -                + "id uuid,"
 -                + "include_all_columns boolean,"
 -                + "max_index_interval int,"
 -                + "memtable_flush_period_in_ms int,"
 -                + "min_index_interval int,"
 -                + "read_repair_chance double,"
 -                + "speculative_retry text,"
 -                + "cdc boolean,"
 -                + "PRIMARY KEY ((keyspace_name), view_name))");
 -
 -    private static final CFMetaData Indexes =
 -        compile(INDEXES,
 -                "secondary index definitions",
 -                "CREATE TABLE %s ("
 -                + "keyspace_name text,"
 -                + "table_name text,"
 -                + "index_name text,"
 -                + "kind text,"
 -                + "options frozen<map<text, text>>,"
 -                + "PRIMARY KEY ((keyspace_name), table_name, index_name))");
 -
 -    private static final CFMetaData Types =
 -        compile(TYPES,
 -                "user defined type definitions",
 -                "CREATE TABLE %s ("
 -                + "keyspace_name text,"
 -                + "type_name text,"
 -                + "field_names frozen<list<text>>,"
 -                + "field_types frozen<list<text>>,"
 -                + "PRIMARY KEY ((keyspace_name), type_name))");
 -
 -    private static final CFMetaData Functions =
 -        compile(FUNCTIONS,
 -                "user defined function definitions",
 -                "CREATE TABLE %s ("
 -                + "keyspace_name text,"
 -                + "function_name text,"
 -                + "argument_types frozen<list<text>>,"
 -                + "argument_names frozen<list<text>>,"
 -                + "body text,"
 -                + "language text,"
 -                + "return_type text,"
 -                + "called_on_null_input boolean,"
 -                + "PRIMARY KEY ((keyspace_name), function_name, argument_types))");
 -
 -    private static final CFMetaData Aggregates =
 -        compile(AGGREGATES,
 -                "user defined aggregate definitions",
 -                "CREATE TABLE %s ("
 -                + "keyspace_name text,"
 -                + "aggregate_name text,"
 -                + "argument_types frozen<list<text>>,"
 -                + "final_func text,"
 -                + "initcond text,"
 -                + "return_type text,"
 -                + "state_func text,"
 -                + "state_type text,"
 -                + "PRIMARY KEY ((keyspace_name), aggregate_name, argument_types))");
 -
 -    public static final List<CFMetaData> ALL_TABLE_METADATA =
 +    private static final TableMetadata Keyspaces =
 +        parse(KEYSPACES,
 +              "keyspace definitions",
 +              "CREATE TABLE %s ("
 +              + "keyspace_name text,"
 +              + "durable_writes boolean,"
 +              + "replication frozen<map<text, text>>,"
 +              + "PRIMARY KEY ((keyspace_name)))");
 +
 +    private static final TableMetadata Tables =
 +        parse(TABLES,
 +              "table definitions",
 +              "CREATE TABLE %s ("
 +              + "keyspace_name text,"
 +              + "table_name text,"
 +              + "bloom_filter_fp_chance double,"
 +              + "caching frozen<map<text, text>>,"
 +              + "comment text,"
 +              + "compaction frozen<map<text, text>>,"
 +              + "compression frozen<map<text, text>>,"
 +              + "crc_check_chance double,"
 +              + "dclocal_read_repair_chance double,"
 +              + "default_time_to_live int,"
 +              + "extensions frozen<map<text, blob>>,"
 +              + "flags frozen<set<text>>," // SUPER, COUNTER, DENSE, COMPOUND
 +              + "gc_grace_seconds int,"
 +              + "id uuid,"
 +              + "max_index_interval int,"
 +              + "memtable_flush_period_in_ms int,"
 +              + "min_index_interval int,"
 +              + "read_repair_chance double,"
 +              + "speculative_retry text,"
 +              + "cdc boolean,"
 +              + "PRIMARY KEY ((keyspace_name), table_name))");
 +
 +    private static final TableMetadata Columns =
 +        parse(COLUMNS,
 +              "column definitions",
 +              "CREATE TABLE %s ("
 +              + "keyspace_name text,"
 +              + "table_name text,"
 +              + "column_name text,"
 +              + "clustering_order text,"
 +              + "column_name_bytes blob,"
 +              + "kind text,"
 +              + "position int,"
 +              + "type text,"
 +              + "PRIMARY KEY ((keyspace_name), table_name, column_name))");
 +
 +    private static final TableMetadata DroppedColumns =
 +        parse(DROPPED_COLUMNS,
 +              "dropped column registry",
 +              "CREATE TABLE %s ("
 +              + "keyspace_name text,"
 +              + "table_name text,"
 +              + "column_name text,"
 +              + "dropped_time timestamp,"
 +              + "type text,"
 +              + "kind text,"
 +              + "PRIMARY KEY ((keyspace_name), table_name, column_name))");
 +
 +    private static final TableMetadata Triggers =
 +        parse(TRIGGERS,
 +              "trigger definitions",
 +              "CREATE TABLE %s ("
 +              + "keyspace_name text,"
 +              + "table_name text,"
 +              + "trigger_name text,"
 +              + "options frozen<map<text, text>>,"
 +              + "PRIMARY KEY ((keyspace_name), table_name, trigger_name))");
 +
 +    private static final TableMetadata Views =
 +        parse(VIEWS,
 +              "view definitions",
 +              "CREATE TABLE %s ("
 +              + "keyspace_name text,"
 +              + "view_name text,"
 +              + "base_table_id uuid,"
 +              + "base_table_name text,"
 +              + "where_clause text,"
 +              + "bloom_filter_fp_chance double,"
 +              + "caching frozen<map<text, text>>,"
 +              + "comment text,"
 +              + "compaction frozen<map<text, text>>,"
 +              + "compression frozen<map<text, text>>,"
 +              + "crc_check_chance double,"
 +              + "dclocal_read_repair_chance double,"
 +              + "default_time_to_live int,"
 +              + "extensions frozen<map<text, blob>>,"
 +              + "gc_grace_seconds int,"
 +              + "id uuid,"
 +              + "include_all_columns boolean,"
 +              + "max_index_interval int,"
 +              + "memtable_flush_period_in_ms int,"
 +              + "min_index_interval int,"
 +              + "read_repair_chance double,"
 +              + "speculative_retry text,"
 +              + "cdc boolean,"
 +              + "PRIMARY KEY ((keyspace_name), view_name))");
 +
 +    private static final TableMetadata Indexes =
 +        parse(INDEXES,
 +              "secondary index definitions",
 +              "CREATE TABLE %s ("
 +              + "keyspace_name text,"
 +              + "table_name text,"
 +              + "index_name text,"
 +              + "kind text,"
 +              + "options frozen<map<text, text>>,"
 +              + "PRIMARY KEY ((keyspace_name), table_name, index_name))");
 +
 +    private static final TableMetadata Types =
 +        parse(TYPES,
 +              "user defined type definitions",
 +              "CREATE TABLE %s ("
 +              + "keyspace_name text,"
 +              + "type_name text,"
 +              + "field_names frozen<list<text>>,"
 +              + "field_types frozen<list<text>>,"
 +              + "PRIMARY KEY ((keyspace_name), type_name))");
 +
 +    private static final TableMetadata Functions =
 +        parse(FUNCTIONS,
 +              "user defined function definitions",
 +              "CREATE TABLE %s ("
 +              + "keyspace_name text,"
 +              + "function_name text,"
 +              + "argument_types frozen<list<text>>,"
 +              + "argument_names frozen<list<text>>,"
 +              + "body text,"
 +              + "language text,"
 +              + "return_type text,"
 +              + "called_on_null_input boolean,"
 +              + "PRIMARY KEY ((keyspace_name), function_name, argument_types))");
 +
 +    private static final TableMetadata Aggregates =
 +        parse(AGGREGATES,
 +              "user defined aggregate definitions",
 +              "CREATE TABLE %s ("
 +              + "keyspace_name text,"
 +              + "aggregate_name text,"
 +              + "argument_types frozen<list<text>>,"
 +              + "final_func text,"
 +              + "initcond text,"
 +              + "return_type text,"
 +              + "state_func text,"
 +              + "state_type text,"
 +              + "PRIMARY KEY ((keyspace_name), aggregate_name, argument_types))");
 +
 +    private static final List<TableMetadata> ALL_TABLE_METADATA =
          ImmutableList.of(Keyspaces, Tables, Columns, Triggers, DroppedColumns, Views, Types, Functions, Aggregates, Indexes);
  
 -    private static CFMetaData compile(String name, String description, String schema)
 +    private static TableMetadata parse(String name, String description, String cql)
      {
 -        return CFMetaData.compile(String.format(schema, name), SchemaConstants.SCHEMA_KEYSPACE_NAME)
 -                         .comment(description)
 -                         .gcGraceSeconds((int) TimeUnit.DAYS.toSeconds(7));
 +        return CreateTableStatement.parse(format(cql, name), SchemaConstants.SCHEMA_KEYSPACE_NAME)
 +                                   .id(TableId.forSystemTable(SchemaConstants.SCHEMA_KEYSPACE_NAME, name))
 +                                   .dcLocalReadRepairChance(0.0)
 +                                   .gcGraceSeconds((int) TimeUnit.DAYS.toSeconds(7))
 +                                   .memtableFlushPeriod((int) TimeUnit.HOURS.toMillis(1))
 +                                   .comment(description)
 +                                   .build();
      }
  
      public static KeyspaceMetadata metadata()
@@@ -383,9 -392,9 +390,9 @@@
      private static PartitionUpdate makeUpdateForSchema(UnfilteredRowIterator partition, ColumnFilter filter)
      {
          // This method is used during schema migration tasks, and if cdc is disabled, we want to force excluding the
-         // 'cdc' column from the TABLES schema table because it is problematic if received by older nodes (see #12236
+         // 'cdc' column from the TABLES/VIEWS schema table because it is problematic if received by older nodes (see #12236
          // and #12697). Otherwise though, we just simply "buffer" the content of the partition into a PartitionUpdate.
-         if (DatabaseDescriptor.isCDCEnabled() || !partition.metadata().name.equals(TABLES))
 -        if (DatabaseDescriptor.isCDCEnabled() || !TABLES_WITH_CDC_ADDED.contains(partition.metadata().cfName))
++        if (DatabaseDescriptor.isCDCEnabled() || !TABLES_WITH_CDC_ADDED.contains(partition.metadata().name))
              return PartitionUpdate.fromIterator(partition, filter);
  
          // We want to skip the 'cdc' column. A simple solution for that is based on the fact that


[2/3] cassandra git commit: cdc column addition strikes again

Posted by sl...@apache.org.
cdc column addition strikes again

patch by Sylvain Lebresne; reviewed by Aleksey Yeschenko for CASSANDRA-13382


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

Branch: refs/heads/trunk
Commit: 8ea0afa3dbced228dbe9332bae6b5b2a1760bd8a
Parents: 41befde
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Mon Mar 27 11:27:28 2017 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Mar 28 14:35:00 2017 +0200

----------------------------------------------------------------------
 CHANGES.txt                                              |  1 +
 src/java/org/apache/cassandra/schema/SchemaKeyspace.java | 11 +++++++++--
 2 files changed, 10 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8ea0afa3/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 2122227..c7cc26e 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.11.0
+ * cdc column addition strikes again (CASSANDRA-13382)
  * Fix static column indexes (CASSANDRA-13277) 
  * DataOutputBuffer.asNewBuffer broken (CASSANDRA-13298)
  * unittest CipherFactoryTest failed on MacOS (CASSANDRA-13370)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8ea0afa3/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 58580b9..ff7cf04 100644
--- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
+++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
@@ -26,6 +26,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.MapDifference;
 import com.google.common.collect.Maps;
 import org.slf4j.Logger;
@@ -84,6 +85,12 @@ public final class SchemaKeyspace
     public static final List<String> ALL =
         ImmutableList.of(KEYSPACES, TABLES, COLUMNS, DROPPED_COLUMNS, TRIGGERS, VIEWS, TYPES, FUNCTIONS, AGGREGATES, INDEXES);
 
+    /**
+     * The tables to which we added the cdc column. This is used in {@link #makeUpdateForSchema} below to make sure we skip that
+     * column is cdc is disabled as the columns breaks pre-cdc to post-cdc upgrades (typically, 3.0 -> 3.X).
+     */
+    private static final Set<String> TABLES_WITH_CDC_ADDED = ImmutableSet.of(TABLES, VIEWS);
+
     private static final CFMetaData Keyspaces =
         compile(KEYSPACES,
                 "keyspace definitions",
@@ -385,9 +392,9 @@ public final class SchemaKeyspace
     private static PartitionUpdate makeUpdateForSchema(UnfilteredRowIterator partition, ColumnFilter filter)
     {
         // This method is used during schema migration tasks, and if cdc is disabled, we want to force excluding the
-        // 'cdc' column from the TABLES schema table because it is problematic if received by older nodes (see #12236
+        // 'cdc' column from the TABLES/VIEWS schema table because it is problematic if received by older nodes (see #12236
         // and #12697). Otherwise though, we just simply "buffer" the content of the partition into a PartitionUpdate.
-        if (DatabaseDescriptor.isCDCEnabled() || !partition.metadata().cfName.equals(TABLES))
+        if (DatabaseDescriptor.isCDCEnabled() || !TABLES_WITH_CDC_ADDED.contains(partition.metadata().cfName))
             return PartitionUpdate.fromIterator(partition, filter);
 
         // We want to skip the 'cdc' column. A simple solution for that is based on the fact that