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 2014/08/18 12:09:55 UTC

git commit: Fix dropping collection column when it's the last regular column

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.0 700e81634 -> c44526b29


Fix dropping collection column when it's the last regular column

patch by slebresne; reviewed by iamaleksey for CASSANDRA-7744


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

Branch: refs/heads/cassandra-2.0
Commit: c44526b2961fb9d91ceab17c1f32b2bf58af186d
Parents: 700e816
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Mon Aug 18 12:09:25 2014 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Mon Aug 18 12:09:25 2014 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/config/CFMetaData.java | 38 ++++++++++++++------
 .../cql3/statements/CreateTableStatement.java   |  9 ++++-
 3 files changed, 37 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c44526b2/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f489702..6b610d7 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.0.10
+ * Fix dropping collection when it's the last regular column (CASSANDRA-7744)
  * Properly reject operations on list index with conditions (CASSANDRA-7499)
  * (Hadoop) allow ACFRW to limit nodes to local DC (CASSANDRA-7252)
  * (cqlsh) Wait up to 10 sec for a tracing session (CASSANDRA-7222)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c44526b2/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java
index 9db1c75..296ecce 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -114,6 +114,7 @@ public final class CFMetaData
                                                                     + "keyspace_name text,"
                                                                     + "columnfamily_name text,"
                                                                     + "type text,"
+                                                                    + "is_dense boolean,"
                                                                     + "comparator text,"
                                                                     + "subcomparator text,"
                                                                     + "comment text,"
@@ -406,6 +407,11 @@ public final class CFMetaData
     public static final String DEFAULT_COLUMN_ALIAS = "column";
     public static final String DEFAULT_VALUE_ALIAS = "value";
 
+    // We call dense a CF for which each component of the comparator is a clustering column, i.e. no
+    // component is used to store a regular column names. In other words, non-composite static "thrift"
+    // and CQL3 CF are *not* dense.
+    private volatile Boolean isDense; // null means "we don't know and need to infer from other data"
+
     private volatile Map<ByteBuffer, ColumnDefinition> column_metadata = new HashMap<>();
     private volatile List<ColumnDefinition> partitionKeyColumns;  // Always of size keyValidator.componentsCount, null padded if necessary
     private volatile List<ColumnDefinition> clusteringKeyColumns; // Of size comparator.componentsCount or comparator.componentsCount -1, null padded if necessary
@@ -445,6 +451,7 @@ public final class CFMetaData
     public CFMetaData populateIoCacheOnFlush(boolean prop) {populateIoCacheOnFlush = prop; return this;}
     public CFMetaData droppedColumns(Map<ByteBuffer, Long> cols) {droppedColumns = cols; return this;}
     public CFMetaData triggers(Map<String, TriggerDefinition> prop) {triggers = prop; return this;}
+    public CFMetaData setDense(Boolean prop) {isDense = prop; return this;}
 
     public CFMetaData(String keyspace, String name, ColumnFamilyType type, AbstractType<?> comp, AbstractType<?> subcc)
     {
@@ -597,6 +604,7 @@ public final class CFMetaData
                       .populateIoCacheOnFlush(oldCFMD.populateIoCacheOnFlush)
                       .droppedColumns(new HashMap<>(oldCFMD.droppedColumns))
                       .triggers(new HashMap<>(oldCFMD.triggers))
+                      .setDense(oldCFMD.isDense)
                       .rebuild();
     }
 
@@ -817,6 +825,7 @@ public final class CFMetaData
             .append(populateIoCacheOnFlush, rhs.populateIoCacheOnFlush)
             .append(droppedColumns, rhs.droppedColumns)
             .append(triggers, rhs.triggers)
+            .append(isDense, rhs.isDense)
             .isEquals();
     }
 
@@ -1106,6 +1115,8 @@ public final class CFMetaData
 
         triggers = cfm.triggers;
 
+        setDense(cfm.isDense);
+
         rebuild();
         logger.debug("application result is {}", this);
     }
@@ -1640,6 +1651,9 @@ public final class CFMetaData
         for (Map.Entry<ByteBuffer, Long> entry : droppedColumns.entrySet())
             cf.addColumn(new Column(makeDroppedColumnName(entry.getKey()), LongType.instance.decompose(entry.getValue()), timestamp));
 
+        cf.addColumn(isDense == null ? DeletedColumn.create(ldt, timestamp, cfName, "is_dense")
+                                     : Column.create(isDense, timestamp, cfName, "is_dense"));
+
         // Save the CQL3 metadata "the old way" for compatibility sake
         cf.addColumn(Column.create(aliasesToJson(partitionKeyColumns), timestamp, cfName, "key_aliases"));
         cf.addColumn(Column.create(aliasesToJson(clusteringKeyColumns), timestamp, cfName, "column_aliases"));
@@ -1695,6 +1709,9 @@ public final class CFMetaData
             if (result.has("populate_io_cache_on_flush"))
                 cfm.populateIoCacheOnFlush(result.getBoolean("populate_io_cache_on_flush"));
 
+            if (result.has("is_dense"))
+                cfm.setDense(result.getBoolean("is_dense"));
+
             /*
              * The info previously hold by key_aliases, column_aliases and value_alias is now stored in column_metadata (because 1) this
              * make more sense and 2) this allow to store indexing information).
@@ -1944,7 +1961,8 @@ public final class CFMetaData
     private void rebuildCQL3Metadata()
     {
         List<ColumnDefinition> pkCols = nullInitializedList(keyValidator.componentsCount());
-        boolean isDense = isDense(comparator, column_metadata.values());
+        if (isDense == null)
+            setDense(isDense(comparator, column_metadata.values()));
         int nbCkCols = isDense
                      ? comparator.componentsCount()
                      : comparator.componentsCount() - (hasCollection() ? 2 : 1);
@@ -1983,7 +2001,7 @@ public final class CFMetaData
         clusteringKeyColumns = addDefaultColumnAliases(ckCols);
         regularColumns = regCols;
         staticColumns = statCols;
-        compactValueColumn = addDefaultValueAlias(compactCol, isDense);
+        compactValueColumn = addDefaultValueAlias(compactCol);
     }
 
     private List<ColumnDefinition> addDefaultKeyAliases(List<ColumnDefinition> pkCols)
@@ -2032,7 +2050,7 @@ public final class CFMetaData
         return ckCols;
     }
 
-    private ColumnDefinition addDefaultValueAlias(ColumnDefinition compactValueDef, boolean isDense)
+    private ColumnDefinition addDefaultValueAlias(ColumnDefinition compactValueDef)
     {
         if (isDense)
         {
@@ -2063,19 +2081,19 @@ public final class CFMetaData
      * We call dense a CF for which each component of the comparator is a clustering column, i.e. no
      * component is used to store a regular column names. In other words, non-composite static "thrift"
      * and CQL3 CF are *not* dense.
-     * Note that his method is only used by rebuildCQL3Metadata. Once said metadata are built, finding
-     * if a CF is dense amounts more simply to check if clusteringKeyColumns.size() == comparator.componentsCount().
+     * We save whether the table is dense or not during table creation through CQL, but we don't have this
+     * information for table just created through thrift, nor for table prior to CASSANDRA-7744, so this
+     * method does its best to infer whether the table is dense or not based on other elements.
      */
     private static boolean isDense(AbstractType<?> comparator, Collection<ColumnDefinition> defs)
     {
         /*
          * As said above, this method is only here because we need to deal with thrift upgrades.
          * Once a CF has been "upgraded", i.e. we've rebuilt and save its CQL3 metadata at least once,
-         * then checking for isDense amounts to looking whether the maximum componentIndex for the
-         * CLUSTERING_KEY ColumnDefinitions is equal to comparator.componentsCount() - 1 or not.
+         * then we'll have saved the "is_dense" value and will be good to go.
          *
-         * But non-upgraded thrift CF will have no such CLUSTERING_KEY column definitions, so we need
-         * to infer that information without relying on them in that case. And for the most part this is
+         * But non-upgraded thrift CF (and pre-7744 CF) will have no value for "is_dense", so we need
+         * to infer that information without relying on it in that case. And for the most part this is
          * easy, a CF that has at least one REGULAR definition is not dense. But the subtlety is that not
          * having a REGULAR definition may not mean dense because of CQL3 definitions that have only the
          * PRIMARY KEY defined.
@@ -2106,7 +2124,6 @@ public final class CFMetaData
         return maxClusteringIdx >= 0
              ? maxClusteringIdx == comparator.componentsCount() - 1
              : !hasRegular && !isCQL3OnlyPKComparator(comparator);
-
     }
 
     private static boolean isCQL3OnlyPKComparator(AbstractType<?> comparator)
@@ -2197,6 +2214,7 @@ public final class CFMetaData
             .append("populateIoCacheOnFlush", populateIoCacheOnFlush)
             .append("droppedColumns", droppedColumns)
             .append("triggers", triggers)
+            .append("isDense", isDense)
             .toString();
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c44526b2/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
index 28c7fb2..08ad069 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
@@ -51,6 +51,8 @@ public class CreateTableStatement extends SchemaAlteringStatement
     private final List<ByteBuffer> columnAliases = new ArrayList<ByteBuffer>();
     private ByteBuffer valueAlias;
 
+    private boolean isDense;
+
     private final Map<ColumnIdentifier, AbstractType> columns = new HashMap<ColumnIdentifier, AbstractType>();
     private final Set<ColumnIdentifier> staticColumns;
     private final CFPropDefs properties;
@@ -153,7 +155,8 @@ public class CreateTableStatement extends SchemaAlteringStatement
     {
         cfmd.defaultValidator(defaultValidator)
             .keyValidator(keyValidator)
-            .columnMetadata(getColumns());
+            .columnMetadata(getColumns())
+            .setDense(isDense);
 
         cfmd.addColumnMetadataFromAliases(keyAliases, keyValidator, ColumnDefinition.Type.PARTITION_KEY);
         cfmd.addColumnMetadataFromAliases(columnAliases, comparator, ColumnDefinition.Type.CLUSTERING_KEY);
@@ -237,6 +240,10 @@ public class CreateTableStatement extends SchemaAlteringStatement
             }
             stmt.keyValidator = keyTypes.size() == 1 ? keyTypes.get(0) : CompositeType.getInstance(keyTypes);
 
+            // Dense means that no part of the comparator stores a CQL column name. This means
+            // COMPACT STORAGE with at least one columnAliases (otherwise it's a thrift "static" CF).
+            stmt.isDense = useCompactStorage && !columnAliases.isEmpty();
+
             // Handle column aliases
             if (columnAliases.isEmpty())
             {