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/02/20 17:51:56 UTC

[1/7] Add static columns in CQL3

Repository: cassandra
Updated Branches:
  refs/heads/trunk 1debf25fa -> 4d75ed206


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java b/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
index 4852cf7..cd5f2a2 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
@@ -31,9 +31,9 @@ import org.apache.cassandra.utils.Pair;
  */
 public class DeleteStatement extends ModificationStatement
 {
-    private DeleteStatement(CFMetaData cfm, Attributes attrs)
+    private DeleteStatement(StatementType type, CFMetaData cfm, Attributes attrs)
     {
-        super(cfm, attrs);
+        super(type, cfm, attrs);
     }
 
     public boolean requireFullClusteringKey()
@@ -44,15 +44,30 @@ public class DeleteStatement extends ModificationStatement
     public ColumnFamily updateForKey(ByteBuffer key, ColumnNameBuilder builder, UpdateParameters params)
     throws InvalidRequestException
     {
-        CFDefinition cfDef = cfm.getCfDef();
         ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(cfm);
+        addUpdateForKey(cf, key, builder, params);
+        return cf;
+    }
+
+    public void addUpdateForKey(ColumnFamily cf, ByteBuffer key, ColumnNameBuilder builder, UpdateParameters params)
+    throws InvalidRequestException
+    {
+        CFDefinition cfDef = cfm.getCfDef();
         List<Operation> deletions = getOperations();
 
-        boolean fullKey = builder.componentCount() == cfDef.columns.size();
+        boolean fullKey = builder.componentCount() == cfDef.clusteringColumnsCount();
         boolean isRange = cfDef.isCompact ? !fullKey : (!fullKey || deletions.isEmpty());
 
         if (!deletions.isEmpty() && isRange)
-            throw new InvalidRequestException(String.format("Missing mandatory PRIMARY KEY part %s since %s specified", getFirstEmptyKey(), deletions.get(0).columnName));
+        {
+            // We only get there if we have at least one non-static columns selected, as otherwise the builder will be
+            // the "static" builder and isRange will be false. But we may still have static columns, so pick the first
+            // non static one for the error message so it's not confusing
+            for (Operation deletion : deletions)
+                if (cfm.getCfDef().get(deletion.columnName).kind != CFDefinition.Name.Kind.STATIC)
+                    throw new InvalidRequestException(String.format("Missing mandatory PRIMARY KEY part %s since %s specified", getFirstEmptyKey(), deletion.columnName));
+            throw new AssertionError();
+        }
 
         if (deletions.isEmpty() && builder.componentCount() == 0)
         {
@@ -83,8 +98,6 @@ public class DeleteStatement extends ModificationStatement
                 }
             }
         }
-
-        return cf;
     }
 
     public static class Parsed extends ModificationStatement.Parsed
@@ -96,7 +109,7 @@ public class DeleteStatement extends ModificationStatement
                       Attributes.Raw attrs,
                       List<Operation.RawDeletion> deletions,
                       List<Relation> whereClause,
-                      List<Pair<ColumnIdentifier, Operation.RawUpdate>> conditions)
+                      List<Pair<ColumnIdentifier, ColumnCondition.Raw>> conditions)
         {
             super(name, attrs, conditions, false);
             this.deletions = deletions;
@@ -105,7 +118,7 @@ public class DeleteStatement extends ModificationStatement
 
         protected ModificationStatement prepareInternal(CFDefinition cfDef, VariableSpecifications boundNames, Attributes attrs) throws InvalidRequestException
         {
-            DeleteStatement stmt = new DeleteStatement(cfDef.cfm, attrs);
+            DeleteStatement stmt = new DeleteStatement(ModificationStatement.StatementType.DELETE, cfDef.cfm, attrs);
 
             for (Operation.RawDeletion deletion : deletions)
             {
@@ -115,7 +128,7 @@ public class DeleteStatement extends ModificationStatement
 
                 // For compact, we only have one value except the key, so the only form of DELETE that make sense is without a column
                 // list. However, we support having the value name for coherence with the static/sparse case
-                if (name.kind != CFDefinition.Name.Kind.COLUMN_METADATA && name.kind != CFDefinition.Name.Kind.VALUE_ALIAS)
+                if (name.isPrimaryKeyColumn())
                     throw new InvalidRequestException(String.format("Invalid identifier %s for deletion (should not be a PRIMARY KEY part)", name));
 
                 Operation op = deletion.prepare(name);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index 676286c..ac8d2e1 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -20,12 +20,15 @@ package org.apache.cassandra.cql3.statements;
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import com.google.common.base.Function;
+import com.google.common.collect.Iterables;
 import org.github.jamm.MemoryMeter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnSlice;
@@ -56,6 +59,9 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
     private static boolean loggedCounterTTL = false;
     private static boolean loggedCounterTimestamp = false;
 
+    public static enum StatementType { INSERT, UPDATE, DELETE }
+    public final StatementType type;
+
     public final CFMetaData cfm;
     public final Attributes attrs;
 
@@ -63,11 +69,25 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
     private final List<Operation> columnOperations = new ArrayList<Operation>();
 
     private int boundTerms;
-    private List<Operation> columnConditions;
+    // Separating normal and static conditions makes things somewhat easier
+    private List<ColumnCondition> columnConditions;
+    private List<ColumnCondition> staticConditions;
     private boolean ifNotExists;
 
-    public ModificationStatement(CFMetaData cfm, Attributes attrs)
+    private boolean hasNoClusteringColumns = true;
+    private boolean setsOnlyStaticColumns;
+
+    private final Function<ColumnCondition, ColumnIdentifier> getColumnForCondition = new Function<ColumnCondition, ColumnIdentifier>()
     {
+        public ColumnIdentifier apply(ColumnCondition cond)
+        {
+            return cond.column.name;
+        }
+    };
+
+    public ModificationStatement(StatementType type, CFMetaData cfm, Attributes attrs)
+    {
+        this.type = type;
         this.cfm = cfm;
         this.attrs = attrs;
     }
@@ -78,11 +98,12 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
              + meter.measureDeep(attrs)
              + meter.measureDeep(processedKeys)
              + meter.measureDeep(columnOperations)
-             + (columnConditions == null ? 0 : meter.measureDeep(columnConditions));
+             + (columnConditions == null ? 0 : meter.measureDeep(columnConditions))
+             + (staticConditions == null ? 0 : meter.measureDeep(staticConditions));
     }
 
     public abstract boolean requireFullClusteringKey();
-    public abstract ColumnFamily updateForKey(ByteBuffer key, ColumnNameBuilder builder, UpdateParameters params) throws InvalidRequestException;
+    public abstract void addUpdateForKey(ColumnFamily updates, ByteBuffer key, ColumnNameBuilder builder, UpdateParameters params) throws InvalidRequestException;
 
     public int getBoundTerms()
     {
@@ -155,6 +176,15 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
 
     public void addOperation(Operation op)
     {
+        if (op.isStatic(cfm))
+        {
+            if (columnOperations.isEmpty())
+                setsOnlyStaticColumns = true;
+        }
+        else
+        {
+            setsOnlyStaticColumns = false;
+        }
         columnOperations.add(op);
     }
 
@@ -163,12 +193,31 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
         return columnOperations;
     }
 
-    public void addCondition(Operation op)
+    public Iterable<ColumnIdentifier> getColumnsWithConditions()
     {
-        if (columnConditions == null)
-            columnConditions = new ArrayList<Operation>();
+        if (ifNotExists)
+            return null;
+
+        return Iterables.concat(columnConditions == null ? Collections.<ColumnIdentifier>emptyList() : Iterables.transform(columnConditions, getColumnForCondition),
+                                staticConditions == null ? Collections.<ColumnIdentifier>emptyList() : Iterables.transform(staticConditions, getColumnForCondition));
+    }
 
-        columnConditions.add(op);
+    public void addCondition(ColumnCondition cond) throws InvalidRequestException
+    {
+        List<ColumnCondition> conds = null;
+        if (cond.column.kind == CFDefinition.Name.Kind.STATIC)
+        {
+            if (staticConditions == null)
+                staticConditions = new ArrayList<ColumnCondition>();
+            conds = staticConditions;
+        }
+        else
+        {
+            if (columnConditions == null)
+                columnConditions = new ArrayList<ColumnCondition>();
+            conds = columnConditions;
+        }
+        conds.add(cond);
     }
 
     public void setIfNotExistCondition()
@@ -176,13 +225,20 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
         ifNotExists = true;
     }
 
-    private void addKeyValues(ColumnIdentifier name, Restriction values) throws InvalidRequestException
+    public boolean hasIfNotExistCondition()
     {
-        if (processedKeys.put(name, values) != null)
-            throw new InvalidRequestException(String.format("Multiple definitions found for PRIMARY KEY part %s", name));
+        return ifNotExists;
     }
 
-    public void addKeyValue(ColumnIdentifier name, Term value) throws InvalidRequestException
+    private void addKeyValues(CFDefinition.Name name, Restriction values) throws InvalidRequestException
+    {
+        if (name.kind == CFDefinition.Name.Kind.COLUMN_ALIAS)
+            hasNoClusteringColumns = false;
+        if (processedKeys.put(name.name, values) != null)
+            throw new InvalidRequestException(String.format("Multiple definitions found for PRIMARY KEY part %s", name.name));
+    }
+
+    public void addKeyValue(CFDefinition.Name name, Term value) throws InvalidRequestException
     {
         addKeyValues(name, new Restriction.EQ(value, false));
     }
@@ -233,10 +289,11 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
                         throw new InvalidRequestException(String.format("Invalid operator %s for PRIMARY KEY part %s", rel.operator(), name));
                     }
 
-                    addKeyValues(name.name, restriction);
+                    addKeyValues(name, restriction);
                     break;
                 case VALUE_ALIAS:
                 case COLUMN_METADATA:
+                case STATIC:
                     throw new InvalidRequestException(String.format("Non PRIMARY KEY %s found in where clause", name));
             }
         }
@@ -248,7 +305,7 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
         CFDefinition cfDef = cfm.getCfDef();
         ColumnNameBuilder keyBuilder = cfDef.getKeyNameBuilder();
         List<ByteBuffer> keys = new ArrayList<ByteBuffer>();
-        for (CFDefinition.Name name : cfDef.keys.values())
+        for (CFDefinition.Name name : cfDef.partitionKeys())
         {
             Restriction r = processedKeys.get(name.name);
             if (r == null)
@@ -262,7 +319,9 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
                 {
                     if (val == null)
                         throw new InvalidRequestException(String.format("Invalid null value for partition key part %s", name));
-                    keys.add(keyBuilder.copy().add(val).build());
+                    ByteBuffer key = keyBuilder.copy().add(val).build();
+                    ThriftValidation.validateKey(cfm, key);
+                    keys.add(key);
                 }
             }
             else
@@ -281,10 +340,46 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
     public ColumnNameBuilder createClusteringPrefixBuilder(List<ByteBuffer> variables)
     throws InvalidRequestException
     {
+        // If the only updated/deleted columns are static, then we don't need clustering columns.
+        // And in fact, unless it is an INSERT, we reject if clustering colums are provided as that
+        // suggest something unintended. For instance, given:
+        //   CREATE TABLE t (k int, v int, s int static, PRIMARY KEY (k, v))
+        // it can make sense to do:
+        //   INSERT INTO t(k, v, s) VALUES (0, 1, 2)
+        // but both
+        //   UPDATE t SET s = 3 WHERE k = 0 AND v = 1
+        //   DELETE v FROM t WHERE k = 0 AND v = 1
+        // sounds like you don't really understand what your are doing.
+        if (setsOnlyStaticColumns && columnConditions == null && (type != StatementType.INSERT || hasNoClusteringColumns))
+        {
+            // Reject if any clustering columns is set
+            for (CFDefinition.Name name : cfm.getCfDef().clusteringColumns())
+                if (processedKeys.get(name.name) != null)
+                    throw new InvalidRequestException(String.format("Invalid restriction on clustering column %s since the %s statement modifies only static columns", name.name, type));
+            return cfm.getStaticColumnNameBuilder();
+        }
+
+        return createClusteringPrefixBuilderInternal(variables);
+    }
+
+    private ColumnNameBuilder updatePrefixFor(ByteBuffer name, ColumnNameBuilder prefix)
+    {
+        return isStatic(name) ? cfm.getStaticColumnNameBuilder() : prefix;
+    }
+
+    public boolean isStatic(ByteBuffer name)
+    {
+        ColumnDefinition def = cfm.getColumnDefinition(name);
+        return def != null && def.type == ColumnDefinition.Type.STATIC;
+    }
+
+    private ColumnNameBuilder createClusteringPrefixBuilderInternal(List<ByteBuffer> variables)
+    throws InvalidRequestException
+    {
         CFDefinition cfDef = cfm.getCfDef();
         ColumnNameBuilder builder = cfDef.getColumnNameBuilder();
         CFDefinition.Name firstEmptyKey = null;
-        for (CFDefinition.Name name : cfDef.columns.values())
+        for (CFDefinition.Name name : cfDef.clusteringColumns())
         {
             Restriction r = processedKeys.get(name.name);
             if (r == null)
@@ -312,7 +407,7 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
 
     protected CFDefinition.Name getFirstEmptyKey()
     {
-        for (CFDefinition.Name name : cfm.getCfDef().columns.values())
+        for (CFDefinition.Name name : cfm.getCfDef().clusteringColumns())
         {
             if (processedKeys.get(name.name) == null)
                 return name;
@@ -354,8 +449,9 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
         int i = 0;
         for (ByteBuffer name : toRead)
         {
-            ByteBuffer start = clusteringPrefix.copy().add(name).build();
-            ByteBuffer finish = clusteringPrefix.copy().add(name).buildAsEndOfRange();
+            ColumnNameBuilder prefix = updatePrefixFor(name, clusteringPrefix);
+            ByteBuffer start = prefix.copy().add(name).build();
+            ByteBuffer finish = prefix.copy().add(name).buildAsEndOfRange();
             slices[i++] = new ColumnSlice(start, finish);
         }
 
@@ -392,7 +488,9 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
 
     public boolean hasConditions()
     {
-        return ifNotExists || (columnConditions != null && !columnConditions.isEmpty());
+        return ifNotExists
+            || (columnConditions != null && !columnConditions.isEmpty())
+            || (staticConditions != null && !staticConditions.isEmpty());
     }
 
     public ResultMessage execute(QueryState queryState, QueryOptions options)
@@ -434,20 +532,14 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
         if (keys.size() > 1)
             throw new InvalidRequestException("IN on the partition key is not supported with conditional updates");
 
-        ColumnNameBuilder clusteringPrefix = createClusteringPrefixBuilder(variables);
-
         ByteBuffer key = keys.get(0);
-        ThriftValidation.validateKey(cfm, key);
-
-        UpdateParameters updParams = new UpdateParameters(cfm, variables, queryState.getTimestamp(), getTimeToLive(variables), null);
-        ColumnFamily updates = updateForKey(key, clusteringPrefix, updParams);
 
         // It's cleaner to use the query timestamp below, but it's in seconds while the conditions expects microseconds, so just
         // put it back in millis (we don't really lose precision because the ultimate consumer, Column.isLive, re-divide it).
-        long now = queryState.getTimestamp() * 1000;
-        CASConditions conditions = ifNotExists
-                                 ? new NotExistCondition(clusteringPrefix, now)
-                                 : new ColumnsConditions(clusteringPrefix, cfm, key, columnConditions, variables, now);
+        CQL3CasConditions conditions = new CQL3CasConditions(cfm, queryState.getTimestamp() * 1000);
+        ColumnNameBuilder prefix = createClusteringPrefixBuilder(variables);
+        ColumnFamily updates = UnsortedColumns.factory.create(cfm);
+        addUpdatesAndConditions(key, prefix, updates, conditions, variables, getTimestamp(queryState.getTimestamp(), variables));
 
         ColumnFamily result = StorageProxy.cas(keyspace(),
                                                columnFamily(),
@@ -459,16 +551,44 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
         return new ResultMessage.Rows(buildCasResultSet(key, result));
     }
 
+    public void addUpdatesAndConditions(ByteBuffer key, ColumnNameBuilder clusteringPrefix, ColumnFamily updates, CQL3CasConditions conditions, List<ByteBuffer> variables, long now)
+    throws InvalidRequestException
+    {
+        UpdateParameters updParams = new UpdateParameters(cfm, variables, now, getTimeToLive(variables), null);
+        addUpdateForKey(updates, key, clusteringPrefix, updParams);
+
+        if (ifNotExists)
+        {
+            // If we use ifNotExists, if the statement applies to any non static columns, then the condition is on the row of the non-static
+            // columns and the prefix should be the rowPrefix. But if only static columns are set, then the ifNotExists apply to the existence
+            // of any static columns and we should use the prefix for the "static part" of the partition.
+            conditions.addNotExist(setsOnlyStaticColumns ? cfm.getStaticColumnNameBuilder() : clusteringPrefix);
+        }
+        else
+        {
+            if (columnConditions != null)
+                conditions.addConditions(clusteringPrefix, columnConditions, variables);
+            if (staticConditions != null)
+                conditions.addConditions(cfm.getStaticColumnNameBuilder(), staticConditions, variables);
+        }
+    }
+
     private ResultSet buildCasResultSet(ByteBuffer key, ColumnFamily cf) throws InvalidRequestException
     {
+        return buildCasResultSet(keyspace(), key, columnFamily(), cf, getColumnsWithConditions(), false);
+    }
+
+    public static ResultSet buildCasResultSet(String ksName, ByteBuffer key, String cfName, ColumnFamily cf, Iterable<ColumnIdentifier> columnsWithConditions, boolean isBatch)
+    throws InvalidRequestException
+    {
         boolean success = cf == null;
 
-        ColumnSpecification spec = new ColumnSpecification(keyspace(), columnFamily(), CAS_RESULT_COLUMN, BooleanType.instance);
+        ColumnSpecification spec = new ColumnSpecification(ksName, cfName, CAS_RESULT_COLUMN, BooleanType.instance);
         ResultSet.Metadata metadata = new ResultSet.Metadata(Collections.singletonList(spec));
         List<List<ByteBuffer>> rows = Collections.singletonList(Collections.singletonList(BooleanType.instance.decompose(success)));
 
         ResultSet rs = new ResultSet(metadata, rows);
-        return success ? rs : merge(rs, buildCasFailureResultSet(key, cf));
+        return success ? rs : merge(rs, buildCasFailureResultSet(key, cf, columnsWithConditions, isBatch));
     }
 
     private static ResultSet merge(ResultSet left, ResultSet right)
@@ -478,31 +598,44 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
         else if (right.size() == 0)
             return left;
 
-        assert left.size() == 1 && right.size() == 1;
+        assert left.size() == 1;
         int size = left.metadata.names.size() + right.metadata.names.size();
         List<ColumnSpecification> specs = new ArrayList<ColumnSpecification>(size);
         specs.addAll(left.metadata.names);
         specs.addAll(right.metadata.names);
-        List<ByteBuffer> row = new ArrayList<ByteBuffer>(size);
-        row.addAll(left.rows.get(0));
-        row.addAll(right.rows.get(0));
-        return new ResultSet(new ResultSet.Metadata(specs), Collections.singletonList(row));
+        List<List<ByteBuffer>> rows = new ArrayList<>(right.size());
+        for (int i = 0; i < right.size(); i++)
+        {
+            List<ByteBuffer> row = new ArrayList<ByteBuffer>(size);
+            row.addAll(left.rows.get(0));
+            row.addAll(right.rows.get(i));
+            rows.add(row);
+        }
+        return new ResultSet(new ResultSet.Metadata(specs), rows);
     }
 
-    private ResultSet buildCasFailureResultSet(ByteBuffer key, ColumnFamily cf) throws InvalidRequestException
+    private static ResultSet buildCasFailureResultSet(ByteBuffer key, ColumnFamily cf, Iterable<ColumnIdentifier> columnsWithConditions, boolean isBatch)
+    throws InvalidRequestException
     {
-        CFDefinition cfDef = cfm.getCfDef();
+        CFDefinition cfDef = cf.metadata().getCfDef();
 
         Selection selection;
-        if (ifNotExists)
+        if (columnsWithConditions == null)
         {
             selection = Selection.wildcard(cfDef);
         }
         else
         {
-            List<CFDefinition.Name> names = new ArrayList<CFDefinition.Name>(columnConditions.size());
-            for (Operation condition : columnConditions)
-                names.add(cfDef.get(condition.columnName));
+            List<CFDefinition.Name> names = new ArrayList<CFDefinition.Name>();
+            // Adding the partition key for batches to disambiguate if the conditions span multipe rows (we don't add them outside
+            // of batches for compatibility sakes).
+            if (isBatch)
+            {
+                names.addAll(cfDef.partitionKeys());
+                names.addAll(cfDef.clusteringColumns());
+            }
+            for (ColumnIdentifier id : columnsWithConditions)
+                names.add(cfDef.get(id));
             selection = Selection.forColumns(names);
         }
 
@@ -548,7 +681,8 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
         for (ByteBuffer key: keys)
         {
             ThriftValidation.validateKey(cfm, key);
-            ColumnFamily cf = updateForKey(key, clusteringPrefix, params);
+            ColumnFamily cf = UnsortedColumns.factory.create(cfm);
+            addUpdateForKey(cf, key, clusteringPrefix, params);
             mutations.add(makeMutation(key, cf, cl, isBatch));
         }
         return mutations;
@@ -570,104 +704,17 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
         return isCounter() ? new CounterMutation(rm, cl) : rm;
     }
 
-    private static abstract class CQL3CasConditions implements CASConditions
-    {
-        protected final ColumnNameBuilder rowPrefix;
-        protected final long now;
-
-        protected CQL3CasConditions(ColumnNameBuilder rowPrefix, long now)
-        {
-            this.rowPrefix = rowPrefix;
-            this.now = now;
-        }
-
-        public IDiskAtomFilter readFilter()
-        {
-            // We always read the row entirely as on CAS failure we want to be able to distinguish between "row exists
-            // but all values on why there were conditions are null" and "row doesn't exists", and we can't rely on the
-            // row marker for that (see #6623)
-            return new SliceQueryFilter(rowPrefix.build(), rowPrefix.buildAsEndOfRange(), false, 1, rowPrefix.componentCount());
-        }
-    }
-
-    private static class NotExistCondition extends CQL3CasConditions
-    {
-        private NotExistCondition(ColumnNameBuilder rowPrefix, long now)
-        {
-            super(rowPrefix, now);
-        }
-
-        public boolean appliesTo(ColumnFamily current)
-        {
-            return current == null || current.hasOnlyTombstones(now);
-        }
-    }
-
-    private static class ColumnsConditions extends CQL3CasConditions
-    {
-        private final ColumnFamily expected;
-
-        private ColumnsConditions(ColumnNameBuilder rowPrefix,
-                                  CFMetaData cfm,
-                                  ByteBuffer key,
-                                  Collection<Operation> conditions,
-                                  List<ByteBuffer> variables,
-                                  long now) throws InvalidRequestException
-        {
-            super(rowPrefix, now);
-            this.expected = TreeMapBackedSortedColumns.factory.create(cfm);
-
-            // When building the conditions, we should not use a TTL. It's not useful, and if a very low ttl (1 seconds) is used, it's possible
-            // for it to expire before the actual build of the conditions which would break since we would then testing for the presence of tombstones.
-            UpdateParameters params = new UpdateParameters(cfm, variables, now, 0, null);
-
-            // Conditions
-            for (Operation condition : conditions)
-                condition.execute(key, expected, rowPrefix.copy(), params);
-        }
-
-        public boolean appliesTo(ColumnFamily current)
-        {
-            if (current == null)
-                return false;
-
-            for (Column e : expected)
-            {
-                Column c = current.getColumn(e.name());
-                if (e.isLive(now))
-                {
-                    if (c == null || !c.isLive(now) || !c.value().equals(e.value()))
-                        return false;
-                }
-                else
-                {
-                    // If we have a tombstone in expected, it means the condition tests that the column is
-                    // null, so check that we have no value
-                    if (c != null && c.isLive(now))
-                        return false;
-                }
-            }
-            return true;
-        }
-
-        @Override
-        public String toString()
-        {
-            return expected.toString();
-        }
-    }
-
     public static abstract class Parsed extends CFStatement
     {
         protected final Attributes.Raw attrs;
-        private final List<Pair<ColumnIdentifier, Operation.RawUpdate>> conditions;
+        private final List<Pair<ColumnIdentifier, ColumnCondition.Raw>> conditions;
         private final boolean ifNotExists;
 
-        protected Parsed(CFName name, Attributes.Raw attrs, List<Pair<ColumnIdentifier, Operation.RawUpdate>> conditions, boolean ifNotExists)
+        protected Parsed(CFName name, Attributes.Raw attrs, List<Pair<ColumnIdentifier, ColumnCondition.Raw>> conditions, boolean ifNotExists)
         {
             super(name);
             this.attrs = attrs;
-            this.conditions = conditions == null ? Collections.<Pair<ColumnIdentifier, Operation.RawUpdate>>emptyList() : conditions;
+            this.conditions = conditions == null ? Collections.<Pair<ColumnIdentifier, ColumnCondition.Raw>>emptyList() : conditions;
             this.ifNotExists = ifNotExists;
         }
 
@@ -709,24 +756,13 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
                 }
                 else
                 {
-                    for (Pair<ColumnIdentifier, Operation.RawUpdate> entry : conditions)
+                    for (Pair<ColumnIdentifier, ColumnCondition.Raw> entry : conditions)
                     {
                         CFDefinition.Name name = cfDef.get(entry.left);
                         if (name == null)
                             throw new InvalidRequestException(String.format("Unknown identifier %s", entry.left));
 
-                        /*
-                         * Lists column names are based on a server-side generated timeuuid. So we can't allow lists
-                         * operation or that would yield unexpected results (update that should apply wouldn't). So for
-                         * now, we just refuse lists, which also save use from having to bother about the read that some
-                         * list operation involve.
-                         */
-                        if (name.type instanceof ListType)
-                            throw new InvalidRequestException(String.format("List operation (%s) are not allowed in conditional updates", name));
-
-                        Operation condition = entry.right.prepare(name);
-                        assert !condition.requiresRead();
-
+                        ColumnCondition condition = entry.right.prepare(name);
                         condition.collectMarkerSpecification(boundNames);
 
                         switch (name.kind)
@@ -736,6 +772,7 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
                                 throw new InvalidRequestException(String.format("PRIMARY KEY part %s found in SET part", entry.left));
                             case VALUE_ALIAS:
                             case COLUMN_METADATA:
+                            case STATIC:
                                 stmt.addCondition(condition);
                                 break;
                         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 52a7c70..2636c83 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -83,18 +83,57 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
 
     private Map<CFDefinition.Name, Integer> orderingIndexes;
 
+    private boolean selectsStaticColumns;
+    private boolean selectsOnlyStaticColumns;
+
     // Used by forSelection below
     private static final Parameters defaultParameters = new Parameters(Collections.<ColumnIdentifier, Boolean>emptyMap(), false, false, null, false);
 
+    private static final Predicate<CFDefinition.Name> isStaticFilter = new Predicate<CFDefinition.Name>()
+    {
+        public boolean apply(CFDefinition.Name name)
+        {
+            return name.kind == CFDefinition.Name.Kind.STATIC;
+        }
+    };
+
     public SelectStatement(CFDefinition cfDef, int boundTerms, Parameters parameters, Selection selection, Term limit)
     {
         this.cfDef = cfDef;
         this.boundTerms = boundTerms;
         this.selection = selection;
-        this.keyRestrictions = new Restriction[cfDef.keys.size()];
-        this.columnRestrictions = new Restriction[cfDef.columns.size()];
+        this.keyRestrictions = new Restriction[cfDef.partitionKeyCount()];
+        this.columnRestrictions = new Restriction[cfDef.clusteringColumnsCount()];
         this.parameters = parameters;
         this.limit = limit;
+
+        // Now gather a few info on whether we should bother with static columns or not for this statement
+        initStaticColumnsInfo();
+    }
+
+    private void initStaticColumnsInfo()
+    {
+        if (!cfDef.cfm.hasStaticColumns())
+            return;
+
+        // If it's a wildcard, we do select static but not only them
+        if (selection.isWildcard())
+        {
+            selectsStaticColumns = true;
+            return;
+        }
+
+        // Otherwise, check the selected columns
+        selectsStaticColumns = !Iterables.isEmpty(Iterables.filter(selection.getColumnsList(), isStaticFilter));
+        selectsOnlyStaticColumns = true;
+        for (CFDefinition.Name name : selection.getColumnsList())
+        {
+            if (name.kind != CFDefinition.Name.Kind.KEY_ALIAS && name.kind != CFDefinition.Name.Kind.STATIC)
+            {
+                selectsOnlyStaticColumns = false;
+                break;
+            }
+        }
     }
 
     // Creates a simple select based on the given selection.
@@ -383,35 +422,91 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             // to account for the grouping of columns.
             // Since that doesn't work for maps/sets/lists, we now use the compositesToGroup option of SliceQueryFilter.
             // But we must preserve backward compatibility too (for mixed version cluster that is).
-            int toGroup = cfDef.isCompact ? -1 : cfDef.columns.size();
+            int toGroup = cfDef.isCompact ? -1 : cfDef.clusteringColumnsCount();
             List<ByteBuffer> startBounds = getRequestedBound(Bound.START, variables);
             List<ByteBuffer> endBounds = getRequestedBound(Bound.END, variables);
             assert startBounds.size() == endBounds.size();
 
+            // Handles fetching static columns. Note that for 2i, the filter is just used to restrict
+            // the part of the index to query so adding the static slice would be useless and confusing.
+            // For 2i, static columns are retrieve in CompositesSearcher with each index hit.
+            ColumnSlice staticSlice = null;
+            if (selectsStaticColumns && !usesSecondaryIndexing)
+            {
+                ColumnNameBuilder staticPrefix = cfDef.cfm.getStaticColumnNameBuilder();
+                // Note: we could use staticPrefix.build() for the start bound, but EMPTY_BYTE_BUFFER gives us the
+                // same effect while saving a few CPU cycles.
+                staticSlice = isReversed
+                            ? new ColumnSlice(staticPrefix.buildAsEndOfRange(), ByteBufferUtil.EMPTY_BYTE_BUFFER)
+                            : new ColumnSlice(ByteBufferUtil.EMPTY_BYTE_BUFFER, staticPrefix.buildAsEndOfRange());
+
+                // In the case where we only select static columns, we want to really only check the static columns.
+                // So we return early as the rest of that method would actually make us query everything
+                if (selectsOnlyStaticColumns)
+                    return sliceFilter(staticSlice, limit, toGroup);
+            }
+
             // The case where startBounds == 1 is common enough that it's worth optimizing
-            ColumnSlice[] slices;
             if (startBounds.size() == 1)
             {
                 ColumnSlice slice = new ColumnSlice(startBounds.get(0), endBounds.get(0));
                 if (slice.isAlwaysEmpty(cfDef.cfm.comparator, isReversed))
-                    return null;
-                slices = new ColumnSlice[]{slice};
+                    return staticSlice == null ? null : sliceFilter(staticSlice, limit, toGroup);
+
+                return staticSlice == null
+                     ? sliceFilter(slice, limit, toGroup)
+                     : (slice.includes(cfDef.cfm.comparator, staticSlice.finish) ? sliceFilter(new ColumnSlice(staticSlice.start, slice.finish), limit, toGroup)
+                                                                                 : sliceFilter(new ColumnSlice[]{ staticSlice, slice }, limit, toGroup));
+            }
+
+            List<ColumnSlice> l = new ArrayList<ColumnSlice>(startBounds.size());
+            for (int i = 0; i < startBounds.size(); i++)
+            {
+                ColumnSlice slice = new ColumnSlice(startBounds.get(i), endBounds.get(i));
+                if (!slice.isAlwaysEmpty(cfDef.cfm.comparator, isReversed))
+                    l.add(slice);
+            }
+
+            if (l.isEmpty())
+                return staticSlice == null ? null : sliceFilter(staticSlice, limit, toGroup);
+            if (staticSlice == null)
+                return sliceFilter(l.toArray(new ColumnSlice[l.size()]), limit, toGroup);
+
+            // The slices should not overlap. We know the slices built from startBounds/endBounds don't, but
+            // if there is a static slice, it could overlap with the 2nd slice. Check for it and correct if
+            // that's the case
+            ColumnSlice[] slices;
+            if (isReversed)
+            {
+                if (l.get(l.size() - 1).includes(cfDef.cfm.comparator, staticSlice.start))
+                {
+                    slices = l.toArray(new ColumnSlice[l.size()]);
+                    slices[slices.length-1] = new ColumnSlice(slices[slices.length-1].start, ByteBufferUtil.EMPTY_BYTE_BUFFER);
+                }
+                else
+                {
+                    slices = l.toArray(new ColumnSlice[l.size()+1]);
+                    slices[slices.length-1] = staticSlice;
+                }
             }
             else
             {
-                List<ColumnSlice> l = new ArrayList<ColumnSlice>(startBounds.size());
-                for (int i = 0; i < startBounds.size(); i++)
+                if (l.get(0).includes(cfDef.cfm.comparator, staticSlice.finish))
                 {
-                    ColumnSlice slice = new ColumnSlice(startBounds.get(i), endBounds.get(i));
-                    if (!slice.isAlwaysEmpty(cfDef.cfm.comparator, isReversed))
-                        l.add(slice);
+                    slices = new ColumnSlice[l.size()];
+                    slices[0] = new ColumnSlice(ByteBufferUtil.EMPTY_BYTE_BUFFER, l.get(0).finish);
+                    for (int i = 1; i < l.size(); i++)
+                        slices[i] = l.get(i);
+                }
+                else
+                {
+                    slices = new ColumnSlice[l.size()+1];
+                    slices[0] = staticSlice;
+                    for (int i = 0; i < l.size(); i++)
+                        slices[i] = l.get(i);
                 }
-                if (l.isEmpty())
-                    return null;
-                slices = l.toArray(new ColumnSlice[l.size()]);
             }
-
-            return new SliceQueryFilter(slices, isReversed, limit, toGroup);
+            return sliceFilter(slices, limit, toGroup);
         }
         else
         {
@@ -423,6 +518,16 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         }
     }
 
+    private SliceQueryFilter sliceFilter(ColumnSlice slice, int limit, int toGroup)
+    {
+        return sliceFilter(new ColumnSlice[]{ slice }, limit, toGroup);
+    }
+
+    private SliceQueryFilter sliceFilter(ColumnSlice[] slices, int limit, int toGroup)
+    {
+        return new SliceQueryFilter(slices, isReversed, limit, toGroup);
+    }
+
     private int getLimit(List<ByteBuffer> variables) throws InvalidRequestException
     {
         int l = Integer.MAX_VALUE;
@@ -458,7 +563,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
     {
         List<ByteBuffer> keys = new ArrayList<ByteBuffer>();
         ColumnNameBuilder builder = cfDef.getKeyNameBuilder();
-        for (CFDefinition.Name name : cfDef.keys.values())
+        for (CFDefinition.Name name : cfDef.partitionKeys())
         {
             Restriction r = keyRestrictions[name.position];
             assert r != null && !r.isSlice();
@@ -497,7 +602,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 return ByteBufferUtil.EMPTY_BYTE_BUFFER;
 
         // We deal with IN queries for keys in other places, so we know buildBound will return only one result
-        return buildBound(b, cfDef.keys.values(), keyRestrictions, false, cfDef.getKeyNameBuilder(), variables).get(0);
+        return buildBound(b, cfDef.partitionKeys(), keyRestrictions, false, cfDef.getKeyNameBuilder(), variables).get(0);
     }
 
     private Token getTokenBound(Bound b, List<ByteBuffer> variables, IPartitioner<?> p) throws InvalidRequestException
@@ -556,13 +661,15 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
 
     private SortedSet<ByteBuffer> getRequestedColumns(List<ByteBuffer> variables) throws InvalidRequestException
     {
+        // Note: getRequestedColumns don't handle static columns, but due to CASSANDRA-5762
+        // we always do a slice for CQL3 tables, so it's ok to ignore them here
         assert !isColumnRange();
 
         ColumnNameBuilder builder = cfDef.getColumnNameBuilder();
-        Iterator<ColumnIdentifier> idIter = cfDef.columns.keySet().iterator();
+        Iterator<CFDefinition.Name> idIter = cfDef.clusteringColumns().iterator();
         for (Restriction r : columnRestrictions)
         {
-            ColumnIdentifier id = idIter.next();
+            ColumnIdentifier id = idIter.next().name;
             assert r != null && !r.isSlice();
 
             List<ByteBuffer> values = r.values(variables);
@@ -625,15 +732,16 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 columns.add(builder.copy().add(ByteBufferUtil.EMPTY_BYTE_BUFFER).build());
 
                 // selected columns
-                for (ColumnIdentifier id : selection.regularColumnsToFetch())
+                for (ColumnIdentifier id : selection.regularAndStaticColumnsToFetch())
                     columns.add(builder.copy().add(id.key).build());
             }
             else
             {
-                Iterator<ColumnIdentifier> iter = cfDef.metadata.keySet().iterator();
+                // We now that we're not composite so we can ignore static columns
+                Iterator<CFDefinition.Name> iter = cfDef.regularColumns().iterator();
                 while (iter.hasNext())
                 {
-                    ColumnIdentifier name = iter.next();
+                    ColumnIdentifier name = iter.next().name;
                     ColumnNameBuilder b = iter.hasNext() ? builder.copy() : builder;
                     ByteBuffer cname = b.add(name.key).build();
                     columns.add(cname);
@@ -760,7 +868,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
     private List<ByteBuffer> getRequestedBound(Bound b, List<ByteBuffer> variables) throws InvalidRequestException
     {
         assert isColumnRange();
-        return buildBound(b, cfDef.columns.values(), columnRestrictions, isReversed, cfDef.getColumnNameBuilder(), variables);
+        return buildBound(b, cfDef.clusteringColumns(), columnRestrictions, isReversed, cfDef.getColumnNameBuilder(), variables);
     }
 
     public List<IndexExpression> getIndexExpressions(List<ByteBuffer> variables) throws InvalidRequestException
@@ -781,6 +889,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                     restriction = columnRestrictions[name.position];
                     break;
                 case COLUMN_METADATA:
+                case STATIC:
                     restriction = metadataRestrictions.get(name);
                     break;
                 default:
@@ -956,6 +1065,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                             result.add(c);
                             break;
                         case COLUMN_METADATA:
+                        case STATIC:
                             // This should not happen for compact CF
                             throw new AssertionError();
                         default:
@@ -979,8 +1089,35 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 builder.add(c);
             }
 
+            Map<CFDefinition.Name, ByteBuffer> staticValues = Collections.emptyMap();
+            // Gather up static values first
+            if (!builder.isEmpty() && builder.firstGroup().isStatic)
+            {
+                staticValues = new HashMap<>();
+                ColumnGroupMap group = builder.firstGroup();
+                for (CFDefinition.Name name : Iterables.filter(selection.getColumnsList(), isStaticFilter))
+                    staticValues.put(name, name.type.isCollection() ? getCollectionValue(name, group) : getSimpleValue(name, group));
+                builder.discardFirst();
+
+                // If there was static columns but there is no actual row, then provided the select was a full
+                // partition selection (i.e. not a 2ndary index search and there was no condition on clustering columns)
+                // then we want to include the static columns in the result set.
+                if (!staticValues.isEmpty() && builder.isEmpty() && !usesSecondaryIndexing && hasNoClusteringColumnsRestriction())
+                {
+                    result.newRow();
+                    for (CFDefinition.Name name : selection.getColumnsList())
+                    {
+                        if (name.kind == CFDefinition.Name.Kind.KEY_ALIAS)
+                            result.add(keyComponents[name.position]);
+                        else
+                            result.add(name.kind == CFDefinition.Name.Kind.STATIC ? staticValues.get(name) : null);
+                    }
+                    return;
+                }
+            }
+
             for (ColumnGroupMap group : builder.groups())
-                handleGroup(selection, result, keyComponents, group);
+                handleGroup(selection, result, keyComponents, group, staticValues);
         }
         else
         {
@@ -999,6 +1136,14 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         }
     }
 
+    private boolean hasNoClusteringColumnsRestriction()
+    {
+        for (int i = 0; i < columnRestrictions.length; i++)
+            if (columnRestrictions[i] != null)
+                return false;
+        return true;
+    }
+
     /**
      * Orders results when multiple keys are selected (using IN)
      */
@@ -1039,7 +1184,11 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         Collections.sort(cqlRows.rows, new CompositeComparator(types, positions));
     }
 
-    private void handleGroup(Selection selection, Selection.ResultSetBuilder result, ByteBuffer[] keyComponents, ColumnGroupMap columns) throws InvalidRequestException
+    private void handleGroup(Selection selection,
+                             Selection.ResultSetBuilder result,
+                             ByteBuffer[] keyComponents,
+                             ColumnGroupMap columns,
+                             Map<CFDefinition.Name, ByteBuffer> staticValues) throws InvalidRequestException
     {
         // Respect requested order
         result.newRow();
@@ -1059,21 +1208,32 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 case COLUMN_METADATA:
                     if (name.type.isCollection())
                     {
-                        List<Pair<ByteBuffer, Column>> collection = columns.getCollection(name.name.key);
-                        ByteBuffer value = collection == null
-                                         ? null
-                                         : ((CollectionType)name.type).serialize(collection);
-                        result.add(value);
+                        result.add(getCollectionValue(name, columns));
                     }
                     else
                     {
                         result.add(columns.getSimple(name.name.key));
                     }
                     break;
+                case STATIC:
+                    result.add(staticValues.get(name));
+                    break;
             }
         }
     }
 
+    private static ByteBuffer getCollectionValue(CFDefinition.Name name, ColumnGroupMap columns)
+    {
+        List<Pair<ByteBuffer, Column>> collection = columns.getCollection(name.name.key);
+        return collection == null ? null : ((CollectionType)name.type).serialize(collection);
+    }
+
+    private static ByteBuffer getSimpleValue(CFDefinition.Name name, ColumnGroupMap columns)
+    {
+        Column c = columns.getSimple(name.name.key);
+        return c == null ? null : c.value();
+    }
+
     private static boolean isReversedType(CFDefinition.Name name)
     {
         return name.type instanceof ReversedType;
@@ -1089,6 +1249,14 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         return true;
     }
 
+    private boolean hasClusteringColumnsRestriction()
+    {
+        for (int i = 0; i < columnRestrictions.length; i++)
+            if (columnRestrictions[i] != null)
+                return true;
+        return false;
+    }
+
     public static class RawStatement extends CFStatement
     {
         private final Parameters parameters;
@@ -1122,7 +1290,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                                 : Selection.fromSelectors(cfDef, selectClause);
 
             if (parameters.isDistinct)
-                validateDistinctSelection(selection.getColumnsList(), cfDef.keys.values());
+                validateDistinctSelection(selection.getColumnsList(), cfDef.partitionKeys());
 
             Term prepLimit = null;
             if (limit != null)
@@ -1174,6 +1342,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                     case VALUE_ALIAS:
                         throw new InvalidRequestException(String.format("Predicates on the non-primary-key column (%s) of a COMPACT table are not yet supported", name.name));
                     case COLUMN_METADATA:
+                    case STATIC:
                         // We only all IN on the row key and last clustering key so far, never on non-PK columns, and this even if there's an index
                         Restriction r = updateRestriction(cfm, name, stmt.metadataRestrictions.get(name), rel, names);
                         if (r.isIN() && !((Restriction.IN)r).canHaveOnlyOneValue())
@@ -1197,7 +1366,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             boolean canRestrictFurtherComponents = true;
             CFDefinition.Name previous = null;
             stmt.keyIsInRelation = false;
-            Iterator<CFDefinition.Name> iter = cfDef.keys.values().iterator();
+            Iterator<CFDefinition.Name> iter = cfDef.partitionKeys().iterator();
             for (int i = 0; i < stmt.keyRestrictions.length; i++)
             {
                 CFDefinition.Name cname = iter.next();
@@ -1266,7 +1435,10 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             // All (or none) of the partition key columns have been specified;
             // hence there is no need to turn these restrictions into index expressions.
             if (!stmt.usesSecondaryIndexing)
-                stmt.restrictedNames.removeAll(cfDef.keys.values());
+                stmt.restrictedNames.removeAll(cfDef.partitionKeys());
+
+            if (stmt.selectsOnlyStaticColumns && stmt.hasClusteringColumnsRestriction())
+                throw new InvalidRequestException("Cannot restrict clustering columns when selecting only static columns");
 
             // If a clustering key column is restricted by a non-EQ relation, all preceding
             // columns must have a EQ, and all following must have no restriction. Unless
@@ -1274,7 +1446,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             canRestrictFurtherComponents = true;
             previous = null;
             boolean previousIsSlice = false;
-            iter = cfDef.columns.values().iterator();
+            iter = cfDef.clusteringColumns().iterator();
             for (int i = 0; i < stmt.columnRestrictions.length; i++)
             {
                 CFDefinition.Name cname = iter.next();
@@ -1332,7 +1504,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 stmt.usesSecondaryIndexing = true;
 
             if (!stmt.usesSecondaryIndexing)
-                stmt.restrictedNames.removeAll(cfDef.columns.values());
+                stmt.restrictedNames.removeAll(cfDef.clusteringColumns());
 
             // Even if usesSecondaryIndexing is false at this point, we'll still have to use one if
             // there is restrictions not covered by the PK.
@@ -1343,8 +1515,16 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 stmt.usesSecondaryIndexing = true;
             }
 
-            if (stmt.usesSecondaryIndexing && stmt.keyIsInRelation)
-                throw new InvalidRequestException("Select on indexed columns and with IN clause for the PRIMARY KEY are not supported");
+            if (stmt.usesSecondaryIndexing)
+            {
+                if (stmt.keyIsInRelation)
+                    throw new InvalidRequestException("Select on indexed columns and with IN clause for the PRIMARY KEY are not supported");
+                // When the user only select static columns, the intent is that we don't query the whole partition but just
+                // the static parts. But 1) we don't have an easy way to do that with 2i and 2) since we don't support index on static columns
+                // so far, 2i means that you've restricted a non static column, so the query is somewhat non-sensical.
+                if (stmt.selectsOnlyStaticColumns)
+                    throw new InvalidRequestException("Queries using 2ndary indexes don't support selecting only static columns");
+            }
 
             if (!stmt.parameters.orderings.isEmpty())
             {
@@ -1401,7 +1581,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                     }
                 }
 
-                Boolean[] reversedMap = new Boolean[cfDef.columns.size()];
+                Boolean[] reversedMap = new Boolean[cfDef.clusteringColumnsCount()];
                 int i = 0;
                 for (Map.Entry<ColumnIdentifier, Boolean> entry : stmt.parameters.orderings.entrySet())
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/cql3/statements/Selection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/Selection.java b/src/java/org/apache/cassandra/cql3/statements/Selection.java
index 600ee1b..9760311 100644
--- a/src/java/org/apache/cassandra/cql3/statements/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Selection.java
@@ -49,6 +49,12 @@ public abstract class Selection
         this.collectTTLs = collectTTLs;
     }
 
+    // Overriden by SimpleSelection when appropriate.
+    public boolean isWildcard()
+    {
+        return false;
+    }
+
     public ResultSet.Metadata getResultMetadata()
     {
         return new ResultSet.Metadata(metadata);
@@ -59,12 +65,12 @@ public abstract class Selection
         List<CFDefinition.Name> all = new ArrayList<CFDefinition.Name>();
         for (CFDefinition.Name name : cfDef)
             all.add(name);
-        return new SimpleSelection(all);
+        return new SimpleSelection(all, true);
     }
 
     public static Selection forColumns(List<CFDefinition.Name> columnsList)
     {
-        return new SimpleSelection(columnsList);
+        return new SimpleSelection(columnsList, false);
     }
 
     private static boolean isUsingFunction(List<RawSelector> rawSelectors)
@@ -105,7 +111,7 @@ public abstract class Selection
             CFDefinition.Name name = cfDef.get(tot.id);
             if (name == null)
                 throw new InvalidRequestException(String.format("Undefined name %s in selection clause", tot.id));
-            if (name.kind != CFDefinition.Name.Kind.COLUMN_METADATA && name.kind != CFDefinition.Name.Kind.VALUE_ALIAS)
+            if (name.isPrimaryKeyColumn())
                 throw new InvalidRequestException(String.format("Cannot use selection function %s on PRIMARY KEY part %s", tot.isWritetime ? "writeTime" : "ttl", name));
             if (name.type.isCollection())
                 throw new InvalidRequestException(String.format("Cannot use selection function %s on collections", tot.isWritetime ? "writeTime" : "ttl"));
@@ -195,7 +201,7 @@ public abstract class Selection
                 names.add(name);
                 metadata.add(rawSelector.alias == null ? name : makeAliasSpec(cfDef, name.type, rawSelector.alias));
             }
-            return new SimpleSelection(names, metadata);
+            return new SimpleSelection(names, metadata, false);
         }
     }
 
@@ -204,12 +210,12 @@ public abstract class Selection
     /**
      * @return the list of CQL3 "regular" (the "COLUMN_METADATA" ones) column names to fetch.
      */
-    public List<ColumnIdentifier> regularColumnsToFetch()
+    public List<ColumnIdentifier> regularAndStaticColumnsToFetch()
     {
         List<ColumnIdentifier> toFetch = new ArrayList<ColumnIdentifier>();
         for (CFDefinition.Name name : columnsList)
         {
-            if (name.kind == CFDefinition.Name.Kind.COLUMN_METADATA)
+            if (name.kind == CFDefinition.Name.Kind.COLUMN_METADATA || name.kind == CFDefinition.Name.Kind.STATIC)
                 toFetch.add(name.name);
         }
         return toFetch;
@@ -307,12 +313,14 @@ public abstract class Selection
     // Special cased selection for when no function is used (this save some allocations).
     private static class SimpleSelection extends Selection
     {
-        public SimpleSelection(List<CFDefinition.Name> columnsList)
+        private final boolean isWildcard;
+
+        public SimpleSelection(List<CFDefinition.Name> columnsList, boolean isWildcard)
         {
-            this(columnsList, new ArrayList<ColumnSpecification>(columnsList));
+            this(columnsList, new ArrayList<ColumnSpecification>(columnsList), isWildcard);
         }
 
-        public SimpleSelection(List<CFDefinition.Name> columnsList, List<ColumnSpecification> metadata)
+        public SimpleSelection(List<CFDefinition.Name> columnsList, List<ColumnSpecification> metadata, boolean isWildcard)
         {
             /*
              * In theory, even a simple selection could have multiple time the same column, so we
@@ -320,12 +328,19 @@ public abstract class Selection
              * get much duplicate in practice, it's more efficient not to bother.
              */
             super(columnsList, metadata, false, false);
+            this.isWildcard = isWildcard;
         }
 
         protected List<ByteBuffer> handleRow(ResultSetBuilder rs)
         {
             return rs.current;
         }
+
+        @Override
+        public boolean isWildcard()
+        {
+            return isWildcard;
+        }
     }
 
     private interface Selector extends AssignementTestable

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
index a387962..0e6481b 100644
--- a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
@@ -35,9 +35,9 @@ public class UpdateStatement extends ModificationStatement
 {
     private static final Operation setToEmptyOperation = new Constants.Setter(null, new Constants.Value(ByteBufferUtil.EMPTY_BYTE_BUFFER));
 
-    private UpdateStatement(CFMetaData cfm, Attributes attrs)
+    private UpdateStatement(StatementType type, CFMetaData cfm, Attributes attrs)
     {
-        super(cfm, attrs);
+        super(type, cfm, attrs);
     }
 
     public boolean requireFullClusteringKey()
@@ -72,9 +72,9 @@ public class UpdateStatement extends ModificationStatement
         if (cfDef.isCompact)
         {
             if (builder.componentCount() == 0)
-                throw new InvalidRequestException(String.format("Missing PRIMARY KEY part %s", cfDef.columns.values().iterator().next()));
+                throw new InvalidRequestException(String.format("Missing PRIMARY KEY part %s", cfDef.clusteringColumns().iterator().next()));
 
-            if (cfDef.value == null)
+            if (cfDef.compactValue() == null)
             {
                 // compact + no compact value implies there is no column outside the PK. So no operation could
                 // have passed through validation
@@ -85,7 +85,7 @@ public class UpdateStatement extends ModificationStatement
             {
                 // compact means we don't have a row marker, so don't accept to set only the PK. See CASSANDRA-5648.
                 if (updates.isEmpty())
-                    throw new InvalidRequestException(String.format("Column %s is mandatory for this COMPACT STORAGE table", cfDef.value));
+                    throw new InvalidRequestException(String.format("Column %s is mandatory for this COMPACT STORAGE table", cfDef.compactValue()));
 
                 for (Operation update : updates)
                     update.execute(key, cf, builder.copy(), params);
@@ -131,7 +131,7 @@ public class UpdateStatement extends ModificationStatement
 
         protected ModificationStatement prepareInternal(CFDefinition cfDef, VariableSpecifications boundNames, Attributes attrs) throws InvalidRequestException
         {
-            UpdateStatement stmt = new UpdateStatement(cfDef.cfm, attrs);
+            UpdateStatement stmt = new UpdateStatement(ModificationStatement.StatementType.INSERT, cfDef.cfm, attrs);
 
             // Created from an INSERT
             if (stmt.isCounter())
@@ -159,10 +159,11 @@ public class UpdateStatement extends ModificationStatement
                     case COLUMN_ALIAS:
                         Term t = value.prepare(name);
                         t.collectMarkerSpecification(boundNames);
-                        stmt.addKeyValue(name.name, t);
+                        stmt.addKeyValue(name, t);
                         break;
                     case VALUE_ALIAS:
                     case COLUMN_METADATA:
+                    case STATIC:
                         Operation operation = new Operation.SetValue(value).prepare(name);
                         operation.collectMarkerSpecification(boundNames);
                         stmt.addOperation(operation);
@@ -192,7 +193,7 @@ public class UpdateStatement extends ModificationStatement
                             Attributes.Raw attrs,
                             List<Pair<ColumnIdentifier, Operation.RawUpdate>> updates,
                             List<Relation> whereClause,
-                            List<Pair<ColumnIdentifier, Operation.RawUpdate>> conditions)
+                            List<Pair<ColumnIdentifier, ColumnCondition.Raw>> conditions)
         {
             super(name, attrs, conditions, false);
             this.updates = updates;
@@ -201,7 +202,7 @@ public class UpdateStatement extends ModificationStatement
 
         protected ModificationStatement prepareInternal(CFDefinition cfDef, VariableSpecifications boundNames, Attributes attrs) throws InvalidRequestException
         {
-            UpdateStatement stmt = new UpdateStatement(cfDef.cfm, attrs);
+            UpdateStatement stmt = new UpdateStatement(ModificationStatement.StatementType.UPDATE, cfDef.cfm, attrs);
 
             for (Pair<ColumnIdentifier, Operation.RawUpdate> entry : updates)
             {
@@ -219,6 +220,7 @@ public class UpdateStatement extends ModificationStatement
                         throw new InvalidRequestException(String.format("PRIMARY KEY part %s found in SET part", entry.left));
                     case VALUE_ALIAS:
                     case COLUMN_METADATA:
+                    case STATIC:
                         stmt.addOperation(operation);
                         break;
                 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/db/filter/ColumnSlice.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ColumnSlice.java b/src/java/org/apache/cassandra/db/filter/ColumnSlice.java
index e0c576e..9eff12a 100644
--- a/src/java/org/apache/cassandra/db/filter/ColumnSlice.java
+++ b/src/java/org/apache/cassandra/db/filter/ColumnSlice.java
@@ -63,7 +63,6 @@ public class ColumnSlice
         return !finish.equals(ByteBufferUtil.EMPTY_BYTE_BUFFER) && cmp.compare(finish, name) < 0;
     }
 
-
     @Override
     public final int hashCode()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java b/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
index 90e7089..eb618f4 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
@@ -26,10 +26,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.cql3.ColumnNameBuilder;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.filter.ExtendedFilter;
-import org.apache.cassandra.db.filter.IDiskAtomFilter;
-import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.db.filter.SliceQueryFilter;
+import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.db.index.SecondaryIndexManager;
 import org.apache.cassandra.db.index.SecondaryIndexSearcher;
 import org.apache.cassandra.db.marshal.CompositeType;
@@ -238,11 +235,28 @@ public class CompositesSearcher extends SecondaryIndexSearcher
 
                         // We always query the whole CQL3 row. In the case where the original filter was a name filter this might be
                         // slightly wasteful, but this probably doesn't matter in practice and it simplify things.
-                        SliceQueryFilter dataFilter = new SliceQueryFilter(start,
-                                                                           entry.indexedEntryEnd(),
-                                                                           false,
-                                                                           Integer.MAX_VALUE,
-                                                                           baseCfs.metadata.clusteringKeyColumns().size());
+                        ColumnSlice dataSlice = new ColumnSlice(start, entry.indexedEntryEnd());
+                        ColumnSlice[] slices;
+                        if (baseCfs.metadata.hasStaticColumns())
+                        {
+                            // If the table has static columns, we must fetch them too as they may need to be returned too.
+                            // Note that this is potentially wasteful for 2 reasons:
+                            //  1) we will retrieve the static parts for each indexed row, even if we have more than one row in
+                            //     the same partition. If we were to group data queries to rows on the same slice, which would
+                            //     speed up things in general, we would also optimize here since we would fetch static columns only
+                            //     once for each group.
+                            //  2) at this point we don't know if the user asked for static columns or not, so we might be fetching
+                            //     them for nothing. We would however need to ship the list of "CQL3 columns selected" with getRangeSlice
+                            //     to be able to know that.
+                            // TODO: we should improve both point above
+                            ColumnSlice staticSlice = new ColumnSlice(ByteBufferUtil.EMPTY_BYTE_BUFFER, baseCfs.metadata.getStaticColumnNameBuilder().buildAsEndOfRange());
+                            slices = new ColumnSlice[]{ staticSlice, dataSlice };
+                        }
+                        else
+                        {
+                            slices = new ColumnSlice[]{ dataSlice };
+                        }
+                        SliceQueryFilter dataFilter = new SliceQueryFilter(slices, false, Integer.MAX_VALUE, baseCfs.metadata.clusteringKeyColumns().size());
                         ColumnFamily newData = baseCfs.getColumnFamily(new QueryFilter(dk, baseCfs.name, dataFilter, filter.timestamp));
                         if (newData == null || index.isStale(entry, newData, filter.timestamp))
                         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java b/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java
index d002aa7..9250b0f 100644
--- a/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java
@@ -34,6 +34,7 @@ import java.util.List;
  */
 public abstract class AbstractCompositeType extends AbstractType<ByteBuffer>
 {
+
     // changes bb position
     protected static int getShortLength(ByteBuffer bb)
     {
@@ -41,6 +42,13 @@ public abstract class AbstractCompositeType extends AbstractType<ByteBuffer>
         return length | (bb.get() & 0xFF);
     }
 
+    // Doesn't change bb position
+    protected static int getShortLength(ByteBuffer bb, int position)
+    {
+        int length = (bb.get(position) & 0xFF) << 8;
+        return length | (bb.get(position + 1) & 0xFF);
+    }
+
     // changes bb position
     protected static void putShortLength(ByteBuffer bb, int length)
     {
@@ -66,11 +74,17 @@ public abstract class AbstractCompositeType extends AbstractType<ByteBuffer>
 
     public int compare(ByteBuffer o1, ByteBuffer o2)
     {
-        if (o1 == null)
-            return o2 == null ? 0 : -1;
+        if (o1 == null || !o1.hasRemaining())
+            return o2 == null || !o2.hasRemaining() ? 0 : -1;
 
         ByteBuffer bb1 = o1.duplicate();
         ByteBuffer bb2 = o2.duplicate();
+
+        boolean isStatic1 = readIsStatic(bb1);
+        boolean isStatic2 = readIsStatic(bb2);
+        if (isStatic1 != isStatic2)
+            return isStatic1 ? -1 : 1;
+
         int i = 0;
 
         ByteBuffer previous = null;
@@ -90,22 +104,9 @@ public abstract class AbstractCompositeType extends AbstractType<ByteBuffer>
 
             byte b1 = bb1.get();
             byte b2 = bb2.get();
-            if (b1 < 0)
-            {
-                if (b2 >= 0)
-                    return -1;
-            }
-            else if (b1 > 0)
-            {
-                if (b2 <= 0)
-                    return 1;
-            }
-            else
-            {
-                // b1 == 0
-                if (b2 != 0)
-                    return -b2;
-            }
+            if (b1 != b2)
+                return b1 - b2;
+
             ++i;
         }
 
@@ -116,6 +117,10 @@ public abstract class AbstractCompositeType extends AbstractType<ByteBuffer>
         return 1;
     }
 
+    // Check if the provided BB represents a static name and advance the
+    // buffer to the real beginning if so.
+    protected abstract boolean readIsStatic(ByteBuffer bb);
+
     /**
      * Split a composite column names into it's components.
      */
@@ -123,6 +128,7 @@ public abstract class AbstractCompositeType extends AbstractType<ByteBuffer>
     {
         List<ByteBuffer> l = new ArrayList<ByteBuffer>();
         ByteBuffer bb = name.duplicate();
+        readIsStatic(bb);
         int i = 0;
         while (bb.remaining() > 0)
         {
@@ -150,6 +156,7 @@ public abstract class AbstractCompositeType extends AbstractType<ByteBuffer>
         List<CompositeComponent> list = new ArrayList<CompositeComponent>();
 
         ByteBuffer bb = bytes.duplicate();
+        readIsStatic(bb);
         int i = 0;
 
         while (bb.remaining() > 0)
@@ -219,8 +226,9 @@ public abstract class AbstractCompositeType extends AbstractType<ByteBuffer>
     {
         StringBuilder sb = new StringBuilder();
         ByteBuffer bb = bytes.duplicate();
-        int i = 0;
+        readIsStatic(bb);
 
+        int i = 0;
         while (bb.remaining() > 0)
         {
             if (bb.remaining() != bytes.remaining())
@@ -290,6 +298,7 @@ public abstract class AbstractCompositeType extends AbstractType<ByteBuffer>
     public void validate(ByteBuffer bytes) throws MarshalException
     {
         ByteBuffer bb = bytes.duplicate();
+        readIsStatic(bb);
 
         int i = 0;
         ByteBuffer previous = null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/db/marshal/CompositeType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/CompositeType.java b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
index 2c0e121..83e3b97 100644
--- a/src/java/org/apache/cassandra/db/marshal/CompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
@@ -42,8 +42,8 @@ import org.apache.cassandra.utils.ByteBufferUtil;
  *   <component><component><component> ...
  * where <component> is:
  *   <length of value><value><'end-of-component' byte>
- * where <length of value> is a 2 bytes unsigned short the and the
- * 'end-of-component' byte should always be 0 for actual column name.
+ * where <length of value> is a 2 bytes unsigned short (but 0xFFFF is invalid, see
+ * below) and the 'end-of-component' byte should always be 0 for actual column name.
  * However, it can set to 1 for query bounds. This allows to query for the
  * equivalent of 'give me the full super-column'. That is, if during a slice
  * query uses:
@@ -56,9 +56,16 @@ import org.apache.cassandra.utils.ByteBufferUtil;
  *   start = <3><"foo".getBytes()><-1>
  * allows to query everything that is greater than <3><"foo".getBytes()>, but
  * not <3><"foo".getBytes()> itself.
+ *
+ * On top of that, CQL3 uses a specific prefix (0xFFFF) to encode "static columns"
+ * (CASSANDRA-6561). This does mean the maximum size of the first component of a
+ * composite is 65534, not 65535 (or we wouldn't be able to detect if the first 2
+ * bytes is the static prefix or not).
  */
 public class CompositeType extends AbstractCompositeType
 {
+    public static final int STATIC_MARKER = 0xFFFF;
+
     public final List<AbstractType<?>> types;
 
     // interning instances
@@ -74,6 +81,24 @@ public class CompositeType extends AbstractCompositeType
         return getInstance(Arrays.<AbstractType<?>>asList(types));
     }
 
+    protected boolean readIsStatic(ByteBuffer bb)
+    {
+        return readStatic(bb);
+    }
+
+    private static boolean readStatic(ByteBuffer bb)
+    {
+        if (bb.remaining() < 2)
+            return false;
+
+        int header = getShortLength(bb, bb.position());
+        if ((header & 0xFFFF) != STATIC_MARKER)
+            return false;
+
+        getShortLength(bb); // Skip header
+        return true;
+    }
+
     public static synchronized CompositeType getInstance(List<AbstractType<?>> types)
     {
         assert types != null && !types.isEmpty();
@@ -149,6 +174,7 @@ public class CompositeType extends AbstractCompositeType
     public static ByteBuffer extractComponent(ByteBuffer bb, int idx)
     {
         bb = bb.duplicate();
+        readStatic(bb);
         int i = 0;
         while (bb.remaining() > 0)
         {
@@ -169,6 +195,11 @@ public class CompositeType extends AbstractCompositeType
         return extractComponent(bb, idx);
     }
 
+    public static boolean isStaticName(ByteBuffer bb)
+    {
+        return bb.remaining() >= 2 && (getShortLength(bb, bb.position()) & 0xFFFF) == STATIC_MARKER;
+    }
+
     @Override
     public int componentsCount()
     {
@@ -317,24 +348,33 @@ public class CompositeType extends AbstractCompositeType
         private final List<ByteBuffer> components;
         private final byte[] endOfComponents;
         private int serializedSize;
+        private final boolean isStatic;
 
         public Builder(CompositeType composite)
         {
-            this(composite, new ArrayList<ByteBuffer>(composite.types.size()), new byte[composite.types.size()]);
+            this(composite, new ArrayList<ByteBuffer>(composite.types.size()), new byte[composite.types.size()], false);
+        }
+
+        public static Builder staticBuilder(CompositeType composite)
+        {
+            return new Builder(composite, new ArrayList<ByteBuffer>(composite.types.size()), new byte[composite.types.size()], true);
         }
 
-        public Builder(CompositeType composite, List<ByteBuffer> components, byte[] endOfComponents)
+        private Builder(CompositeType composite, List<ByteBuffer> components, byte[] endOfComponents, boolean isStatic)
         {
             assert endOfComponents.length == composite.types.size();
 
             this.composite = composite;
             this.components = components;
             this.endOfComponents = endOfComponents;
+            this.isStatic = isStatic;
+            if (isStatic)
+                serializedSize = 2;
         }
 
         private Builder(Builder b)
         {
-            this(b.composite, new ArrayList<ByteBuffer>(b.components), Arrays.copyOf(b.endOfComponents, b.endOfComponents.length));
+            this(b.composite, new ArrayList<ByteBuffer>(b.components), Arrays.copyOf(b.endOfComponents, b.endOfComponents.length), b.isStatic);
             this.serializedSize = b.serializedSize;
         }
 
@@ -344,6 +384,7 @@ public class CompositeType extends AbstractCompositeType
                 throw new IllegalStateException("Composite column is already fully constructed");
 
             components.add(bb);
+            serializedSize += 3 + bb.remaining(); // 2 bytes lenght + 1 byte eoc
             return this;
         }
 
@@ -364,20 +405,23 @@ public class CompositeType extends AbstractCompositeType
 
         public ByteBuffer build()
         {
-            DataOutputBuffer out = new DataOutputBuffer(serializedSize);
-            for (int i = 0; i < components.size(); i++)
+            try
             {
-                try
+                DataOutputBuffer out = new DataOutputBuffer(serializedSize);
+                if (isStatic)
+                    out.writeShort(STATIC_MARKER);
+
+                for (int i = 0; i < components.size(); i++)
                 {
                     ByteBufferUtil.writeWithShortLength(components.get(i), out);
+                    out.write(endOfComponents[i]);
                 }
-                catch (IOException e)
-                {
-                    throw new RuntimeException(e);
-                }
-                out.write(endOfComponents[i]);
+                return ByteBuffer.wrap(out.getData(), 0, out.getLength());
+            }
+            catch (IOException e)
+            {
+                throw new RuntimeException(e);
             }
-            return ByteBuffer.wrap(out.getData(), 0, out.getLength());
         }
 
         public ByteBuffer buildAsEndOfRange()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java b/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java
index 35e6e33..9b56a82 100644
--- a/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java
@@ -75,6 +75,12 @@ public class DynamicCompositeType extends AbstractCompositeType
         this.aliases = aliases;
     }
 
+    protected boolean readIsStatic(ByteBuffer bb)
+    {
+        // We don't have the static nothing for DCT
+        return false;
+    }
+
     private AbstractType<?> getComparator(ByteBuffer bb)
     {
         try

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
index 3fb1c5a..65e3be1 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
@@ -25,6 +25,7 @@ import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;
 import java.util.*;
 
+import com.google.common.collect.Iterables;
 
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
@@ -674,11 +675,11 @@ public abstract class AbstractCassandraStorage extends LoadFunc implements Store
 
             // otherwise for CqlStorage, check metadata for classic thrift tables
             CFDefinition cfDefinition = getCfDefinition(keyspace, column_family, client);
-            for (ColumnIdentifier column : cfDefinition.metadata.keySet())
+            for (CFDefinition.Name column : Iterables.concat(cfDefinition.staticColumns(), cfDefinition.regularColumns()))
             {
                 ColumnDef cDef = new ColumnDef();
-                String columnName = column.toString();
-                String type = cfDefinition.metadata.get(column).type.toString();
+                String columnName = column.name.toString();
+                String type = column.type.toString();
                 logger.debug("name: {}, type: {} ", columnName, type);
                 cDef.name = ByteBufferUtil.bytes(columnName);
                 cDef.validation_class = type;
@@ -688,12 +689,12 @@ public abstract class AbstractCassandraStorage extends LoadFunc implements Store
             // could have already processed it as schema_columnfamilies.value_alias
             if (columnDefs.size() == 0 && includeCompactValueColumn)
             {
-                String value = cfDefinition.value != null ? cfDefinition.value.toString() : null;
+                String value = cfDefinition.compactValue() != null ? cfDefinition.compactValue().toString() : null;
                 if ("value".equals(value))
                 {
                     ColumnDef cDef = new ColumnDef();
                     cDef.name = ByteBufferUtil.bytes(value);
-                    cDef.validation_class = cfDefinition.value.type.toString();
+                    cDef.validation_class = cfDefinition.compactValue().type.toString();
                     columnDefs.add(cDef);
                 }
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
index c6db82f..a5156b9 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
@@ -524,17 +524,17 @@ public class CqlStorage extends AbstractCassandraStorage
             if (keys.size() == 0)
             {
                 CFDefinition cfDefinition = getCfDefinition(keyspace, column_family, client);
-                for (ColumnIdentifier column : cfDefinition.keys.keySet())
+                for (CFDefinition.Name column : cfDefinition.partitionKeys())
                 {
-                    String key = column.toString();
+                    String key = column.name.toString();
                     logger.debug("name: {} ", key);
                     ColumnDef cDef = new ColumnDef();
                     cDef.name = ByteBufferUtil.bytes(key);
                     keys.add(cDef);
                 }
-                for (ColumnIdentifier column : cfDefinition.columns.keySet())
+                for (CFDefinition.Name column : cfDefinition.clusteringColumns())
                 {
-                    String key = column.toString();
+                    String key = column.name.toString();
                     logger.debug("name: {} ", key);
                     ColumnDef cDef = new ColumnDef();
                     cDef.name = ByteBufferUtil.bytes(key);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/service/CASConditions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/CASConditions.java b/src/java/org/apache/cassandra/service/CASConditions.java
index d4b3e19..c0a2111 100644
--- a/src/java/org/apache/cassandra/service/CASConditions.java
+++ b/src/java/org/apache/cassandra/service/CASConditions.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.service;
 
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.filter.IDiskAtomFilter;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 
 /**
  * Abstract the conditions to be fulfilled by a CAS operation.
@@ -34,5 +35,5 @@ public interface CASConditions
      * Returns whether the provided CF, that represents the values fetched using the
      * readFilter(), match the CAS conditions this object stands for.
      */
-    public boolean appliesTo(ColumnFamily current);
+    public boolean appliesTo(ColumnFamily current) throws InvalidRequestException;
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/ThriftValidation.java b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
index f2efc03..2b435ff 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@ -245,7 +245,8 @@ public class ThriftValidation
                     continue; // Row marker, ok
 
                 ColumnIdentifier columnId = new ColumnIdentifier(CQL3ColumnName, composite.types.get(columnIndex));
-                if (cfDef.metadata.get(columnId) == null)
+                CFDefinition.Name columnName = cfDef.get(columnId);
+                if (columnName == null || columnName.isPrimaryKeyColumn())
                     throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("Invalid cell for CQL3 table %s. The CQL3 column component (%s) does not correspond to a defined CQL3 column",
                                                                                                     metadata.cfName, columnId));
 


[4/7] Merge branch 'cassandra-2.0' into cassandra-2.1

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/AbstractCType.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/AbstractCType.java
index 2b07a46,0000000..d3b8940
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/AbstractCType.java
+++ b/src/java/org/apache/cassandra/db/composites/AbstractCType.java
@@@ -1,336 -1,0 +1,339 @@@
 +/*
 + * 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.composites;
 +
 +import java.io.DataInput;
 +import java.io.DataOutput;
 +import java.io.IOException;
 +import java.nio.ByteBuffer;
 +import java.util.Comparator;
 +
 +import org.apache.cassandra.db.DeletionInfo;
 +import org.apache.cassandra.db.RangeTombstone;
 +import org.apache.cassandra.db.RowIndexEntry;
 +import org.apache.cassandra.db.TypeSizes;
 +import org.apache.cassandra.db.filter.ColumnSlice;
 +import org.apache.cassandra.db.filter.SliceQueryFilter;
 +import org.apache.cassandra.db.marshal.AbstractCompositeType;
 +import org.apache.cassandra.db.marshal.AbstractType;
 +import org.apache.cassandra.io.ISerializer;
 +import org.apache.cassandra.io.IVersionedSerializer;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +
 +import static org.apache.cassandra.io.sstable.IndexHelper.IndexInfo;
 +
 +public abstract class AbstractCType implements CType
 +{
 +    private final Comparator<Composite> reverseComparator;
 +    private final Comparator<IndexInfo> indexComparator;
 +    private final Comparator<IndexInfo> indexReverseComparator;
 +
 +    private final Serializer serializer;
 +
 +    private final ISerializer<IndexInfo> indexSerializer;
 +    private final IVersionedSerializer<ColumnSlice> sliceSerializer;
 +    private final IVersionedSerializer<SliceQueryFilter> sliceQueryFilterSerializer;
 +    private final DeletionInfo.Serializer deletionInfoSerializer;
 +    private final RangeTombstone.Serializer rangeTombstoneSerializer;
 +    private final RowIndexEntry.Serializer rowIndexEntrySerializer;
 +
 +    protected AbstractCType()
 +    {
 +        reverseComparator = new Comparator<Composite>()
 +        {
 +            public int compare(Composite c1, Composite c2)
 +            {
 +                return AbstractCType.this.compare(c2, c1);
 +            }
 +        };
 +        indexComparator = new Comparator<IndexInfo>()
 +        {
 +            public int compare(IndexInfo o1, IndexInfo o2)
 +            {
 +                return AbstractCType.this.compare(o1.lastName, o2.lastName);
 +            }
 +        };
 +        indexReverseComparator = new Comparator<IndexInfo>()
 +        {
 +            public int compare(IndexInfo o1, IndexInfo o2)
 +            {
 +                return AbstractCType.this.compare(o1.firstName, o2.firstName);
 +            }
 +        };
 +
 +        serializer = new Serializer(this);
 +
 +        indexSerializer = new IndexInfo.Serializer(this);
 +        sliceSerializer = new ColumnSlice.Serializer(this);
 +        sliceQueryFilterSerializer = new SliceQueryFilter.Serializer(this);
 +        deletionInfoSerializer = new DeletionInfo.Serializer(this);
 +        rangeTombstoneSerializer = new RangeTombstone.Serializer(this);
 +        rowIndexEntrySerializer = new RowIndexEntry.Serializer(this);
 +    }
 +
 +    public int compare(Composite c1, Composite c2)
 +    {
-         if (c1 == null)
-             return c2 == null ? 0 : -1;
++        if (c1 == null || c1.isEmpty())
++            return c2 == null || c2.isEmpty() ? 0 : -1;
++
++        if (c1.isStatic() != c2.isStatic())
++            return c1.isStatic() ? -1 : 1;
 +
 +        ByteBuffer previous = null;
 +        int i;
 +        int minSize = Math.min(c1.size(), c2.size());
 +        for (i = 0; i < minSize; i++)
 +        {
 +            AbstractType<?> comparator = subtype(i);
 +            ByteBuffer value1 = c1.get(i);
 +            ByteBuffer value2 = c2.get(i);
 +
 +            int cmp = comparator.compareCollectionMembers(value1, value2, previous);
 +            if (cmp != 0)
 +                return cmp;
 +
 +            previous = value1;
 +        }
 +
 +        if (c1.size() == c2.size())
 +        {
 +            if (c1.eoc() != c2.eoc())
 +            {
 +                switch (c1.eoc())
 +                {
 +                    case START: return -1;
 +                    case END:   return 1;
 +                    case NONE:  return c2.eoc() == Composite.EOC.START ? 1 : -1;
 +                }
 +            }
 +            return 0;
 +        }
 +
 +        if (i == c1.size())
 +        {
 +            return c1.eoc() == Composite.EOC.END ? 1 : -1;
 +        }
 +        else
 +        {
 +            assert i == c2.size();
 +            return c2.eoc() == Composite.EOC.END ? -1 : 1;
 +        }
 +    }
 +
 +    public void validate(Composite name)
 +    {
 +        ByteBuffer previous = null;
 +        for (int i = 0; i < name.size(); i++)
 +        {
 +            AbstractType<?> comparator = subtype(i);
 +            ByteBuffer value = name.get(i);
 +            comparator.validateCollectionMember(value, previous);
 +            previous = value;
 +        }
 +    }
 +
 +    public boolean isCompatibleWith(CType previous)
 +    {
 +        if (this == previous)
 +            return true;
 +
 +        // Extending with new components is fine, shrinking is not
 +        if (size() < previous.size())
 +            return false;
 +
 +        for (int i = 0; i < previous.size(); i++)
 +        {
 +            AbstractType<?> tprev = previous.subtype(i);
 +            AbstractType<?> tnew = subtype(i);
 +            if (!tnew.isCompatibleWith(tprev))
 +                return false;
 +        }
 +        return true;
 +    }
 +
 +    public String getString(Composite c)
 +    {
 +        StringBuilder sb = new StringBuilder();
 +        for (int i = 0; i < c.size(); i++)
 +        {
 +            if (i > 0)
 +                sb.append(":");
 +            sb.append(AbstractCompositeType.escape(subtype(i).getString(c.get(i))));
 +        }
 +        switch (c.eoc())
 +        {
 +            case START:
 +                sb.append(":_");
 +                break;
 +            case END:
 +                sb.append(":!");
 +                break;
 +        }
 +        return sb.toString();
 +    }
 +
 +    public Composite make(Object... components)
 +    {
 +        if (components.length > size())
 +            throw new IllegalArgumentException("Too many components, max is " + size());
 +
 +        CBuilder builder = builder();
 +        for (int i = 0; i < components.length; i++)
 +        {
 +            Object obj = components[i];
 +            if (obj instanceof ByteBuffer)
 +                builder.add((ByteBuffer)obj);
 +            else
 +                builder.add(obj);
 +        }
 +        return builder.build();
 +    }
 +
 +    public CType.Serializer serializer()
 +    {
 +        return serializer;
 +    }
 +
 +    public Comparator<Composite> reverseComparator()
 +    {
 +        return reverseComparator;
 +    }
 +
 +    public Comparator<IndexInfo> indexComparator()
 +    {
 +        return indexComparator;
 +    }
 +
 +    public Comparator<IndexInfo> indexReverseComparator()
 +    {
 +        return indexReverseComparator;
 +    }
 +
 +    public ISerializer<IndexInfo> indexSerializer()
 +    {
 +        return indexSerializer;
 +    }
 +
 +    public IVersionedSerializer<ColumnSlice> sliceSerializer()
 +    {
 +        return sliceSerializer;
 +    }
 +
 +    public IVersionedSerializer<SliceQueryFilter> sliceQueryFilterSerializer()
 +    {
 +        return sliceQueryFilterSerializer;
 +    }
 +
 +    public DeletionInfo.Serializer deletionInfoSerializer()
 +    {
 +        return deletionInfoSerializer;
 +    }
 +
 +    public RangeTombstone.Serializer rangeTombstoneSerializer()
 +    {
 +        return rangeTombstoneSerializer;
 +    }
 +
 +    public RowIndexEntry.Serializer rowIndexEntrySerializer()
 +    {
 +        return rowIndexEntrySerializer;
 +    }
 +
 +    @Override
 +    public boolean equals(Object o)
 +    {
 +        if (this == o)
 +            return true;
 +
 +        if (o == null)
 +            return false;
 +
 +        if (!getClass().equals(o.getClass()))
 +            return false;
 +
 +        CType c = (CType)o;
 +        if (size() != c.size())
 +            return false;
 +
 +        for (int i = 0; i < size(); i++)
 +        {
 +            if (!subtype(i).equals(c.subtype(i)))
 +                return false;
 +        }
 +        return true;
 +    }
 +
 +    @Override
 +    public int hashCode()
 +    {
 +        int h = 31;
 +        for (int i = 0; i < size(); i++)
 +            h += subtype(i).hashCode();
 +        return h + getClass().hashCode();
 +    }
 +
 +    @Override
 +    public String toString()
 +    {
 +        return asAbstractType().toString();
 +    }
 +
 +    protected static ByteBuffer sliceBytes(ByteBuffer bb, int offs, int length)
 +    {
 +        ByteBuffer copy = bb.duplicate();
 +        copy.position(offs);
 +        copy.limit(offs + length);
 +        return copy;
 +    }
 +
 +    protected static void checkRemaining(ByteBuffer bb, int offs, int length)
 +    {
 +        if (offs + length > bb.limit())
 +            throw new IllegalArgumentException("Not enough bytes");
 +    }
 +
 +    private static class Serializer implements CType.Serializer
 +    {
 +        private final CType type;
 +
 +        public Serializer(CType type)
 +        {
 +            this.type = type;
 +        }
 +
 +        public void serialize(Composite c, DataOutput out) throws IOException
 +        {
 +            ByteBufferUtil.writeWithShortLength(c.toByteBuffer(), out);
 +        }
 +
 +        public Composite deserialize(DataInput in) throws IOException
 +        {
 +            return type.fromByteBuffer(ByteBufferUtil.readWithShortLength(in));
 +        }
 +
 +        public long serializedSize(Composite c, TypeSizes type)
 +        {
 +            return type.sizeofWithShortLength(c.toByteBuffer());
 +        }
 +
 +        public void skip(DataInput in) throws IOException
 +        {
 +            ByteBufferUtil.skipShortLength(in);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/AbstractCellNameType.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/AbstractCellNameType.java
index 96fc1eb,0000000..6d4ee12
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/AbstractCellNameType.java
+++ b/src/java/org/apache/cassandra/db/composites/AbstractCellNameType.java
@@@ -1,370 -1,0 +1,428 @@@
 +/*
 + * 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.composites;
 +
 +import java.io.DataInput;
 +import java.io.DataOutput;
 +import java.io.IOException;
 +import java.nio.ByteBuffer;
 +import java.util.*;
 +
 +import com.google.common.collect.AbstractIterator;
++import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.CQL3Row;
 +import org.apache.cassandra.cql3.ColumnIdentifier;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.filter.IDiskAtomFilter;
 +import org.apache.cassandra.db.filter.NamesQueryFilter;
 +import org.apache.cassandra.db.marshal.AbstractType;
 +import org.apache.cassandra.db.marshal.CollectionType;
 +import org.apache.cassandra.db.marshal.ColumnToCollectionType;
 +import org.apache.cassandra.io.ISerializer;
 +import org.apache.cassandra.io.IVersionedSerializer;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +
 +public abstract class AbstractCellNameType extends AbstractCType implements CellNameType
 +{
 +    private final Comparator<Cell> columnComparator;
 +    private final Comparator<Cell> columnReverseComparator;
 +    private final Comparator<OnDiskAtom> onDiskAtomComparator;
 +
 +    private final ISerializer<CellName> cellSerializer;
 +    private final ColumnSerializer columnSerializer;
 +    private final OnDiskAtom.Serializer onDiskAtomSerializer;
 +    private final IVersionedSerializer<NamesQueryFilter> namesQueryFilterSerializer;
 +    private final IVersionedSerializer<IDiskAtomFilter> diskAtomFilterSerializer;
 +
 +    protected AbstractCellNameType()
 +    {
 +        columnComparator = new Comparator<Cell>()
 +        {
 +            public int compare(Cell c1, Cell c2)
 +            {
 +                return AbstractCellNameType.this.compare(c1.name(), c2.name());
 +            }
 +        };
 +        columnReverseComparator = new Comparator<Cell>()
 +        {
 +            public int compare(Cell c1, Cell c2)
 +            {
 +                return AbstractCellNameType.this.compare(c2.name(), c1.name());
 +            }
 +        };
 +        onDiskAtomComparator = new Comparator<OnDiskAtom>()
 +        {
 +            public int compare(OnDiskAtom c1, OnDiskAtom c2)
 +            {
 +                int comp = AbstractCellNameType.this.compare(c1.name(), c2.name());
 +                if (comp != 0)
 +                    return comp;
 +
 +                if (c1 instanceof RangeTombstone)
 +                {
 +                    if (c2 instanceof RangeTombstone)
 +                    {
 +                        RangeTombstone t1 = (RangeTombstone)c1;
 +                        RangeTombstone t2 = (RangeTombstone)c2;
 +                        int comp2 = AbstractCellNameType.this.compare(t1.max, t2.max);
 +                        return comp2 == 0 ? t1.data.compareTo(t2.data) : comp2;
 +                    }
 +                    else
 +                    {
 +                        return -1;
 +                    }
 +                }
 +                else
 +                {
 +                    return c2 instanceof RangeTombstone ? 1 : 0;
 +                }
 +            }
 +        };
 +
 +        // A trivial wrapped over the composite serializer
 +        cellSerializer = new ISerializer<CellName>()
 +        {
 +            public void serialize(CellName c, DataOutput out) throws IOException
 +            {
 +                serializer().serialize(c, out);
 +            }
 +
 +            public CellName deserialize(DataInput in) throws IOException
 +            {
 +                Composite ct = serializer().deserialize(in);
 +                if (ct.isEmpty())
 +                    throw ColumnSerializer.CorruptColumnException.create(in, ByteBufferUtil.EMPTY_BYTE_BUFFER);
 +
 +                assert ct instanceof CellName : ct;
 +                return (CellName)ct;
 +            }
 +
 +            public long serializedSize(CellName c, TypeSizes type)
 +            {
 +                return serializer().serializedSize(c, type);
 +            }
 +        };
 +        columnSerializer = new ColumnSerializer(this);
 +        onDiskAtomSerializer = new OnDiskAtom.Serializer(this);
 +        namesQueryFilterSerializer = new NamesQueryFilter.Serializer(this);
 +        diskAtomFilterSerializer = new IDiskAtomFilter.Serializer(this);
 +    }
 +
 +    public Comparator<Cell> columnComparator()
 +    {
 +        return columnComparator;
 +    }
 +
 +    public Comparator<Cell> columnReverseComparator()
 +    {
 +        return columnReverseComparator;
 +    }
 +
 +    public Comparator<OnDiskAtom> onDiskAtomComparator()
 +    {
 +        return onDiskAtomComparator;
 +    }
 +
 +    public ISerializer<CellName> cellSerializer()
 +    {
 +        return cellSerializer;
 +    }
 +
 +    public ColumnSerializer columnSerializer()
 +    {
 +        return columnSerializer;
 +    }
 +
 +    public OnDiskAtom.Serializer onDiskAtomSerializer()
 +    {
 +        return onDiskAtomSerializer;
 +    }
 +
 +    public IVersionedSerializer<NamesQueryFilter> namesQueryFilterSerializer()
 +    {
 +        return namesQueryFilterSerializer;
 +    }
 +
 +    public IVersionedSerializer<IDiskAtomFilter> diskAtomFilterSerializer()
 +    {
 +        return diskAtomFilterSerializer;
 +    }
 +
 +    public CellName cellFromByteBuffer(ByteBuffer bytes)
 +    {
 +        return (CellName)fromByteBuffer(bytes);
 +    }
 +
-     public CellName create(Composite prefix, ColumnIdentifier columnName, ByteBuffer collectionElement)
++    public CellName create(Composite prefix, ColumnDefinition column, ByteBuffer collectionElement)
 +    {
 +        throw new UnsupportedOperationException();
 +    }
 +
 +    public CellName rowMarker(Composite prefix)
 +    {
 +        throw new UnsupportedOperationException();
 +    }
 +
++    public Composite staticPrefix()
++    {
++        throw new UnsupportedOperationException();
++    }
++
 +    public boolean hasCollections()
 +    {
 +        return false;
 +    }
 +
 +    public boolean supportCollections()
 +    {
 +        return false;
 +    }
 +
 +    public ColumnToCollectionType collectionType()
 +    {
 +        throw new UnsupportedOperationException();
 +    }
 +
 +    public CellNameType addCollection(ColumnIdentifier columnName, CollectionType newCollection)
 +    {
 +        throw new UnsupportedOperationException();
 +    }
 +
 +    @Override
 +    public Composite make(Object... components)
 +    {
 +        return components.length == size() ? makeCellName(components) : super.make(components);
 +    }
 +
 +    public CellName makeCellName(Object... components)
 +    {
 +        ByteBuffer[] rawComponents = new ByteBuffer[components.length];
 +        for (int i = 0; i < components.length; i++)
 +        {
 +            Object c = components[i];
 +            if (c instanceof ByteBuffer)
 +            {
 +                rawComponents[i] = (ByteBuffer)c;
 +            }
 +            else
 +            {
 +                AbstractType<?> type = subtype(i);
 +                // If it's a collection type, we need to find the right collection and use the key comparator (since we're building a cell name)
 +                if (type instanceof ColumnToCollectionType)
 +                {
 +                    assert i > 0;
 +                    type = ((ColumnToCollectionType)type).defined.get(rawComponents[i-1]).nameComparator();
 +                }
 +                rawComponents[i] = ((AbstractType)type).decompose(c);
 +            }
 +        }
 +        return makeCellName(rawComponents);
 +    }
 +
 +    protected abstract CellName makeCellName(ByteBuffer[] components);
 +
 +    protected static CQL3Row.Builder makeDenseCQL3RowBuilder(final long now)
 +    {
 +        return new CQL3Row.Builder()
 +        {
-             public Iterator<CQL3Row> group(final Iterator<Cell> cells)
++            public CQL3Row.RowIterator group(Iterator<Cell> cells)
 +            {
-                 return new AbstractIterator<CQL3Row>()
++                return new DenseRowIterator(cells, now);
++            }
++        };
++    }
++
++    private static class DenseRowIterator extends AbstractIterator<CQL3Row> implements CQL3Row.RowIterator
++    {
++        private final Iterator<Cell> cells;
++        private final long now;
++
++        public DenseRowIterator(Iterator<Cell> cells, long now)
++        {
++            this.cells = cells;
++            this.now = now;
++        }
++
++        public CQL3Row getStaticRow()
++        {
++            // There can't be static columns in dense tables
++            return null;
++        }
++
++        protected CQL3Row computeNext()
++        {
++            while (cells.hasNext())
++            {
++                final Cell cell = cells.next();
++                if (cell.isMarkedForDelete(now))
++                    continue;
++
++                return new CQL3Row()
 +                {
-                     protected CQL3Row computeNext()
++                    public ByteBuffer getClusteringColumn(int i)
++                    {
++                        return cell.name().get(i);
++                    }
++
++                    public Cell getColumn(ColumnIdentifier name)
 +                    {
-                         while (cells.hasNext())
-                         {
-                             final Cell cell = cells.next();
-                             if (cell.isMarkedForDelete(now))
-                                 continue;
- 
-                             return new CQL3Row()
-                             {
-                                 public ByteBuffer getClusteringColumn(int i)
-                                 {
-                                     return cell.name().get(i);
-                                 }
- 
-                                 public Cell getColumn(ColumnIdentifier name)
-                                 {
-                                     return cell;
-                                 }
- 
-                                 public List<Cell> getCollection(ColumnIdentifier name)
-                                 {
-                                     return null;
-                                 }
-                             };
-                         }
-                         return endOfData();
++                        return cell;
++                    }
++
++                    public List<Cell> getCollection(ColumnIdentifier name)
++                    {
++                        return null;
 +                    }
 +                };
 +            }
-         };
++            return endOfData();
++        }
 +    }
 +
 +    protected static CQL3Row.Builder makeSparseCQL3RowBuilder(final long now)
 +    {
 +        return new CQL3Row.Builder()
 +        {
-             public Iterator<CQL3Row> group(final Iterator<Cell> cells)
++            public CQL3Row.RowIterator group(Iterator<Cell> cells)
 +            {
-                 return new AbstractIterator<CQL3Row>()
++                return new SparseRowIterator(cells, now);
++            }
++        };
++    }
++
++    private static class SparseRowIterator extends AbstractIterator<CQL3Row> implements CQL3Row.RowIterator
++    {
++        private final Iterator<Cell> cells;
++        private final long now;
++        private final CQL3Row staticRow;
++
++        private Cell nextCell;
++        private CellName previous;
++        private CQL3RowOfSparse currentRow;
++
++        public SparseRowIterator(Iterator<Cell> cells, long now)
++        {
++            this.cells = cells;
++            this.now = now;
++            this.staticRow = hasNextCell() && nextCell.name().isStatic()
++                           ? computeNext()
++                           : null;
++        }
++
++        public CQL3Row getStaticRow()
++        {
++            return staticRow;
++        }
++
++        private boolean hasNextCell()
++        {
++            if (nextCell != null)
++                return true;
++
++            while (cells.hasNext())
++            {
++                Cell cell = cells.next();
++                if (cell.isMarkedForDelete(now))
++                    continue;
++
++                nextCell = cell;
++                return true;
++            }
++            return false;
++        }
++
++        protected CQL3Row computeNext()
++        {
++            while (hasNextCell())
++            {
++                CQL3Row toReturn = null;
++                CellName current = nextCell.name();
++                if (currentRow == null || !current.isSameCQL3RowAs(previous))
 +                {
-                     private CellName previous;
-                     private CQL3RowOfSparse currentRow;
++                    toReturn = currentRow;
++                    currentRow = new CQL3RowOfSparse(current);
++                }
++                currentRow.add(nextCell);
++                nextCell = null;
++                previous = current;
 +
-                     protected CQL3Row computeNext()
-                     {
-                         while (cells.hasNext())
-                         {
-                             final Cell cell = cells.next();
-                             if (cell.isMarkedForDelete(now))
-                                 continue;
- 
-                             CQL3Row toReturn = null;
-                             CellName current = cell.name();
-                             if (currentRow == null || !current.isSameCQL3RowAs(previous))
-                             {
-                                 toReturn = currentRow;
-                                 currentRow = new CQL3RowOfSparse(current);
-                             }
-                             currentRow.add(cell);
-                             previous = current;
- 
-                             if (toReturn != null)
-                                 return toReturn;
-                         }
-                         if (currentRow != null)
-                         {
-                             CQL3Row toReturn = currentRow;
-                             currentRow = null;
-                             return toReturn;
-                         }
-                         return endOfData();
-                     }
-                 };
++                if (toReturn != null)
++                    return toReturn;
 +            }
-         };
++            if (currentRow != null)
++            {
++                CQL3Row toReturn = currentRow;
++                currentRow = null;
++                return toReturn;
++            }
++            return endOfData();
++        }
 +    }
 +
 +    private static class CQL3RowOfSparse implements CQL3Row
 +    {
 +        private final CellName cell;
 +        private Map<ColumnIdentifier, Cell> columns;
 +        private Map<ColumnIdentifier, List<Cell>> collections;
 +
 +        CQL3RowOfSparse(CellName cell)
 +        {
 +            this.cell = cell;
 +        }
 +
 +        public ByteBuffer getClusteringColumn(int i)
 +        {
 +            return cell.get(i);
 +        }
 +
 +        void add(Cell cell)
 +        {
 +            CellName cellName = cell.name();
 +            ColumnIdentifier columnName =  cellName.cql3ColumnName();
 +            if (cellName.isCollectionCell())
 +            {
 +                if (collections == null)
 +                    collections = new HashMap<>();
 +
 +                List<Cell> values = collections.get(columnName);
 +                if (values == null)
 +                {
 +                    values = new ArrayList<Cell>();
 +                    collections.put(columnName, values);
 +                }
 +                values.add(cell);
 +            }
 +            else
 +            {
 +                if (columns == null)
 +                    columns = new HashMap<>();
 +                columns.put(columnName, cell);
 +            }
 +        }
 +
 +        public Cell getColumn(ColumnIdentifier name)
 +        {
 +            return columns == null ? null : columns.get(name);
 +        }
 +
 +        public List<Cell> getCollection(ColumnIdentifier name)
 +        {
 +            return collections == null ? null : collections.get(name);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/AbstractComposite.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/AbstractComposite.java
index e23c560,0000000..fbff930
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/AbstractComposite.java
+++ b/src/java/org/apache/cassandra/db/composites/AbstractComposite.java
@@@ -1,132 -1,0 +1,141 @@@
 +/*
 + * 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.composites;
 +
 +import java.nio.ByteBuffer;
 +
 +import org.apache.cassandra.db.filter.ColumnSlice;
 +import org.apache.cassandra.db.marshal.AbstractCompositeType;
++import org.apache.cassandra.db.marshal.CompositeType;
 +
 +public abstract class AbstractComposite implements Composite
 +{
 +    public boolean isEmpty()
 +    {
 +        return size() == 0;
 +    }
 +
++    public boolean isStatic()
++    {
++        return false;
++    }
++
 +    public EOC eoc()
 +    {
 +        return EOC.NONE;
 +    }
 +
 +    public Composite start()
 +    {
 +        return withEOC(EOC.START);
 +    }
 +
 +    public Composite end()
 +    {
 +        return withEOC(EOC.END);
 +    }
 +
 +    public Composite withEOC(EOC newEoc)
 +    {
 +        // Note: CompositeBound overwrite this so we assume the EOC of this is NONE
 +        switch (newEoc)
 +        {
 +            case START:
 +                return BoundedComposite.startOf(this);
 +            case END:
 +                return BoundedComposite.endOf(this);
 +            default:
 +                return this;
 +        }
 +    }
 +
 +    public ColumnSlice slice()
 +    {
 +        return new ColumnSlice(start(), end());
 +    }
 +
 +    public ByteBuffer toByteBuffer()
 +    {
 +        // This is the legacy format of composites.
 +        // See org.apache.cassandra.db.marshal.CompositeType for details.
-         ByteBuffer result = ByteBuffer.allocate(dataSize() + 3 * size());
++        ByteBuffer result = ByteBuffer.allocate(dataSize() + 3 * size() + (isStatic() ? 2 : 0));
++        if (isStatic())
++            AbstractCompositeType.putShortLength(result, CompositeType.STATIC_MARKER);
++
 +        for (int i = 0; i < size(); i++)
 +        {
 +            ByteBuffer bb = get(i);
 +            AbstractCompositeType.putShortLength(result, bb.remaining());
 +            result.put(bb.duplicate());
 +            result.put((byte)0);
 +        }
 +        result.flip();
 +        return result;
 +    }
 +
 +    public int dataSize()
 +    {
 +        int size = 0;
 +        for (int i = 0; i < size(); i++)
 +            size += get(i).remaining();
 +        return size;
 +    }
 +
 +    public boolean isPrefixOf(Composite c)
 +    {
-         if (size() > c.size())
++        if (size() > c.size() || isStatic() != c.isStatic())
 +            return false;
 +
 +        for (int i = 0; i < size(); i++)
 +        {
 +            if (!get(i).equals(c.get(i)))
 +                return false;
 +        }
 +        return true;
 +    }
 +
 +    @Override
 +    public boolean equals(Object o)
 +    {
 +        if (this == o)
 +            return true;
 +
 +        if(!(o instanceof Composite))
 +            return false;
 +
 +        Composite c = (Composite)o;
-         if (size() != c.size())
++        if (size() != c.size() || isStatic() != c.isStatic())
 +            return false;
 +
 +        for (int i = 0; i < size(); i++)
 +        {
 +            if (!get(i).equals(c.get(i)))
 +                return false;
 +        }
 +        return eoc() == c.eoc();
 +    }
 +
 +    @Override
 +    public int hashCode()
 +    {
 +        int h = 31;
 +        for (int i = 0; i < size(); i++)
 +            h += get(i).hashCode();
-         return h + eoc().hashCode();
++        return h + eoc().hashCode() + (isStatic() ? 1 : 0);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/AbstractCompoundCellNameType.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/AbstractCompoundCellNameType.java
index 0602f50,0000000..1cb605e
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/AbstractCompoundCellNameType.java
+++ b/src/java/org/apache/cassandra/db/composites/AbstractCompoundCellNameType.java
@@@ -1,264 -1,0 +1,289 @@@
 +/*
 + * 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.composites;
 +
 +import java.io.DataInput;
 +import java.io.IOException;
 +import java.nio.ByteBuffer;
 +
 +import org.apache.cassandra.db.marshal.AbstractType;
 +import org.apache.cassandra.db.marshal.CompositeType;
 +
 +public abstract class AbstractCompoundCellNameType extends AbstractCellNameType
 +{
 +    protected final CompoundCType clusteringType;
 +    protected final CompoundCType fullType;
 +
 +    protected final int clusteringSize;
 +    protected final int fullSize;
 +
 +    protected AbstractCompoundCellNameType(CompoundCType clusteringType, CompoundCType fullType)
 +    {
 +        this.clusteringType = clusteringType;
 +        this.fullType = fullType;
 +
 +        this.clusteringSize = clusteringType.size();
 +        this.fullSize = fullType.size();
 +    }
 +
 +    public int clusteringPrefixSize()
 +    {
 +        return clusteringSize;
 +    }
 +
 +    public boolean isCompound()
 +    {
 +        return true;
 +    }
 +
 +    public int size()
 +    {
 +        return fullSize;
 +    }
 +
 +    public AbstractType<?> subtype(int i)
 +    {
 +        return fullType.subtype(i);
 +    }
 +
 +    public CBuilder prefixBuilder()
 +    {
 +        return clusteringType.builder();
 +    }
 +
 +    public CBuilder builder()
 +    {
 +        return new CompoundCType.CompoundCBuilder(this);
 +    }
 +
 +    @Override
 +    public Composite fromByteBuffer(ByteBuffer bytes)
 +    {
 +        if (!bytes.hasRemaining())
 +            return Composites.EMPTY;
 +
 +        ByteBuffer[] elements = new ByteBuffer[fullSize];
 +        int idx = bytes.position(), i = 0;
 +        byte eoc = 0;
++
++        boolean isStatic = false;
++        if (CompositeType.isStaticName(bytes))
++        {
++            isStatic = true;
++            idx += 2;
++        }
++
 +        while (idx < bytes.limit())
 +        {
 +            checkRemaining(bytes, idx, 2);
 +            int length = bytes.getShort(idx) & 0xFFFF;
 +            idx += 2;
 +
 +            checkRemaining(bytes, idx, length + 1);
 +            elements[i++] = sliceBytes(bytes, idx, length);
 +            idx += length;
 +            eoc = bytes.get(idx++);
 +        }
 +
-         return makeWith(elements, i, Composite.EOC.from(eoc));
++        return makeWith(elements, i, Composite.EOC.from(eoc), isStatic);
 +    }
 +
 +    public AbstractType<?> asAbstractType()
 +    {
 +        return CompositeType.getInstance(fullType.types);
 +    }
 +
 +    public Deserializer newDeserializer(DataInput in)
 +    {
 +        return new CompositeDeserializer(this, in);
 +    }
 +
 +    protected CellName makeCellName(ByteBuffer[] components)
 +    {
-         return (CellName)makeWith(components, components.length, Composite.EOC.NONE);
++        return (CellName)makeWith(components, components.length, Composite.EOC.NONE, false);
 +    }
 +
-     protected abstract Composite makeWith(ByteBuffer[] components, int size, Composite.EOC eoc);
-     protected abstract Composite copyAndMakeWith(ByteBuffer[] components, int size, Composite.EOC eoc);
++    protected abstract Composite makeWith(ByteBuffer[] components, int size, Composite.EOC eoc, boolean isStatic);
++    protected abstract Composite copyAndMakeWith(ByteBuffer[] components, int size, Composite.EOC eoc, boolean isStatic);
 +
 +    private static class CompositeDeserializer implements CellNameType.Deserializer
 +    {
 +        private static byte[] EMPTY = new byte[0];
 +
 +        private final AbstractCompoundCellNameType type;
 +        private final DataInput in;
 +
 +        private byte[] nextFull;
 +        private int nextIdx;
 +
 +        private final ByteBuffer[] nextComponents;
 +        private int nextSize;
 +        private Composite.EOC nextEOC;
++        private boolean nextIsStatic;
 +
 +        public CompositeDeserializer(AbstractCompoundCellNameType type, DataInput in)
 +        {
 +            this.type = type;
 +            this.in = in;
 +            this.nextComponents = new ByteBuffer[type.size()];
 +        }
 +
 +        public boolean hasNext() throws IOException
 +        {
 +            if (nextFull == null)
 +                maybeReadNext();
 +            return nextFull != EMPTY;
 +        }
 +
 +        public boolean hasUnprocessed() throws IOException
 +        {
 +            return nextFull != null;
 +        }
 +
 +        public int compareNextTo(Composite composite) throws IOException
 +        {
 +            maybeReadNext();
 +
 +            if (composite.isEmpty())
 +                return nextFull == EMPTY ? 0 : 1;
 +
 +            if (nextFull == EMPTY)
 +                return -1;
 +
++            if (nextIsStatic != composite.isStatic())
++                return nextIsStatic ? -1 : 1;
++
 +            ByteBuffer previous = null;
 +            for (int i = 0; i < composite.size(); i++)
 +            {
 +                if (!hasComponent(i))
 +                    return nextEOC == Composite.EOC.END ? 1 : -1;
 +
 +                AbstractType<?> comparator = type.subtype(i);
 +                ByteBuffer value1 = nextComponents[i];
 +                ByteBuffer value2 = composite.get(i);
 +
 +                int cmp = comparator.compareCollectionMembers(value1, value2, previous);
 +                if (cmp != 0)
 +                    return cmp;
 +
 +                previous = value1;
 +            }
 +
 +            // If we have more component than composite
 +            if (!allComponentsDeserialized() || composite.size() < nextSize)
 +                return composite.eoc() == Composite.EOC.END ? -1 : 1;
 +
 +            // same size, check eoc
 +            if (nextEOC != composite.eoc())
 +            {
 +                switch (nextEOC)
 +                {
 +                    case START: return -1;
 +                    case END:   return 1;
 +                    case NONE:  return composite.eoc() == Composite.EOC.START ? 1 : -1;
 +                }
 +            }
 +
 +            return 0;
 +        }
 +
 +        private boolean hasComponent(int i)
 +        {
 +            while (i >= nextSize && deserializeOne())
 +                continue;
 +
 +            return i < nextSize;
 +        }
 +
++        private int readShort()
++        {
++            return ((nextFull[nextIdx++] & 0xFF) << 8) | (nextFull[nextIdx++] & 0xFF);
++        }
++
 +        private boolean deserializeOne()
 +        {
 +            if (allComponentsDeserialized())
 +                return false;
 +
-             int length = ((nextFull[nextIdx++] & 0xFF) << 8) | (nextFull[nextIdx++] & 0xFF);
++            nextIsStatic = false;
++
++            int length = readShort();
++            if (length == CompositeType.STATIC_MARKER)
++            {
++                nextIsStatic = true;
++                length = readShort();
++            }
++
 +            ByteBuffer component = ByteBuffer.wrap(nextFull, nextIdx, length);
 +            nextIdx += length;
 +            nextComponents[nextSize++] = component;
 +            nextEOC = Composite.EOC.from(nextFull[nextIdx++]);
 +            return true;
 +        }
 +
 +        private void deserializeAll()
 +        {
 +            while (deserializeOne())
 +                continue;
 +        }
 +
 +        private boolean allComponentsDeserialized()
 +        {
 +            return nextIdx >= nextFull.length;
 +        }
 +
 +        private void maybeReadNext() throws IOException
 +        {
 +            if (nextFull != null)
 +                return;
 +
 +            nextIdx = 0;
 +            nextSize = 0;
 +
 +            int length = in.readShort() & 0xFFFF;
 +            // Note that empty is ok because it marks the end of row
 +            if (length == 0)
 +            {
 +                nextFull = EMPTY;
 +                return;
 +            }
 +
 +            nextFull = new byte[length];
 +            in.readFully(nextFull);
 +        }
 +
 +        public Composite readNext() throws IOException
 +        {
 +            maybeReadNext();
 +            if (nextFull == EMPTY)
 +                return Composites.EMPTY;
 +
 +            deserializeAll();
-             Composite c = type.copyAndMakeWith(nextComponents, nextSize, nextEOC);
++            Composite c = type.copyAndMakeWith(nextComponents, nextSize, nextEOC, nextIsStatic);
 +            nextFull = null;
 +            return c;
 +        }
 +
 +        public void skipNext() throws IOException
 +        {
 +            maybeReadNext();
 +            nextFull = null;
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/BoundedComposite.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/BoundedComposite.java
index 86cdc4e,0000000..d988add
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/BoundedComposite.java
+++ b/src/java/org/apache/cassandra/db/composites/BoundedComposite.java
@@@ -1,102 -1,0 +1,107 @@@
 +/*
 + * 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.composites;
 +
 +import java.nio.ByteBuffer;
 +
 +import org.apache.cassandra.utils.memory.AbstractAllocator;
 +import org.apache.cassandra.utils.ObjectSizes;
 +import org.apache.cassandra.utils.memory.PoolAllocator;
 +
 +public class BoundedComposite extends AbstractComposite
 +{
 +    private static final long EMPTY_SIZE = ObjectSizes.measure(new BoundedComposite(null, false));
 +
 +    private final Composite wrapped;
 +    private final boolean isStart;
 +
 +    private BoundedComposite(Composite wrapped, boolean isStart)
 +    {
 +        this.wrapped = wrapped;
 +        this.isStart = isStart;
 +    }
 +
 +    static Composite startOf(Composite c)
 +    {
 +        return new BoundedComposite(c, true);
 +    }
 +
 +    static Composite endOf(Composite c)
 +    {
 +        return new BoundedComposite(c, false);
 +    }
 +
 +    public int size()
 +    {
 +        return wrapped.size();
 +    }
 +
++    public boolean isStatic()
++    {
++        return wrapped.isStatic();
++    }
++
 +    public ByteBuffer get(int i)
 +    {
 +        return wrapped.get(i);
 +    }
 +
 +    @Override
 +    public EOC eoc()
 +    {
 +        return isStart ? EOC.START : EOC.END;
 +    }
 +
 +    @Override
 +    public Composite withEOC(EOC eoc)
 +    {
 +        switch (eoc)
 +        {
 +            case START:
 +                return isStart ? this : startOf(wrapped);
 +            case END:
 +                return isStart ? endOf(wrapped) : this;
 +            default:
 +                return wrapped;
 +        }
 +    }
 +
 +    @Override
 +    public ByteBuffer toByteBuffer()
 +    {
 +        ByteBuffer bb = wrapped.toByteBuffer();
 +        bb.put(bb.remaining() - 1, (byte)(isStart ? -1 : 1));
 +        return bb;
 +    }
 +
 +    public long unsharedHeapSize()
 +    {
 +        return EMPTY_SIZE + wrapped.unsharedHeapSize();
 +    }
 +
 +    public Composite copy(AbstractAllocator allocator)
 +    {
 +        return new BoundedComposite(wrapped.copy(allocator), isStart);
 +    }
 +
 +    @Override
 +    public void free(PoolAllocator<?> allocator)
 +    {
 +        wrapped.free(allocator);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/CellNameType.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/CellNameType.java
index c4ddb60,0000000..7128c91
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/CellNameType.java
+++ b/src/java/org/apache/cassandra/db/composites/CellNameType.java
@@@ -1,202 -1,0 +1,213 @@@
 +/*
 + * 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.composites;
 +
 +import java.io.DataInput;
 +import java.io.IOException;
 +import java.nio.ByteBuffer;
 +import java.util.Comparator;
 +
++import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.CQL3Row;
 +import org.apache.cassandra.cql3.ColumnIdentifier;
 +import org.apache.cassandra.db.Cell;
 +import org.apache.cassandra.db.ColumnSerializer;
 +import org.apache.cassandra.db.OnDiskAtom;
 +import org.apache.cassandra.db.filter.IDiskAtomFilter;
 +import org.apache.cassandra.db.filter.NamesQueryFilter;
 +import org.apache.cassandra.db.marshal.AbstractType;
 +import org.apache.cassandra.db.marshal.CollectionType;
 +import org.apache.cassandra.db.marshal.ColumnToCollectionType;
 +import org.apache.cassandra.io.ISerializer;
 +import org.apache.cassandra.io.IVersionedSerializer;
 +
 +/**
 + * The type of CellNames.
 + *
 + * In the same way that a CellName is a Composite, a CellNameType is a CType, but
 + * with a number of method specific to cell names.
 + *
 + * On top of the dichotomy simple/truly-composite of composites, cell names comes
 + * in 2 variants: "dense" and "sparse". The sparse ones are CellName where one of
 + * the component (the last or second-to-last for collections) is used to store the
 + * CQL3 column name. Dense are those for which it's not the case.
 + *
 + * In other words, we have 4 types of CellName/CellNameType which correspond to the
 + * 4 type of table layout that we need to distinguish:
 + *   1. Simple (non-truly-composite) dense: this is the dynamic thrift CFs whose
 + *      comparator is not composite.
 + *   2. Composite dense: this is the dynamic thrift CFs with a CompositeType comparator.
 + *   3. Simple (non-truly-composite) sparse: this is the thrift static CFs (that
 + *      don't have a composite comparator).
 + *   4. Composite sparse: this is the CQL3 layout (note that this is the only one that
 + *      support collections).
 + */
 +public interface CellNameType extends CType
 +{
 +    /**
 +     * Whether or not the cell names for this type are dense.
 +     */
 +    public boolean isDense();
 +
 +    /**
 +     * The number of clustering columns for the table this is the type of.
 +     */
 +    public int clusteringPrefixSize();
 +
 +    /**
 +     * A builder for the clustering prefix.
 +     */
 +    public CBuilder prefixBuilder();
 +
 +    /**
++     * The prefix to use for static columns.
++     *
++     * Note that create() methods below for creating CellName automatically handle static columns already
++     * for convenience, and so there is not need to pass this prefix for them. There is few other cases
++     * where we need the prefix directly however.
++     */
++    public Composite staticPrefix();
++
++    /**
 +     * Whether or not there is some collections defined in this type.
 +     */
 +    public boolean hasCollections();
 +
 +    /**
 +     * Whether or not this type layout support collections.
 +     */
 +    public boolean supportCollections();
 +
 +    /**
 +     * The type of the collections (or null if the type has not collections).
 +     */
 +    public ColumnToCollectionType collectionType();
 +
 +    /**
 +     * Return the new type obtained by adding the new collection type for the provided column name
 +     * to this type.
 +     */
 +    public CellNameType addCollection(ColumnIdentifier columnName, CollectionType newCollection);
 +
 +    /**
 +     * Returns a new CellNameType that is equivalent to this one but with one
 +     * of the subtype replaced by the provided new type.
 +     */
 +    @Override
 +    public CellNameType setSubtype(int position, AbstractType<?> newType);
 +
 +    /**
 +     * Creates a row marker for the CQL3 having the provided clustering prefix.
 +     *
 +     * Note that this is only valid for CQL3 tables (isCompound() and !isDense()) and should
 +     * only be called for them.
 +     */
 +    public CellName rowMarker(Composite prefix);
 +
 +    /**
-      * Creates a new CellName given a clustering prefix and a CQL3 columnName.
++     * Creates a new CellName given a clustering prefix and a CQL3 column.
 +     *
-      * Note that for dense types, the columnName can be null.
++     * Note that for dense types, the column can be null as a shortcut for designing the only
++     * COMPACT_VALUE column of the table.
 +     */
-     public CellName create(Composite prefix, ColumnIdentifier columnName);
++    public CellName create(Composite prefix, ColumnDefinition column);
 +
 +    /**
-      * Creates a new collection CellName given a clustering prefix, a CQL3 columnName and the collection element.
++     * Creates a new collection CellName given a clustering prefix, a CQL3 column and the collection element.
 +     */
-     public CellName create(Composite prefix, ColumnIdentifier columnName, ByteBuffer collectionElement);
++    public CellName create(Composite prefix, ColumnDefinition column, ByteBuffer collectionElement);
 +
 +    /**
 +     * Convenience method to create cell names given its components.
 +     *
 +     * This is equivalent to CType#make() but return a full cell name (and thus
 +     * require all the components of the name).
 +     */
 +    public CellName makeCellName(Object... components);
 +
 +    /**
 +     * Deserialize a Composite from a ByteBuffer.
 +     *
 +     * This is equilvalent to CType#fromByteBuffer but assumes the buffer is a full cell
 +     * name. This is meant for thrift/cql2 to convert the fully serialized buffer we
 +     * get from the clients.
 +     */
 +    public CellName cellFromByteBuffer(ByteBuffer bb);
 +
 +    /**
 +     * Creates a new CQL3Row builder for this type. See CQL3Row for details.
 +     */
 +    public CQL3Row.Builder CQL3RowBuilder(long now);
 +
 +    // The two following methods are used to pass the declared regular column names (in CFMetaData)
 +    // to the CellNameType. This is only used for optimization sake, see SparseCellNameType.
 +    public void addCQL3Column(ColumnIdentifier id);
 +    public void removeCQL3Column(ColumnIdentifier id);
 +
 +    /**
 +     * Creates a new Deserializer. This is used by AtomDeserializer to do incremental and on-demand
 +     * deserialization of the on disk atoms. See AtomDeserializer for details.
 +     */
 +    public Deserializer newDeserializer(DataInput in);
 +
 +    /*
 +     * Same as in CType, follows a number of per-CellNameType instances for the Comparator and Serializer used
 +     * throughout the code (those that require full CellName versus just Composite).
 +     */
 +
 +    // Ultimately, those might be split into an IVersionedSerializer and an ISSTableSerializer
 +    public ISerializer<CellName> cellSerializer();
 +
 +    public Comparator<Cell> columnComparator();
 +    public Comparator<Cell> columnReverseComparator();
 +    public Comparator<OnDiskAtom> onDiskAtomComparator();
 +
 +    public ColumnSerializer columnSerializer();
 +    public OnDiskAtom.Serializer onDiskAtomSerializer();
 +    public IVersionedSerializer<NamesQueryFilter> namesQueryFilterSerializer();
 +    public IVersionedSerializer<IDiskAtomFilter> diskAtomFilterSerializer();
 +
 +    public interface Deserializer
 +    {
 +        /**
 +         * Whether this deserializer is done or not, i.e. whether we're reached the end of row marker.
 +         */
 +        public boolean hasNext() throws IOException;
 +
 +        /**
 +         * Whether or not some name has been read but not consumed by readNext.
 +         */
 +        public boolean hasUnprocessed() throws IOException;
 +
 +        /**
 +         * Comparare the next name to read to the provided Composite.
 +         * This does not consume the next name.
 +         */
 +        public int compareNextTo(Composite composite) throws IOException;
 +
 +        /**
 +         * Actually consume the next name and return it.
 +         */
 +        public Composite readNext() throws IOException;
 +
 +        /**
 +         * Skip the next name (consuming it).
 +         */
 +        public void skipNext() throws IOException;
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/Composite.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/Composite.java
index c82d170,0000000..99a0d46
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/Composite.java
+++ b/src/java/org/apache/cassandra/db/composites/Composite.java
@@@ -1,78 -1,0 +1,80 @@@
 +/*
 + * 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.composites;
 +
 +import java.nio.ByteBuffer;
 +
 +import org.apache.cassandra.cache.IMeasurableMemory;
 +import org.apache.cassandra.db.filter.ColumnSlice;
 +import org.apache.cassandra.utils.memory.AbstractAllocator;
 +import org.apache.cassandra.utils.memory.PoolAllocator;
 +
 +/**
 + * A composite value.
 + *
 + * This can be though as a list of ByteBuffer, except that this also include an
 + * 'end-of-component' flag, that allow precise selection of composite ranges.
 + *
 + * We also make a difference between "true" composites and the "simple" ones. The
 + * non-truly composite will have a size() == 1 but differs from true composites with
 + * size() == 1 in the way they are stored. Most code shouldn't have to care about the
 + * difference.
 + */
 +public interface Composite extends IMeasurableMemory
 +{
 +    public enum EOC
 +    {
 +        START, NONE, END;
 +
 +        public static EOC from(int eoc)
 +        {
 +            return eoc == 0 ? NONE : (eoc < 0 ? START : END);
 +        }
 +
 +        public byte toByte()
 +        {
 +            switch (this)
 +            {
 +                case START: return (byte)-1;
 +                case NONE:  return (byte) 0;
 +                case END:   return (byte) 1;
 +                default: throw new AssertionError();
 +            }
 +        }
 +    }
 +
 +    public int size();
 +    public boolean isEmpty();
 +    public ByteBuffer get(int i);
 +
 +    public EOC eoc();
 +    public Composite withEOC(EOC eoc);
 +    public Composite start();
 +    public Composite end();
 +    public ColumnSlice slice();
 +
++    public boolean isStatic();
++
 +    public boolean isPrefixOf(Composite other);
 +
 +    public ByteBuffer toByteBuffer();
 +
 +    public int dataSize();
 +    public Composite copy(AbstractAllocator allocator);
 +    public void free(PoolAllocator<?> allocator);
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/Composites.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/Composites.java
index f1b7eab,0000000..902c82a
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/Composites.java
+++ b/src/java/org/apache/cassandra/db/composites/Composites.java
@@@ -1,117 -1,0 +1,122 @@@
 +/*
 + * 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.composites;
 +
 +import java.nio.ByteBuffer;
 +
 +import org.apache.cassandra.db.filter.ColumnSlice;
 +import org.apache.cassandra.utils.memory.AbstractAllocator;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +import org.apache.cassandra.utils.memory.PoolAllocator;
 +
 +public abstract class Composites
 +{
 +    private Composites() {}
 +
 +    public static final Composite EMPTY = new EmptyComposite();
 +
 +    static final CBuilder EMPTY_BUILDER = new CBuilder()
 +    {
 +        public int remainingCount() { return 0; }
 +
 +        public CBuilder add(ByteBuffer value) { throw new IllegalStateException(); }
 +        public CBuilder add(Object value) { throw new IllegalStateException(); }
 +
 +        public Composite build() { return EMPTY; }
 +        public Composite buildWith(ByteBuffer value) { throw new IllegalStateException(); }
 +    };
 +
 +    private static class EmptyComposite implements Composite
 +    {
 +        public boolean isEmpty()
 +        {
 +            return true;
 +        }
 +
 +        public int size()
 +        {
 +            return 0;
 +        }
 +
 +        public ByteBuffer get(int i)
 +        {
 +            throw new IndexOutOfBoundsException();
 +        }
 +
 +        public EOC eoc()
 +        {
 +            return EOC.NONE;
 +        }
 +
 +        public Composite start()
 +        {
 +            return this;
 +        }
 +
 +        public Composite end()
 +        {
 +            return this;
 +        }
 +
 +        public Composite withEOC(EOC newEoc)
 +        {
 +            return this;
 +        }
 +
 +        public ColumnSlice slice()
 +        {
 +            return ColumnSlice.ALL_COLUMNS;
 +        }
 +
 +        public ByteBuffer toByteBuffer()
 +        {
 +            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
 +        }
 +
++        public boolean isStatic()
++        {
++            return false;
++        }
++
 +        public int dataSize()
 +        {
 +            return 0;
 +        }
 +
 +        public long unsharedHeapSize()
 +        {
 +            return 0;
 +        }
 +
 +        public boolean isPrefixOf(Composite c)
 +        {
 +            return true;
 +        }
 +
 +        public Composite copy(AbstractAllocator allocator)
 +        {
 +            return this;
 +        }
 +
 +        @Override
 +        public void free(PoolAllocator<?> allocator)
 +        {
 +        }
 +
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/CompoundCType.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/CompoundCType.java
index bbad550,0000000..d638f16
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/CompoundCType.java
+++ b/src/java/org/apache/cassandra/db/composites/CompoundCType.java
@@@ -1,158 -1,0 +1,166 @@@
 +/*
 + * 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.composites;
 +
 +import java.nio.ByteBuffer;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.List;
 +
 +import org.apache.cassandra.db.marshal.AbstractType;
 +import org.apache.cassandra.db.marshal.CompositeType;
 +
 +/**
 + * A truly-composite CType.
 + */
 +public class CompoundCType extends AbstractCType
 +{
 +    final List<AbstractType<?>> types;
 +
 +    // It's up to the caller to pass a list that is effectively immutable
 +    public CompoundCType(List<AbstractType<?>> types)
 +    {
 +        this.types = types;
 +    }
 +
 +    public boolean isCompound()
 +    {
 +        return true;
 +    }
 +
 +    public int size()
 +    {
 +        return types.size();
 +    }
 +
 +    public AbstractType<?> subtype(int i)
 +    {
 +        return types.get(i);
 +    }
 +
 +    public Composite fromByteBuffer(ByteBuffer bytes)
 +    {
 +        if (!bytes.hasRemaining())
 +            return Composites.EMPTY;
 +
 +        ByteBuffer[] elements = new ByteBuffer[size()];
 +        int idx = bytes.position(), i = 0;
 +        byte eoc = 0;
++
++        boolean isStatic = false;
++        if (CompositeType.isStaticName(bytes))
++        {
++            isStatic = true;
++            idx += 2;
++        }
++
 +        while (idx < bytes.limit())
 +        {
 +            checkRemaining(bytes, idx, 2);
 +            int length = bytes.getShort(idx) & 0xFFFF;
 +            idx += 2;
 +
 +            checkRemaining(bytes, idx, length + 1);
 +            elements[i++] = sliceBytes(bytes, idx, length);
 +            idx += length;
 +            eoc = bytes.get(idx++);
 +        }
-         return new CompoundComposite(elements, i).withEOC(Composite.EOC.from(eoc));
++        return new CompoundComposite(elements, i, isStatic).withEOC(Composite.EOC.from(eoc));
 +    }
 +
 +    public CBuilder builder()
 +    {
 +        return new CompoundCBuilder(this);
 +    }
 +
 +    public CompoundCType setSubtype(int position, AbstractType<?> newType)
 +    {
 +        List<AbstractType<?>> newTypes = new ArrayList<AbstractType<?>>(types);
 +        newTypes.set(position, newType);
 +        return new CompoundCType(newTypes);
 +    }
 +
 +    public AbstractType<?> asAbstractType()
 +    {
 +        return CompositeType.getInstance(types);
 +    }
 +
 +    public static class CompoundCBuilder implements CBuilder
 +    {
 +        private final CType type;
 +        private final ByteBuffer[] values;
 +        private int size;
 +        private boolean built;
 +
 +        public CompoundCBuilder(CType type)
 +        {
 +            this.type = type;
 +            this.values = new ByteBuffer[type.size()];
 +        }
 +
 +        public int remainingCount()
 +        {
 +            return values.length - size;
 +        }
 +
 +        public CBuilder add(ByteBuffer value)
 +        {
 +            if (isDone())
 +                throw new IllegalStateException();
 +            values[size++] = value;
 +            return this;
 +        }
 +
 +        public CBuilder add(Object value)
 +        {
 +            return add(((AbstractType)type.subtype(size)).decompose(value));
 +        }
 +
 +        private boolean isDone()
 +        {
 +            return remainingCount() == 0 || built;
 +        }
 +
 +        public Composite build()
 +        {
 +            if (size == 0)
 +                return Composites.EMPTY;
 +
 +            // We don't allow to add more element to a builder that has been built so
 +            // that we don't have to copy values.
 +            built = true;
 +
 +            // If the builder is full and we're building a dense cell name, then we can
 +            // directly allocate the CellName object as it's complete.
 +            if (size == values.length && type instanceof CellNameType && ((CellNameType)type).isDense())
 +                return new CompoundDenseCellName(values);
-             return new CompoundComposite(values, size);
++            return new CompoundComposite(values, size, false);
 +        }
 +
 +        public Composite buildWith(ByteBuffer value)
 +        {
 +            ByteBuffer[] newValues = Arrays.copyOf(values, values.length);
 +            newValues[size] = value;
 +            // Same as above
 +            if (size+1 == newValues.length && type instanceof CellNameType && ((CellNameType)type).isDense())
 +                return new CompoundDenseCellName(newValues);
 +
-             return new CompoundComposite(newValues, size+1);
++            return new CompoundComposite(newValues, size+1, false);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/CompoundComposite.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/CompoundComposite.java
index 54ec759,0000000..9475cc0
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/CompoundComposite.java
+++ b/src/java/org/apache/cassandra/db/composites/CompoundComposite.java
@@@ -1,84 -1,0 +1,92 @@@
 +/*
 + * 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.composites;
 +
 +import java.nio.ByteBuffer;
 +
- import org.apache.cassandra.utils.memory.AbstractAllocator;
 +import org.apache.cassandra.utils.ObjectSizes;
++import org.apache.cassandra.utils.memory.AbstractAllocator;
 +import org.apache.cassandra.utils.memory.PoolAllocator;
 +
 +/**
 + * A "truly-composite" Composite.
 + */
 +public class CompoundComposite extends AbstractComposite
 +{
-     private static final long EMPTY_SIZE = ObjectSizes.measure(new CompoundComposite(null, 0));
++    private static final long EMPTY_SIZE = ObjectSizes.measure(new CompoundComposite(null, 0, false));
 +
 +    // We could use a List, but we'll create such object *a lot* and using a array+size is not
 +    // all that harder, so we save the List object allocation.
 +    final ByteBuffer[] elements;
 +    final int size;
++    final boolean isStatic;
 +
-     CompoundComposite(ByteBuffer[] elements, int size)
++    CompoundComposite(ByteBuffer[] elements, int size, boolean isStatic)
 +    {
 +        this.elements = elements;
 +        this.size = size;
++        this.isStatic = isStatic;
 +    }
 +
 +    public int size()
 +    {
 +        return size;
 +    }
 +
 +    public ByteBuffer get(int i)
 +    {
 +        return elements[i];
 +    }
 +
++    @Override
++    public boolean isStatic()
++    {
++        return isStatic;
++    }
++
 +    protected ByteBuffer[] elementsCopy(AbstractAllocator allocator)
 +    {
 +        ByteBuffer[] elementsCopy = new ByteBuffer[size];
 +        for (int i = 0; i < size; i++)
 +            elementsCopy[i] = allocator.clone(elements[i]);
 +        return elementsCopy;
 +    }
 +
 +    public long unsharedHeapSize()
 +    {
 +        return EMPTY_SIZE + ObjectSizes.sizeOnHeapOf(elements);
 +    }
 +
 +    public long excessHeapSizeExcludingData()
 +    {
 +        return EMPTY_SIZE + ObjectSizes.sizeOnHeapExcludingData(elements);
 +    }
 +
 +    public Composite copy(AbstractAllocator allocator)
 +    {
-         return new CompoundComposite(elementsCopy(allocator), size);
++        return new CompoundComposite(elementsCopy(allocator), size, isStatic);
 +    }
 +
 +    @Override
 +    public void free(PoolAllocator<?> allocator)
 +    {
 +        for (ByteBuffer element : elements)
 +            allocator.free(element);
 +    }
 +
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/CompoundDenseCellName.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/CompoundDenseCellName.java
index 26df34b,0000000..e16e390
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/CompoundDenseCellName.java
+++ b/src/java/org/apache/cassandra/db/composites/CompoundDenseCellName.java
@@@ -1,80 -1,0 +1,80 @@@
 +/*
 + * 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.composites;
 +
 +import java.nio.ByteBuffer;
 +
 +import org.apache.cassandra.cql3.ColumnIdentifier;
 +import org.apache.cassandra.utils.memory.AbstractAllocator;
 +import org.apache.cassandra.utils.ObjectSizes;
 +
 +public class CompoundDenseCellName extends CompoundComposite implements CellName
 +{
 +
 +    private static final long HEAP_SIZE = ObjectSizes.measure(new CompoundDenseCellName(new ByteBuffer[0]));
 +
 +    // Not meant to be used directly, you should use the CellNameType method instead
 +    CompoundDenseCellName(ByteBuffer[] elements)
 +    {
-         super(elements, elements.length);
++        super(elements, elements.length, false);
 +    }
 +
 +    public int clusteringSize()
 +    {
 +        return size;
 +    }
 +
 +    public ColumnIdentifier cql3ColumnName()
 +    {
 +        return null;
 +    }
 +
 +    public ByteBuffer collectionElement()
 +    {
 +        return null;
 +    }
 +
 +    public boolean isCollectionCell()
 +    {
 +        return false;
 +    }
 +
 +    public boolean isSameCQL3RowAs(CellName other)
 +    {
 +        // Dense cell imply one cell by CQL row so no other cell will be the same row.
 +        return equals(other);
 +    }
 +
 +    @Override
 +    public long unsharedHeapSize()
 +    {
 +        return HEAP_SIZE + ObjectSizes.sizeOnHeapOf(elements);
 +    }
 +
 +    @Override
 +    public long excessHeapSizeExcludingData()
 +    {
 +        return HEAP_SIZE + ObjectSizes.sizeOnHeapExcludingData(elements);
 +    }
 +
 +    public CellName copy(AbstractAllocator allocator)
 +    {
 +        return new CompoundDenseCellName(elementsCopy(allocator));
 +    }
 +
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/CompoundDenseCellNameType.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/CompoundDenseCellNameType.java
index bb7a06e,0000000..06b25b0
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/CompoundDenseCellNameType.java
+++ b/src/java/org/apache/cassandra/db/composites/CompoundDenseCellNameType.java
@@@ -1,86 -1,0 +1,88 @@@
 +/*
 + * 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.composites;
 +
 +import java.nio.ByteBuffer;
 +import java.util.Arrays;
 +import java.util.List;
 +
++import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.CQL3Row;
 +import org.apache.cassandra.cql3.ColumnIdentifier;
 +import org.apache.cassandra.db.marshal.AbstractType;
 +
 +public class CompoundDenseCellNameType extends AbstractCompoundCellNameType
 +{
 +    public CompoundDenseCellNameType(List<AbstractType<?>> types)
 +    {
 +        this(new CompoundCType(types));
 +    }
 +
 +    private CompoundDenseCellNameType(CompoundCType type)
 +    {
 +        super(type, type);
 +    }
 +
 +    public CellNameType setSubtype(int position, AbstractType<?> newType)
 +    {
 +        if (position != 0)
 +            throw new IllegalArgumentException();
 +        return new SimpleDenseCellNameType(newType);
 +    }
 +
 +    public boolean isDense()
 +    {
 +        return true;
 +    }
 +
-     public CellName create(Composite prefix, ColumnIdentifier columnName)
++    public CellName create(Composite prefix, ColumnDefinition column)
 +    {
-         // We ignore the columnName because it's just the COMPACT_VALUE name which is not store in the cell name
++        // We ignore the column because it's just the COMPACT_VALUE name which is not store in the cell name (and it can be null anyway)
 +        assert prefix.size() == fullSize;
 +        if (prefix instanceof CellName)
 +            return (CellName)prefix;
 +
 +        assert prefix instanceof CompoundComposite;
 +        CompoundComposite lc = (CompoundComposite)prefix;
 +        assert lc.elements.length == lc.size;
 +        return new CompoundDenseCellName(lc.elements);
 +    }
 +
-     protected Composite makeWith(ByteBuffer[] components, int size, Composite.EOC eoc)
++    protected Composite makeWith(ByteBuffer[] components, int size, Composite.EOC eoc, boolean isStatic)
 +    {
++        assert !isStatic;
 +        if (size < fullSize || eoc != Composite.EOC.NONE)
-             return new CompoundComposite(components, size).withEOC(eoc);
++            return new CompoundComposite(components, size, false).withEOC(eoc);
 +
 +        assert components.length == size;
 +        return new CompoundDenseCellName(components);
 +    }
 +
-     protected Composite copyAndMakeWith(ByteBuffer[] components, int size, Composite.EOC eoc)
++    protected Composite copyAndMakeWith(ByteBuffer[] components, int size, Composite.EOC eoc, boolean isStatic)
 +    {
-         return makeWith(Arrays.copyOfRange(components, 0, size), size, eoc);
++        return makeWith(Arrays.copyOfRange(components, 0, size), size, eoc, isStatic);
 +    }
 +
 +    public void addCQL3Column(ColumnIdentifier id) {}
 +    public void removeCQL3Column(ColumnIdentifier id) {}
 +
 +    public CQL3Row.Builder CQL3RowBuilder(long now)
 +    {
 +        return makeDenseCQL3RowBuilder(now);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/CompoundSparseCellName.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/CompoundSparseCellName.java
index dc525d6,0000000..0106309
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/CompoundSparseCellName.java
+++ b/src/java/org/apache/cassandra/db/composites/CompoundSparseCellName.java
@@@ -1,175 -1,0 +1,175 @@@
 +/*
 + * 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.composites;
 +
 +import java.nio.ByteBuffer;
 +
 +import org.apache.cassandra.cql3.ColumnIdentifier;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +import org.apache.cassandra.utils.memory.AbstractAllocator;
 +import org.apache.cassandra.utils.ObjectSizes;
 +import org.apache.cassandra.utils.memory.PoolAllocator;
 +
 +public class CompoundSparseCellName extends CompoundComposite implements CellName
 +{
 +    private static final ByteBuffer[] EMPTY_PREFIX = new ByteBuffer[0];
 +
-     private static final long HEAP_SIZE = ObjectSizes.measure(new CompoundSparseCellName(null));
++    private static final long HEAP_SIZE = ObjectSizes.measure(new CompoundSparseCellName(null, false));
 +
 +    protected final ColumnIdentifier columnName;
 +
 +    // Not meant to be used directly, you should use the CellNameType method instead
-     CompoundSparseCellName(ColumnIdentifier columnName)
++    CompoundSparseCellName(ColumnIdentifier columnName, boolean isStatic)
 +    {
-         this(EMPTY_PREFIX, columnName);
++        this(EMPTY_PREFIX, columnName, isStatic);
 +    }
 +
-     CompoundSparseCellName(ByteBuffer[] elements, ColumnIdentifier columnName)
++    CompoundSparseCellName(ByteBuffer[] elements, ColumnIdentifier columnName, boolean isStatic)
 +    {
-         this(elements, elements.length, columnName);
++        this(elements, elements.length, columnName, isStatic);
 +    }
 +
-     CompoundSparseCellName(ByteBuffer[] elements, int size, ColumnIdentifier columnName)
++    CompoundSparseCellName(ByteBuffer[] elements, int size, ColumnIdentifier columnName, boolean isStatic)
 +    {
-         super(elements, size);
++        super(elements, size, isStatic);
 +        this.columnName = columnName;
 +    }
 +
 +    public int size()
 +    {
 +        return size + 1;
 +    }
 +
 +    public ByteBuffer get(int i)
 +    {
 +        return i == size ? columnName.bytes : elements[i];
 +    }
 +
 +    public int clusteringSize()
 +    {
 +        return size;
 +    }
 +
 +    public ColumnIdentifier cql3ColumnName()
 +    {
 +        return columnName;
 +    }
 +
 +    public ByteBuffer collectionElement()
 +    {
 +        return null;
 +    }
 +
 +    public boolean isCollectionCell()
 +    {
 +        return false;
 +    }
 +
 +    public boolean isSameCQL3RowAs(CellName other)
 +    {
-         if (clusteringSize() != other.clusteringSize())
++        if (clusteringSize() != other.clusteringSize() || other.isStatic() != isStatic())
 +            return false;
 +
 +        for (int i = 0; i < clusteringSize(); i++)
 +        {
 +            if (!elements[i].equals(other.get(i)))
 +                return false;
 +        }
 +        return true;
 +    }
 +
 +    public CellName copy(AbstractAllocator allocator)
 +    {
 +        if (elements.length == 0)
 +            return this;
 +
 +        // We don't copy columnName because it's interned in SparseCellNameType
-         return new CompoundSparseCellName(elementsCopy(allocator), columnName);
++        return new CompoundSparseCellName(elementsCopy(allocator), columnName, isStatic());
 +    }
 +
 +    public static class WithCollection extends CompoundSparseCellName
 +    {
-         private static final long HEAP_SIZE = ObjectSizes.measure(new WithCollection(null, ByteBufferUtil.EMPTY_BYTE_BUFFER));
++        private static final long HEAP_SIZE = ObjectSizes.measure(new WithCollection(null, ByteBufferUtil.EMPTY_BYTE_BUFFER, false));
 +
 +        private final ByteBuffer collectionElement;
 +
-         WithCollection(ColumnIdentifier columnName, ByteBuffer collectionElement)
++        WithCollection(ColumnIdentifier columnName, ByteBuffer collectionElement, boolean isStatic)
 +        {
-             this(EMPTY_PREFIX, columnName, collectionElement);
++            this(EMPTY_PREFIX, columnName, collectionElement, isStatic);
 +        }
 +
-         WithCollection(ByteBuffer[] elements, ColumnIdentifier columnName, ByteBuffer collectionElement)
++        WithCollection(ByteBuffer[] elements, ColumnIdentifier columnName, ByteBuffer collectionElement, boolean isStatic)
 +        {
-             this(elements, elements.length, columnName, collectionElement);
++            this(elements, elements.length, columnName, collectionElement, isStatic);
 +        }
 +
-         WithCollection(ByteBuffer[] elements, int size, ColumnIdentifier columnName, ByteBuffer collectionElement)
++        WithCollection(ByteBuffer[] elements, int size, ColumnIdentifier columnName, ByteBuffer collectionElement, boolean isStatic)
 +        {
-             super(elements, size, columnName);
++            super(elements, size, columnName, isStatic);
 +            this.collectionElement = collectionElement;
 +        }
 +
 +        public int size()
 +        {
 +            return size + 2;
 +        }
 +
 +        public ByteBuffer get(int i)
 +        {
 +            return i == size + 1 ? collectionElement : super.get(i);
 +        }
 +
 +        @Override
 +        public ByteBuffer collectionElement()
 +        {
 +            return collectionElement;
 +        }
 +
 +        @Override
 +        public boolean isCollectionCell()
 +        {
 +            return true;
 +        }
 +
 +        @Override
 +        public CellName copy(AbstractAllocator allocator)
 +        {
 +            // We don't copy columnName because it's interned in SparseCellNameType
-             return new CompoundSparseCellName.WithCollection(elements.length == 0 ? elements : elementsCopy(allocator), size, columnName, allocator.clone(collectionElement));
++            return new CompoundSparseCellName.WithCollection(elements.length == 0 ? elements : elementsCopy(allocator), size, columnName, allocator.clone(collectionElement), isStatic());
 +        }
 +
 +        @Override
 +        public long unsharedHeapSize()
 +        {
 +            return super.unsharedHeapSize() + ObjectSizes.sizeOnHeapOf(collectionElement);
 +        }
 +
 +        @Override
 +        public long excessHeapSizeExcludingData()
 +        {
 +            return super.excessHeapSizeExcludingData() + ObjectSizes.sizeOnHeapExcludingData(collectionElement);
 +        }
 +
 +        @Override
 +        public void free(PoolAllocator<?> allocator)
 +        {
 +            super.free(allocator);
 +            allocator.free(collectionElement);
 +        }
 +    }
 +}


[7/7] git commit: Merge branch 'cassandra-2.1' into trunk

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


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

Branch: refs/heads/trunk
Commit: 4d75ed206c4df5f9cb880b8a1133f4b5ccdd6ecf
Parents: 1debf25 63b1ef4
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Feb 20 17:51:44 2014 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Feb 20 17:51:44 2014 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 doc/cql3/CQL.textile                            |  29 +-
 .../org/apache/cassandra/config/CFMetaData.java |  30 +-
 .../cassandra/config/ColumnDefinition.java      |  16 +
 src/java/org/apache/cassandra/cql3/CQL3Row.java |   7 +-
 .../apache/cassandra/cql3/ColumnCondition.java  | 183 ++++++++++
 .../org/apache/cassandra/cql3/Constants.java    |   8 +-
 src/java/org/apache/cassandra/cql3/Cql.g        |  29 +-
 src/java/org/apache/cassandra/cql3/Lists.java   |  12 +-
 src/java/org/apache/cassandra/cql3/Maps.java    |  14 +-
 src/java/org/apache/cassandra/cql3/Sets.java    |  12 +-
 .../cql3/statements/AlterTableStatement.java    |  30 +-
 .../cql3/statements/BatchStatement.java         | 162 +++++++--
 .../cql3/statements/CQL3CasConditions.java      | 164 +++++++++
 .../cql3/statements/CreateIndexStatement.java   |   9 +
 .../cql3/statements/CreateTableStatement.java   |  41 ++-
 .../cql3/statements/DeleteStatement.java        |  23 +-
 .../cql3/statements/ModificationStatement.java  | 330 ++++++++++---------
 .../cql3/statements/SelectStatement.java        | 238 +++++++++++--
 .../cassandra/cql3/statements/Selection.java    |  29 +-
 .../cql3/statements/UpdateStatement.java        |  20 +-
 .../org/apache/cassandra/db/CFRowAdder.java     |  10 +-
 .../cassandra/db/composites/AbstractCType.java  |   7 +-
 .../db/composites/AbstractCellNameType.java     | 188 +++++++----
 .../db/composites/AbstractComposite.java        |  17 +-
 .../AbstractCompoundCellNameType.java           |  37 ++-
 .../db/composites/BoundedComposite.java         |   5 +
 .../cassandra/db/composites/CellNameType.java   |  21 +-
 .../cassandra/db/composites/Composite.java      |   2 +
 .../cassandra/db/composites/Composites.java     |   5 +
 .../cassandra/db/composites/CompoundCType.java  |  14 +-
 .../db/composites/CompoundComposite.java        |  16 +-
 .../db/composites/CompoundDenseCellName.java    |   2 +-
 .../composites/CompoundDenseCellNameType.java   |  14 +-
 .../db/composites/CompoundSparseCellName.java   |  34 +-
 .../composites/CompoundSparseCellNameType.java  |  98 ++++--
 .../db/composites/SimpleDenseCellNameType.java  |   5 +-
 .../db/composites/SimpleSparseCellNameType.java |   7 +-
 .../cassandra/db/filter/ExtendedFilter.java     |  10 +-
 .../CompositesIndexOnCollectionKey.java         |   2 +-
 .../CompositesIndexOnCollectionValue.java       |   2 +-
 .../composites/CompositesIndexOnRegular.java    |   2 +-
 .../db/index/composites/CompositesSearcher.java |  25 +-
 .../db/marshal/AbstractCompositeType.java       |  47 +--
 .../cassandra/db/marshal/CompositeType.java     |  72 +++-
 .../db/marshal/DynamicCompositeType.java        |   6 +
 .../hadoop/pig/AbstractCassandraStorage.java    |   3 +-
 .../apache/cassandra/service/CASConditions.java |   3 +-
 48 files changed, 1552 insertions(+), 489 deletions(-)
----------------------------------------------------------------------



[6/7] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1

Posted by sl...@apache.org.
Merge branch 'cassandra-2.0' into cassandra-2.1

Conflicts:
	src/java/org/apache/cassandra/config/CFMetaData.java
	src/java/org/apache/cassandra/config/ColumnDefinition.java
	src/java/org/apache/cassandra/cql3/CFDefinition.java
	src/java/org/apache/cassandra/cql3/Constants.java
	src/java/org/apache/cassandra/cql3/Cql.g
	src/java/org/apache/cassandra/cql3/Lists.java
	src/java/org/apache/cassandra/cql3/Maps.java
	src/java/org/apache/cassandra/cql3/Operation.java
	src/java/org/apache/cassandra/cql3/Sets.java
	src/java/org/apache/cassandra/cql3/functions/TokenFct.java
	src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
	src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
	src/java/org/apache/cassandra/cql3/statements/ColumnGroupMap.java
	src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
	src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
	src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
	src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
	src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
	src/java/org/apache/cassandra/cql3/statements/Selection.java
	src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
	src/java/org/apache/cassandra/db/filter/ColumnSlice.java
	src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
	src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
	src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
	src/java/org/apache/cassandra/thrift/ThriftValidation.java


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

Branch: refs/heads/trunk
Commit: 63b1ef4eebb26523d69b96cfcf20d9e5ae0b15c0
Parents: a73db3f b09d876
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Feb 20 17:51:13 2014 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Feb 20 17:51:13 2014 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 doc/cql3/CQL.textile                            |  29 +-
 .../org/apache/cassandra/config/CFMetaData.java |  30 +-
 .../cassandra/config/ColumnDefinition.java      |  16 +
 src/java/org/apache/cassandra/cql3/CQL3Row.java |   7 +-
 .../apache/cassandra/cql3/ColumnCondition.java  | 183 ++++++++++
 .../org/apache/cassandra/cql3/Constants.java    |   8 +-
 src/java/org/apache/cassandra/cql3/Cql.g        |  29 +-
 src/java/org/apache/cassandra/cql3/Lists.java   |  12 +-
 src/java/org/apache/cassandra/cql3/Maps.java    |  14 +-
 src/java/org/apache/cassandra/cql3/Sets.java    |  12 +-
 .../cql3/statements/AlterTableStatement.java    |  30 +-
 .../cql3/statements/BatchStatement.java         | 162 +++++++--
 .../cql3/statements/CQL3CasConditions.java      | 164 +++++++++
 .../cql3/statements/CreateIndexStatement.java   |   9 +
 .../cql3/statements/CreateTableStatement.java   |  41 ++-
 .../cql3/statements/DeleteStatement.java        |  23 +-
 .../cql3/statements/ModificationStatement.java  | 330 ++++++++++---------
 .../cql3/statements/SelectStatement.java        | 238 +++++++++++--
 .../cassandra/cql3/statements/Selection.java    |  29 +-
 .../cql3/statements/UpdateStatement.java        |  20 +-
 .../org/apache/cassandra/db/CFRowAdder.java     |  10 +-
 .../cassandra/db/composites/AbstractCType.java  |   7 +-
 .../db/composites/AbstractCellNameType.java     | 188 +++++++----
 .../db/composites/AbstractComposite.java        |  17 +-
 .../AbstractCompoundCellNameType.java           |  37 ++-
 .../db/composites/BoundedComposite.java         |   5 +
 .../cassandra/db/composites/CellNameType.java   |  21 +-
 .../cassandra/db/composites/Composite.java      |   2 +
 .../cassandra/db/composites/Composites.java     |   5 +
 .../cassandra/db/composites/CompoundCType.java  |  14 +-
 .../db/composites/CompoundComposite.java        |  16 +-
 .../db/composites/CompoundDenseCellName.java    |   2 +-
 .../composites/CompoundDenseCellNameType.java   |  14 +-
 .../db/composites/CompoundSparseCellName.java   |  34 +-
 .../composites/CompoundSparseCellNameType.java  |  98 ++++--
 .../db/composites/SimpleDenseCellNameType.java  |   5 +-
 .../db/composites/SimpleSparseCellNameType.java |   7 +-
 .../cassandra/db/filter/ExtendedFilter.java     |  10 +-
 .../CompositesIndexOnCollectionKey.java         |   2 +-
 .../CompositesIndexOnCollectionValue.java       |   2 +-
 .../composites/CompositesIndexOnRegular.java    |   2 +-
 .../db/index/composites/CompositesSearcher.java |  25 +-
 .../db/marshal/AbstractCompositeType.java       |  47 +--
 .../cassandra/db/marshal/CompositeType.java     |  72 +++-
 .../db/marshal/DynamicCompositeType.java        |   6 +
 .../hadoop/pig/AbstractCassandraStorage.java    |   3 +-
 .../apache/cassandra/service/CASConditions.java |   3 +-
 48 files changed, 1552 insertions(+), 489 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/doc/cql3/CQL.textile
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/config/CFMetaData.java
index ba7e6e6,a319930..25b7314
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@@ -26,12 -27,12 +26,13 @@@ import java.util.*
  
  import com.google.common.annotations.VisibleForTesting;
  import com.google.common.base.Objects;
 +import com.google.common.collect.AbstractIterator;
+ import com.google.common.collect.Iterables;
  import com.google.common.collect.MapDifference;
  import com.google.common.collect.Maps;
 +import org.apache.cassandra.db.composites.*;
  import org.apache.commons.lang3.ArrayUtils;
  import org.apache.commons.lang3.StringUtils;
 -import org.apache.commons.lang3.builder.EqualsBuilder;
  import org.apache.commons.lang3.builder.HashCodeBuilder;
  import org.apache.commons.lang3.builder.ToStringBuilder;
  import org.slf4j.Logger;
@@@ -474,10 -405,11 +475,11 @@@ public final class CFMetaDat
      public static final String DEFAULT_COLUMN_ALIAS = "column";
      public static final String DEFAULT_VALUE_ALIAS = "value";
  
 -    private volatile Map<ByteBuffer, ColumnDefinition> column_metadata = new HashMap<>();
 +    private volatile Map<ByteBuffer, ColumnDefinition> columnMetadata = 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
 -    private volatile Set<ColumnDefinition> regularColumns;
 -    private volatile Set<ColumnDefinition> staticColumns;
 +    private volatile List<ColumnDefinition> clusteringColumns;    // Of size comparator.componentsCount or comparator.componentsCount -1, null padded if necessary
 +    private volatile SortedSet<ColumnDefinition> regularColumns;  // We use a sorted set so iteration is of predictable order (for SELECT for instance)
++    private volatile SortedSet<ColumnDefinition> staticColumns;   // Same as above
      private volatile ColumnDefinition compactValueColumn;
  
      public volatile Class<? extends AbstractCompactionStrategy> compactionStrategyClass = DEFAULT_COMPACTION_STRATEGY_CLASS;
@@@ -793,42 -705,7 +795,46 @@@
  
      public Collection<ColumnDefinition> allColumns()
      {
 -        return column_metadata.values();
 +        return columnMetadata.values();
 +    }
 +
 +    // An iterator over all column definitions but that respect the order of a SELECT *.
 +    public Iterator<ColumnDefinition> allColumnsInSelectOrder()
 +    {
 +        return new AbstractIterator<ColumnDefinition>()
 +        {
 +            private final Iterator<ColumnDefinition> partitionKeyIter = partitionKeyColumns.iterator();
 +            private final Iterator<ColumnDefinition> clusteringIter = clusteringColumns.iterator();
 +            private boolean valueDone;
++            private final Iterator<ColumnDefinition> staticIter = staticColumns.iterator();
 +            private final Iterator<ColumnDefinition> regularIter = regularColumns.iterator();
 +
 +            protected ColumnDefinition computeNext()
 +            {
 +                if (partitionKeyIter.hasNext())
 +                    return partitionKeyIter.next();
 +
 +                if (clusteringIter.hasNext())
 +                    return clusteringIter.next();
 +
++                if (staticIter.hasNext())
++                    return staticIter.next();
++
 +                if (compactValueColumn != null && !valueDone)
 +                {
 +                    valueDone = true;
 +                    // If the compactValueColumn is empty, this means we have a dense table but
 +                    // with only a PK. As far as selects are concerned, we should ignore the value.
 +                    if (compactValueColumn.name.bytes.hasRemaining())
 +                        return compactValueColumn;
 +                }
 +
 +                if (regularIter.hasNext())
 +                    return regularIter.next();
 +
 +                return endOfData();
 +            }
 +        };
      }
  
      public List<ColumnDefinition> partitionKeyColumns()
@@@ -1507,9 -1340,9 +1523,9 @@@
          // Mixing counter with non counter columns is not supported (#2614)
          if (defaultValidator instanceof CounterColumnType)
          {
-             for (ColumnDefinition def : regularColumns)
+             for (ColumnDefinition def : regularAndStaticColumns())
 -                if (!(def.getValidator() instanceof CounterColumnType))
 -                    throw new ConfigurationException("Cannot add a non counter column (" + getColumnDefinitionComparator(def).getString(def.name) + ") in a counter column family");
 +                if (!(def.type instanceof CounterColumnType))
 +                    throw new ConfigurationException("Cannot add a non counter column (" + def.name + ") in a counter column family");
          }
          else
          {
@@@ -2017,18 -1842,18 +2033,18 @@@
          droppedColumns.put(def.name, FBUtilities.timestampMicros());
      }
  
 -    public void renameColumn(ByteBuffer from, String strFrom, ByteBuffer to, String strTo) throws InvalidRequestException
 +    public void renameColumn(ColumnIdentifier from, ColumnIdentifier to) throws InvalidRequestException
      {
 -        ColumnDefinition def = column_metadata.get(from);
 +        ColumnDefinition def = getColumnDefinition(from);
          if (def == null)
 -            throw new InvalidRequestException(String.format("Cannot rename unknown column %s in keyspace %s", strFrom, cfName));
 +            throw new InvalidRequestException(String.format("Cannot rename unknown column %s in keyspace %s", from, cfName));
  
 -        if (column_metadata.get(to) != null)
 -            throw new InvalidRequestException(String.format("Cannot rename column %s to %s in keyspace %s; another column of that name already exist", strFrom, strTo, cfName));
 +        if (getColumnDefinition(to) != null)
 +            throw new InvalidRequestException(String.format("Cannot rename column %s to %s in keyspace %s; another column of that name already exist", from, to, cfName));
  
-         if (def.kind == ColumnDefinition.Kind.REGULAR)
 -        if (def.type == ColumnDefinition.Type.REGULAR || def.type == ColumnDefinition.Type.STATIC)
++        if (def.kind == ColumnDefinition.Kind.REGULAR || def.kind == ColumnDefinition.Kind.STATIC)
          {
 -            throw new InvalidRequestException(String.format("Cannot rename non PRIMARY KEY part %s", strFrom));
 +            throw new InvalidRequestException(String.format("Cannot rename non PRIMARY KEY part %s", from));
          }
          else if (def.isIndexed())
          {
@@@ -2044,23 -1869,46 +2060,24 @@@
  
      public CFMetaData rebuild()
      {
 -        /*
 -         * TODO: There is definitively some repetition between the CQL3  metadata stored in this
 -         * object (partitionKeyColumns, ...) and the one stored in CFDefinition.
 -         * Ultimately, we should probably merge both. However, there is enough details to fix that
 -         * it's worth doing that in a separate issue.
 -         */
 -        rebuildCQL3Metadata();
 -        cqlCfDef = new CFDefinition(this);
 -        return this;
 -    }
 -
 -    public CFDefinition getCfDef()
 -    {
 -        assert cqlCfDef != null;
 -        return cqlCfDef;
 -    }
 -
 -    private void rebuildCQL3Metadata()
 -    {
          List<ColumnDefinition> pkCols = nullInitializedList(keyValidator.componentsCount());
 -        boolean isDense = isDense(comparator, column_metadata.values());
 -        int nbCkCols = isDense
 -                     ? comparator.componentsCount()
 -                     : comparator.componentsCount() - (hasCollection() ? 2 : 1);
 -        List<ColumnDefinition> ckCols = nullInitializedList(nbCkCols);
 -        Set<ColumnDefinition> regCols = new HashSet<ColumnDefinition>();
 -        Set<ColumnDefinition> statCols = new HashSet<ColumnDefinition>();
 +        List<ColumnDefinition> ckCols = nullInitializedList(comparator.clusteringPrefixSize());
 +        // We keep things sorted to get consistent/predicatable order in select queries
 +        SortedSet<ColumnDefinition> regCols = new TreeSet<>(regularColumnComparator);
++        SortedSet<ColumnDefinition> statCols = new TreeSet<>(regularColumnComparator);
          ColumnDefinition compactCol = null;
  
 -        for (ColumnDefinition def : column_metadata.values())
 +        for (ColumnDefinition def : allColumns())
          {
 -            switch (def.type)
 +            switch (def.kind)
              {
                  case PARTITION_KEY:
 -                    assert !(def.componentIndex == null && keyValidator instanceof CompositeType);
 -                    pkCols.set(def.componentIndex == null ? 0 : def.componentIndex, def);
 +                    assert !(def.isOnAllComponents() && keyValidator instanceof CompositeType);
 +                    pkCols.set(def.position(), def);
                      break;
 -                case CLUSTERING_KEY:
 -                    assert !(def.componentIndex == null && comparator instanceof CompositeType);
 -                    ckCols.set(def.componentIndex == null ? 0 : def.componentIndex, def);
 +                case CLUSTERING_COLUMN:
 +                    assert !(def.isOnAllComponents() && comparator.isCompound());
 +                    ckCols.set(def.position(), def);
                      break;
                  case REGULAR:
                      regCols.add(def);
@@@ -2074,10 -1925,10 +2094,11 @@@
  
          // Now actually assign the correct value. This is not atomic, but then again, updating CFMetaData is never atomic anyway.
          partitionKeyColumns = addDefaultKeyAliases(pkCols);
 -        clusteringKeyColumns = addDefaultColumnAliases(ckCols);
 +        clusteringColumns = addDefaultColumnAliases(ckCols);
          regularColumns = regCols;
+         staticColumns = statCols;
 -        compactValueColumn = addDefaultValueAlias(compactCol, isDense);
 +        compactValueColumn = addDefaultValueAlias(compactCol, comparator.isDense());
 +        return this;
      }
  
      private List<ColumnDefinition> addDefaultKeyAliases(List<ColumnDefinition> pkCols)
@@@ -2241,15 -2091,24 +2262,20 @@@
          return true;
      }
  
 -    public boolean hasStaticColumns()
 +    public boolean isCounter()
      {
 -        return !staticColumns.isEmpty();
 +        return defaultValidator.isCounter();
      }
  
 -    public ColumnNameBuilder getStaticColumnNameBuilder()
++    public boolean hasStaticColumns()
+     {
 -        assert comparator instanceof CompositeType && clusteringKeyColumns().size() > 0;
 -        CompositeType.Builder builder = CompositeType.Builder.staticBuilder((CompositeType)comparator);
 -        for (int i = 0; i < clusteringKeyColumns().size(); i++)
 -            builder.add(ByteBufferUtil.EMPTY_BYTE_BUFFER);
 -        return builder;
++        return !staticColumns.isEmpty();
+     }
+ 
 -    public void validateColumns(Iterable<Column> columns)
 +    public void validateColumns(Iterable<Cell> columns)
      {
 -        for (Column column : columns)
 -            column.validateFields(this);
 +        for (Cell cell : columns)
 +            cell.validateFields(this);
      }
  
      @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/config/ColumnDefinition.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/config/ColumnDefinition.java
index 79f50bf,11340e7..bb1dd71
--- a/src/java/org/apache/cassandra/config/ColumnDefinition.java
+++ b/src/java/org/apache/cassandra/config/ColumnDefinition.java
@@@ -58,71 -59,57 +58,77 @@@ public class ColumnDefinition extends C
       * Note that thrift/CQL2 only know about definitions of type REGULAR (and
       * the ones whose componentIndex == null).
       */
 -    public enum Type
 +    public enum Kind
      {
          PARTITION_KEY,
 -        CLUSTERING_KEY,
 +        CLUSTERING_COLUMN,
          REGULAR,
 -        COMPACT_VALUE,
 -        STATIC
++        STATIC,
 +        COMPACT_VALUE;
 +
 +        public String serialize()
 +        {
 +            // For backward compatibility we need to special case CLUSTERING_COLUMN
 +            return this == CLUSTERING_COLUMN ? "clustering_key" : this.toString().toLowerCase();
 +        }
 +
 +        public static Kind deserialize(String value)
 +        {
 +            if (value.equalsIgnoreCase("clustering_key"))
 +                return CLUSTERING_COLUMN;
 +            return Enum.valueOf(Kind.class, value.toUpperCase());
 +        }
      }
  
 -    public final ByteBuffer name;
 -    private AbstractType<?> validator;
 +    public final Kind kind;
 +
 +    private String indexName;
      private IndexType indexType;
      private Map<String,String> indexOptions;
 -    private String indexName;
 -    public final Type type;
  
      /*
       * If the column comparator is a composite type, indicates to which
       * component this definition refers to. If null, the definition refers to
       * the full column name.
       */
 -    public final Integer componentIndex;
 +    private final Integer componentIndex;
  
 -    public static ColumnDefinition partitionKeyDef(ByteBuffer name, AbstractType<?> validator, Integer componentIndex)
 +    public static ColumnDefinition partitionKeyDef(CFMetaData cfm, ByteBuffer name, AbstractType<?> validator, Integer componentIndex)
      {
 -        return new ColumnDefinition(name, validator, componentIndex, Type.PARTITION_KEY);
 +        return new ColumnDefinition(cfm, name, validator, componentIndex, Kind.PARTITION_KEY);
      }
  
 -    public static ColumnDefinition clusteringKeyDef(ByteBuffer name, AbstractType<?> validator, Integer componentIndex)
 +    public static ColumnDefinition clusteringKeyDef(CFMetaData cfm, ByteBuffer name, AbstractType<?> validator, Integer componentIndex)
      {
 -        return new ColumnDefinition(name, validator, componentIndex, Type.CLUSTERING_KEY);
 +        return new ColumnDefinition(cfm, name, validator, componentIndex, Kind.CLUSTERING_COLUMN);
      }
  
 -    public static ColumnDefinition regularDef(ByteBuffer name, AbstractType<?> validator, Integer componentIndex)
 +    public static ColumnDefinition regularDef(CFMetaData cfm, ByteBuffer name, AbstractType<?> validator, Integer componentIndex)
      {
 -        return new ColumnDefinition(name, validator, componentIndex, Type.REGULAR);
 +        return new ColumnDefinition(cfm, name, validator, componentIndex, Kind.REGULAR);
      }
  
 -    public static ColumnDefinition staticDef(ByteBuffer name, AbstractType<?> validator, Integer componentIndex)
++    public static ColumnDefinition staticDef(CFMetaData cfm, ByteBuffer name, AbstractType<?> validator, Integer componentIndex)
+     {
 -        return new ColumnDefinition(name, validator, componentIndex, Type.STATIC);
++        return new ColumnDefinition(cfm, name, validator, componentIndex, Kind.STATIC);
+     }
+ 
 -    public static ColumnDefinition compactValueDef(ByteBuffer name, AbstractType<?> validator)
 +    public static ColumnDefinition compactValueDef(CFMetaData cfm, ByteBuffer name, AbstractType<?> validator)
      {
 -        return new ColumnDefinition(name, validator, null, Type.COMPACT_VALUE);
 +        return new ColumnDefinition(cfm, name, validator, null, Kind.COMPACT_VALUE);
      }
  
 -    public ColumnDefinition(ByteBuffer name, AbstractType<?> validator, Integer componentIndex, Type type)
 +    public ColumnDefinition(CFMetaData cfm, ByteBuffer name, AbstractType<?> validator, Integer componentIndex, Kind kind)
      {
 -        this(name, validator, null, null, null, componentIndex, type);
 +        this(cfm.ksName,
 +             cfm.cfName,
 +             new ColumnIdentifier(name, cfm.getComponentComparator(componentIndex, kind)),
 +             validator,
 +             null,
 +             null,
 +             null,
 +             componentIndex,
 +             kind);
      }
  
      @VisibleForTesting
@@@ -134,42 -119,25 +140,47 @@@
                              Map<String, String> indexOptions,
                              String indexName,
                              Integer componentIndex,
 -                            Type type)
 +                            Kind kind)
      {
 +        super(ksName, cfName, name, validator);
          assert name != null && validator != null;
 -        this.name = name;
 +        this.kind = kind;
          this.indexName = indexName;
 -        this.validator = validator;
          this.componentIndex = componentIndex;
          this.setIndexType(indexType, indexOptions);
 -        this.type = type;
      }
  
 -    public ColumnDefinition clone()
 +    public ColumnDefinition copy()
 +    {
 +        return new ColumnDefinition(ksName, cfName, name, type, indexType, indexOptions, indexName, componentIndex, kind);
 +    }
 +
 +    public ColumnDefinition withNewName(ColumnIdentifier newName)
 +    {
 +        return new ColumnDefinition(ksName, cfName, newName, type, indexType, indexOptions, indexName, componentIndex, kind);
 +    }
 +
 +    public ColumnDefinition withNewType(AbstractType<?> newType)
      {
 -        return new ColumnDefinition(name, validator, indexType, indexOptions, indexName, componentIndex, type);
 +        return new ColumnDefinition(ksName, cfName, name, newType, indexType, indexOptions, indexName, componentIndex, kind);
      }
  
 -    public ColumnDefinition cloneWithNewName(ByteBuffer newName)
 +    public boolean isOnAllComponents()
      {
 -        return new ColumnDefinition(newName, validator, indexType, indexOptions, indexName, componentIndex, type);
 +        return componentIndex == null;
 +    }
 +
++    public boolean isStatic()
++    {
++        return kind == Kind.STATIC;
++    }
++
 +    // The componentIndex. This never return null however for convenience sake:
 +    // if componentIndex == null, this return 0. So caller should first check
 +    // isOnAllComponents() to distinguish if that's a possibility.
 +    public int position()
 +    {
 +        return componentIndex == null ? 0 : componentIndex;
      }
  
      @Override
@@@ -215,9 -180,10 +226,14 @@@
  
      public boolean isThriftCompatible()
      {
 -        // componentIndex == null should always imply isStatic in practice, but there is no harm in being too careful here.
 -        return type == ColumnDefinition.Type.REGULAR && componentIndex == null;
 +        return kind == ColumnDefinition.Kind.REGULAR && componentIndex == null;
 +    }
 +
++    public boolean isPrimaryKeyColumn()
++    {
++        return kind == Kind.PARTITION_KEY || kind == Kind.CLUSTERING_COLUMN;
+     }
+ 
      public static List<ColumnDef> toThrift(Map<ByteBuffer, ColumnDefinition> columns)
      {
          List<ColumnDef> thriftDefs = new ArrayList<>(columns.size());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/cql3/CQL3Row.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/CQL3Row.java
index afeb095,0000000..6fa2b64
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/CQL3Row.java
+++ b/src/java/org/apache/cassandra/cql3/CQL3Row.java
@@@ -1,36 -1,0 +1,41 @@@
 +/*
 + * 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;
 +
 +import java.nio.ByteBuffer;
 +import java.util.Iterator;
 +import java.util.List;
 +
 +import org.apache.cassandra.db.Cell;
 +
 +public interface CQL3Row
 +{
 +    public ByteBuffer getClusteringColumn(int i);
 +    public Cell getColumn(ColumnIdentifier name);
 +    public List<Cell> getCollection(ColumnIdentifier name);
 +
 +    public interface Builder
 +    {
-         Iterator<CQL3Row> group(Iterator<Cell> cells);
++        public RowIterator group(Iterator<Cell> cells);
++    }
++
++    public interface RowIterator extends Iterator<CQL3Row>
++    {
++        public CQL3Row getStaticRow();
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/cql3/ColumnCondition.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/ColumnCondition.java
index 0000000,797dba6..735388d
mode 000000,100644..100644
--- a/src/java/org/apache/cassandra/cql3/ColumnCondition.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnCondition.java
@@@ -1,0 -1,191 +1,183 @@@
+ /*
+  * 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;
+ 
+ import java.nio.ByteBuffer;
+ import java.util.*;
+ 
+ import com.google.common.base.Predicate;
+ import com.google.common.collect.Iterators;
+ 
+ import org.apache.cassandra.config.CFMetaData;
++import org.apache.cassandra.config.ColumnDefinition;
+ import org.apache.cassandra.db.*;
++import org.apache.cassandra.db.composites.CellName;
++import org.apache.cassandra.db.composites.Composite;
+ import org.apache.cassandra.db.filter.ColumnSlice;
+ import org.apache.cassandra.db.marshal.CollectionType;
+ import org.apache.cassandra.db.marshal.CounterColumnType;
+ import org.apache.cassandra.db.marshal.CompositeType;
+ import org.apache.cassandra.exceptions.InvalidRequestException;
+ 
+ /**
+  * A CQL3 condition.
+  */
+ public class ColumnCondition
+ {
 -    public final CFDefinition.Name column;
++    public final ColumnDefinition column;
+     private final Term value;
+ 
+     private List<ByteBuffer> variables;
+ 
 -    private ColumnCondition(CFDefinition.Name column, Term value)
++    private ColumnCondition(ColumnDefinition column, Term value)
+     {
+         this.column = column;
+         this.value = value;
+     }
+ 
+     // The only ones we support so far
 -    public static ColumnCondition equal(CFDefinition.Name column, Term value)
++    public static ColumnCondition equal(ColumnDefinition column, Term value)
+     {
+         return new ColumnCondition(column, value);
+     }
+ 
+     // See CQL3CasConditions for why it's convenient to have this
+     public ColumnCondition attach(List<ByteBuffer> variables)
+     {
+         this.variables = variables;
+         return this;
+     }
+ 
+     /**
+      * Collects the column specification for the bind variables of this operation.
+      *
+      * @param boundNames the list of column specification where to collect the
+      * bind variables of this term in.
+      */
+     public void collectMarkerSpecification(VariableSpecifications boundNames)
+     {
+         value.collectMarkerSpecification(boundNames);
+     }
+ 
+     // Not overriding equals() because we need the variables to have been attached when this is
+     // called and so having a non standard method name might help avoid mistakes
+     public boolean equalsTo(ColumnCondition other) throws InvalidRequestException
+     {
+         return column.equals(other.column)
+             && value.bindAndGet(variables).equals(other.value.bindAndGet(other.variables));
+     }
+ 
 -    private ColumnNameBuilder copyOrUpdatePrefix(CFMetaData cfm, ColumnNameBuilder rowPrefix)
 -    {
 -        return column.kind == CFDefinition.Name.Kind.STATIC ? cfm.getStaticColumnNameBuilder() : rowPrefix.copy();
 -    }
 -
+     /**
+      * Validates whether this condition applies to {@code current}.
+      */
 -    public boolean appliesTo(ColumnNameBuilder rowPrefix, ColumnFamily current, long now) throws InvalidRequestException
++    public boolean appliesTo(Composite rowPrefix, ColumnFamily current, long now) throws InvalidRequestException
+     {
+         if (column.type instanceof CollectionType)
+             return collectionAppliesTo((CollectionType)column.type, rowPrefix, current, now);
+ 
 -        Column c = current.getColumn(copyOrUpdatePrefix(current.metadata(), rowPrefix).add(column.name.key).build());
++        Cell c = current.getColumn(current.metadata().comparator.create(rowPrefix, column));
+         ByteBuffer v = value.bindAndGet(variables);
+         return v == null
+              ? c == null || !c.isLive(now)
+              : c != null && c.isLive(now) && c.value().equals(v);
+     }
+ 
 -    private boolean collectionAppliesTo(CollectionType type, ColumnNameBuilder rowPrefix, ColumnFamily current, final long now) throws InvalidRequestException
++    private boolean collectionAppliesTo(CollectionType type, Composite rowPrefix, ColumnFamily current, final long now) throws InvalidRequestException
+     {
 -        ColumnNameBuilder collectionPrefix = copyOrUpdatePrefix(current.metadata(), rowPrefix).add(column.name.key);
++        CellName name = current.metadata().comparator.create(rowPrefix, column);
+         // We are testing for collection equality, so we need to have the expected values *and* only those.
 -        ColumnSlice[] collectionSlice = new ColumnSlice[]{ new ColumnSlice(collectionPrefix.build(), collectionPrefix.buildAsEndOfRange()) };
++        ColumnSlice[] collectionSlice = new ColumnSlice[]{ name.slice() };
+         // Filter live columns, this makes things simpler afterwards
 -        Iterator<Column> iter = Iterators.filter(current.iterator(collectionSlice), new Predicate<Column>()
++        Iterator<Cell> iter = Iterators.filter(current.iterator(collectionSlice), new Predicate<Cell>()
+         {
 -            public boolean apply(Column c)
++            public boolean apply(Cell c)
+             {
+                 // we only care about live columns
+                 return c.isLive(now);
+             }
+         });
+ 
+         Term.Terminal v = value.bind(variables);
+         if (v == null)
+             return !iter.hasNext();
+ 
+         switch (type.kind)
+         {
+             case LIST: return listAppliesTo(current.metadata(), iter, ((Lists.Value)v).elements);
+             case SET: return setAppliesTo(current.metadata(), iter, ((Sets.Value)v).elements);
+             case MAP: return mapAppliesTo(current.metadata(), iter, ((Maps.Value)v).map);
+         }
+         throw new AssertionError();
+     }
+ 
 -    private static ByteBuffer collectionKey(CFMetaData cfm, Column c)
 -    {
 -        ByteBuffer[] bbs = ((CompositeType)cfm.comparator).split(c.name());
 -        return bbs[bbs.length - 1];
 -    }
 -
 -    private boolean listAppliesTo(CFMetaData cfm, Iterator<Column> iter, List<ByteBuffer> elements)
++    private boolean listAppliesTo(CFMetaData cfm, Iterator<Cell> iter, List<ByteBuffer> elements)
+     {
+         for (ByteBuffer e : elements)
+             if (!iter.hasNext() || iter.next().value().equals(e))
+                 return false;
+         // We must not have more elements than expected
+         return !iter.hasNext();
+     }
+ 
 -    private boolean setAppliesTo(CFMetaData cfm, Iterator<Column> iter, Set<ByteBuffer> elements)
++    private boolean setAppliesTo(CFMetaData cfm, Iterator<Cell> iter, Set<ByteBuffer> elements)
+     {
+         Set<ByteBuffer> remaining = new HashSet<>(elements);
+         while (iter.hasNext())
+         {
+             if (remaining.isEmpty())
+                 return false;
+ 
 -            if (!remaining.remove(collectionKey(cfm, iter.next())))
++            if (!remaining.remove(iter.next().name().collectionElement()))
+                 return false;
+         }
+         return remaining.isEmpty();
+     }
+ 
 -    private boolean mapAppliesTo(CFMetaData cfm, Iterator<Column> iter, Map<ByteBuffer, ByteBuffer> elements)
++    private boolean mapAppliesTo(CFMetaData cfm, Iterator<Cell> iter, Map<ByteBuffer, ByteBuffer> elements)
+     {
+         Map<ByteBuffer, ByteBuffer> remaining = new HashMap<>(elements);
+         while (iter.hasNext())
+         {
+             if (remaining.isEmpty())
+                 return false;
+ 
 -            Column c = iter.next();
 -            if (!remaining.remove(collectionKey(cfm, c)).equals(c.value()))
++            Cell c = iter.next();
++            if (!remaining.remove(c.name().collectionElement()).equals(c.value()))
+                 return false;
+         }
+         return remaining.isEmpty();
+     }
+ 
+     public static class Raw
+     {
+         private final Term.Raw value;
+ 
+         public Raw(Term.Raw value)
+         {
+             this.value = value;
+         }
+ 
 -        public ColumnCondition prepare(CFDefinition.Name receiver) throws InvalidRequestException
++        public ColumnCondition prepare(String keyspace, ColumnDefinition receiver) throws InvalidRequestException
+         {
+             if (receiver.type instanceof CounterColumnType)
+                 throw new InvalidRequestException("Condtions on counters are not supported");
+ 
 -            return ColumnCondition.equal(receiver, value.prepare(receiver));
++            return ColumnCondition.equal(receiver, value.prepare(keyspace, receiver));
+         }
+     }
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/cql3/Constants.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Constants.java
index 44e96ef,f99fd02..3b7b4c4
--- a/src/java/org/apache/cassandra/cql3/Constants.java
+++ b/src/java/org/apache/cassandra/cql3/Constants.java
@@@ -296,9 -294,10 +296,9 @@@ public abstract class Constant
              super(column, t);
          }
  
 -        public void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder prefix, UpdateParameters params) throws InvalidRequestException
 +        public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException
          {
-             CellName cname = cf.getComparator().create(prefix, column.name);
 -            prefix = maybeUpdatePrefix(cf.metadata(), prefix);
 -            ByteBuffer cname = columnName == null ? prefix.build() : prefix.add(columnName.key).build();
++            CellName cname = cf.getComparator().create(prefix, column);
              ByteBuffer value = t.bindAndGet(params.variables);
              cf.addColumn(value == null ? params.makeTombstone(cname) : params.makeColumn(cname, value));
          }
@@@ -317,7 -316,8 +317,7 @@@
              if (bytes == null)
                  throw new InvalidRequestException("Invalid null value for counter increment");
              long increment = ByteBufferUtil.toLong(bytes);
-             CellName cname = cf.getComparator().create(prefix, column.name);
 -            prefix = maybeUpdatePrefix(cf.metadata(), prefix);
 -            ByteBuffer cname = columnName == null ? prefix.build() : prefix.add(columnName.key).build();
++            CellName cname = cf.getComparator().create(prefix, column);
              cf.addCounter(cname, increment);
          }
      }
@@@ -339,7 -339,8 +339,7 @@@
              if (increment == Long.MIN_VALUE)
                  throw new InvalidRequestException("The negation of " + increment + " overflows supported counter precision (signed 8 bytes integer)");
  
-             CellName cname = cf.getComparator().create(prefix, column.name);
 -            prefix = maybeUpdatePrefix(cf.metadata(), prefix);
 -            ByteBuffer cname = columnName == null ? prefix.build() : prefix.add(columnName.key).build();
++            CellName cname = cf.getComparator().create(prefix, column);
              cf.addCounter(cname, -increment);
          }
      }
@@@ -348,18 -349,22 +348,18 @@@
      // duplicating this further
      public static class Deleter extends Operation
      {
 -        private final boolean isCollection;
 -
 -        public Deleter(ColumnIdentifier column, boolean isCollection)
 +        public Deleter(ColumnDefinition column)
          {
              super(column, null);
 -            this.isCollection = isCollection;
          }
  
 -        public void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder prefix, UpdateParameters params) throws InvalidRequestException
 +        public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException
          {
-             CellName cname = cf.getComparator().create(prefix, column.name);
 -            ColumnNameBuilder column = maybeUpdatePrefix(cf.metadata(), prefix).add(columnName.key);
 -
 -            if (isCollection)
 -                cf.addAtom(params.makeRangeTombstone(column.build(), column.buildAsEndOfRange()));
++            CellName cname = cf.getComparator().create(prefix, column);
 +            if (column.type.isCollection())
 +                cf.addAtom(params.makeRangeTombstone(cname.slice()));
              else
 -                cf.addColumn(params.makeTombstone(column.build()));
 +                cf.addColumn(params.makeTombstone(cname));
          }
      };
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Cql.g
index 55d8aac,a11a818..0ec283c
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@@ -1081,8 -988,9 +1088,9 @@@ basic_unreserved_keyword returns [Strin
          | K_CUSTOM
          | K_TRIGGER
          | K_DISTINCT
 +        | K_CONTAINS
+         | K_STATIC
          ) { $str = $k.text; }
 -    | t=native_type { $str = t.toString(); }
      ;
  
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/cql3/Lists.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Lists.java
index 8dbe59c,4ad39db..580a2c9
--- a/src/java/org/apache/cassandra/cql3/Lists.java
+++ b/src/java/org/apache/cassandra/cql3/Lists.java
@@@ -265,12 -263,12 +265,12 @@@ public abstract class List
              super(column, t);
          }
  
 -        public void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder prefix, UpdateParameters params) throws InvalidRequestException
 +        public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException
          {
              // delete + append
-             CellName name = cf.getComparator().create(prefix, column.name);
 -            ColumnNameBuilder column = maybeUpdatePrefix(cf.metadata(), prefix).add(columnName.key);
 -            cf.addAtom(params.makeTombstoneForOverwrite(column.build(), column.buildAsEndOfRange()));
 -            Appender.doAppend(t, cf, column, params);
++            CellName name = cf.getComparator().create(prefix, column);
 +            cf.addAtom(params.makeTombstoneForOverwrite(name.slice()));
-             Appender.doAppend(t, cf, prefix, column.name, params);
++            Appender.doAppend(t, cf, prefix, column, params);
          }
      }
  
@@@ -335,12 -335,12 +335,12 @@@
              super(column, t);
          }
  
 -        public void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder prefix, UpdateParameters params) throws InvalidRequestException
 +        public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException
          {
-             doAppend(t, cf, prefix, column.name, params);
 -            doAppend(t, cf, maybeUpdatePrefix(cf.metadata(), prefix).add(columnName.key), params);
++            doAppend(t, cf, prefix, column, params);
          }
  
-         static void doAppend(Term t, ColumnFamily cf, Composite prefix, ColumnIdentifier columnName, UpdateParameters params) throws InvalidRequestException
 -        static void doAppend(Term t, ColumnFamily cf, ColumnNameBuilder columnName, UpdateParameters params) throws InvalidRequestException
++        static void doAppend(Term t, ColumnFamily cf, Composite prefix, ColumnDefinition column, UpdateParameters params) throws InvalidRequestException
          {
              Term.Terminal value = t.bind(params.variables);
              // If we append null, do nothing. Note that for Setter, we've
@@@ -352,8 -352,10 +352,8 @@@
              List<ByteBuffer> toAdd = ((Lists.Value)value).elements;
              for (int i = 0; i < toAdd.size(); i++)
              {
 -                ColumnNameBuilder b = i == toAdd.size() - 1 ? columnName : columnName.copy();
                  ByteBuffer uuid = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes());
-                 cf.addColumn(params.makeColumn(cf.getComparator().create(prefix, columnName, uuid), toAdd.get(i)));
 -                ByteBuffer cellName = b.add(uuid).build();
 -                cf.addColumn(params.makeColumn(cellName, toAdd.get(i)));
++                cf.addColumn(params.makeColumn(cf.getComparator().create(prefix, column, uuid), toAdd.get(i)));
              }
          }
      }
@@@ -375,11 -377,14 +375,11 @@@
              long time = PrecisionTime.REFERENCE_TIME - (System.currentTimeMillis() - PrecisionTime.REFERENCE_TIME);
  
              List<ByteBuffer> toAdd = ((Lists.Value)value).elements;
 -            ColumnNameBuilder column = maybeUpdatePrefix(cf.metadata(), prefix).add(columnName.key);
              for (int i = 0; i < toAdd.size(); i++)
              {
 -                ColumnNameBuilder b = i == toAdd.size() - 1 ? column : column.copy();
                  PrecisionTime pt = PrecisionTime.getNext(time);
                  ByteBuffer uuid = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes(pt.millis, pt.nanos));
-                 cf.addColumn(params.makeColumn(cf.getComparator().create(prefix, column.name, uuid), toAdd.get(i)));
 -                ByteBuffer cellName = b.add(uuid).build();
 -                cf.addColumn(params.makeColumn(cellName, toAdd.get(i)));
++                cf.addColumn(params.makeColumn(cf.getComparator().create(prefix, column, uuid), toAdd.get(i)));
              }
          }
      }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/cql3/Maps.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Maps.java
index 3f9cc95,c332999..d113b57
--- a/src/java/org/apache/cassandra/cql3/Maps.java
+++ b/src/java/org/apache/cassandra/cql3/Maps.java
@@@ -244,12 -241,12 +244,12 @@@ public abstract class Map
              super(column, t);
          }
  
 -        public void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder prefix, UpdateParameters params) throws InvalidRequestException
 +        public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException
          {
              // delete + put
-             CellName name = cf.getComparator().create(prefix, column.name);
 -            ColumnNameBuilder column = maybeUpdatePrefix(cf.metadata(), prefix).add(columnName.key);
 -            cf.addAtom(params.makeTombstoneForOverwrite(column.build(), column.buildAsEndOfRange()));
 -            Putter.doPut(t, cf, column, params);
++            CellName name = cf.getComparator().create(prefix, column);
 +            cf.addAtom(params.makeTombstoneForOverwrite(name.slice()));
-             Putter.doPut(t, cf, prefix, column.name, params);
++            Putter.doPut(t, cf, prefix, column, params);
          }
      }
  
@@@ -277,7 -274,7 +277,7 @@@
              if (key == null)
                  throw new InvalidRequestException("Invalid null map key");
  
-             CellName cellName = cf.getComparator().create(prefix, column.name, key);
 -            ByteBuffer cellName = maybeUpdatePrefix(cf.metadata(), prefix).add(columnName.key).add(key).build();
++            CellName cellName = cf.getComparator().create(prefix, column, key);
  
              if (value == null)
              {
@@@ -303,12 -300,12 +303,12 @@@
              super(column, t);
          }
  
 -        public void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder prefix, UpdateParameters params) throws InvalidRequestException
 +        public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException
          {
-             doPut(t, cf, prefix, column.name, params);
 -            doPut(t, cf, maybeUpdatePrefix(cf.metadata(), prefix).add(columnName.key), params);
++            doPut(t, cf, prefix, column, params);
          }
  
-         static void doPut(Term t, ColumnFamily cf, Composite prefix, ColumnIdentifier columnName, UpdateParameters params) throws InvalidRequestException
 -        static void doPut(Term t, ColumnFamily cf, ColumnNameBuilder columnName, UpdateParameters params) throws InvalidRequestException
++        static void doPut(Term t, ColumnFamily cf, Composite prefix, ColumnDefinition column, UpdateParameters params) throws InvalidRequestException
          {
              Term.Terminal value = t.bind(params.variables);
              if (value == null)
@@@ -318,7 -315,7 +318,7 @@@
              Map<ByteBuffer, ByteBuffer> toAdd = ((Maps.Value)value).map;
              for (Map.Entry<ByteBuffer, ByteBuffer> entry : toAdd.entrySet())
              {
-                 CellName cellName = cf.getComparator().create(prefix, columnName, entry.getKey());
 -                ByteBuffer cellName = columnName.copy().add(entry.getKey()).build();
++                CellName cellName = cf.getComparator().create(prefix, column, entry.getKey());
                  cf.addColumn(params.makeColumn(cellName, entry.getValue()));
              }
          }
@@@ -338,7 -335,7 +338,7 @@@
                  throw new InvalidRequestException("Invalid null map key");
              assert key instanceof Constants.Value;
  
-             CellName cellName = cf.getComparator().create(prefix, column.name, ((Constants.Value)key).bytes);
 -            ByteBuffer cellName = maybeUpdatePrefix(cf.metadata(), prefix).add(columnName.key).add(((Constants.Value)key).bytes).build();
++            CellName cellName = cf.getComparator().create(prefix, column, ((Constants.Value)key).bytes);
              cf.addColumn(params.makeTombstone(cellName));
          }
      }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/cql3/Sets.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Sets.java
index dddea09,69bc3d3..e48a3ce
--- a/src/java/org/apache/cassandra/cql3/Sets.java
+++ b/src/java/org/apache/cassandra/cql3/Sets.java
@@@ -230,12 -227,12 +230,12 @@@ public abstract class Set
              super(column, t);
          }
  
 -        public void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder prefix, UpdateParameters params) throws InvalidRequestException
 +        public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException
          {
              // delete + add
-             CellName name = cf.getComparator().create(prefix, column.name);
 -            ColumnNameBuilder column = maybeUpdatePrefix(cf.metadata(), prefix).add(columnName.key);
 -            cf.addAtom(params.makeTombstoneForOverwrite(column.build(), column.buildAsEndOfRange()));
 -            Adder.doAdd(t, cf, column, params);
++            CellName name = cf.getComparator().create(prefix, column);
 +            cf.addAtom(params.makeTombstoneForOverwrite(name.slice()));
-             Adder.doAdd(t, cf, prefix, column.name, params);
++            Adder.doAdd(t, cf, prefix, column, params);
          }
      }
  
@@@ -246,12 -243,12 +246,12 @@@
              super(column, t);
          }
  
 -        public void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder prefix, UpdateParameters params) throws InvalidRequestException
 +        public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException
          {
-             doAdd(t, cf, prefix, column.name, params);
 -            doAdd(t, cf, maybeUpdatePrefix(cf.metadata(), prefix).add(columnName.key), params);
++            doAdd(t, cf, prefix, column, params);
          }
  
-         static void doAdd(Term t, ColumnFamily cf, Composite prefix, ColumnIdentifier columnName, UpdateParameters params) throws InvalidRequestException
 -        static void doAdd(Term t, ColumnFamily cf, ColumnNameBuilder columnName, UpdateParameters params) throws InvalidRequestException
++        static void doAdd(Term t, ColumnFamily cf, Composite prefix, ColumnDefinition column, UpdateParameters params) throws InvalidRequestException
          {
              Term.Terminal value = t.bind(params.variables);
              if (value == null)
@@@ -262,7 -259,7 +262,7 @@@
              Set<ByteBuffer> toAdd = ((Sets.Value)value).elements;
              for (ByteBuffer bb : toAdd)
              {
-                 CellName cellName = cf.getComparator().create(prefix, columnName, bb);
 -                ByteBuffer cellName = columnName.copy().add(bb).build();
++                CellName cellName = cf.getComparator().create(prefix, column, bb);
                  cf.addColumn(params.makeColumn(cellName, ByteBufferUtil.EMPTY_BYTE_BUFFER));
              }
          }
@@@ -286,9 -283,11 +286,9 @@@
                                        ? Collections.singleton(((Constants.Value)value).bytes)
                                        : ((Sets.Value)value).elements;
  
 -            ColumnNameBuilder column = maybeUpdatePrefix(cf.metadata(), prefix).add(columnName.key);
              for (ByteBuffer bb : toDiscard)
              {
-                 cf.addColumn(params.makeTombstone(cf.getComparator().create(prefix, column.name, bb)));
 -                ByteBuffer cellName = column.copy().add(bb).build();
 -                cf.addColumn(params.makeTombstone(cellName));
++                cf.addColumn(params.makeTombstone(cf.getComparator().create(prefix, column, bb)));
              }
          }
      }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
index 0932b5c,85b3547..9c097a3
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
@@@ -45,8 -47,15 +45,15 @@@ public class AlterTableStatement extend
      public final ColumnIdentifier columnName;
      private final CFPropDefs cfProps;
      private final Map<ColumnIdentifier, ColumnIdentifier> renames;
- 
-     public AlterTableStatement(CFName name, Type type, ColumnIdentifier columnName, CQL3Type.Raw validator, CFPropDefs cfProps, Map<ColumnIdentifier, ColumnIdentifier> renames)
+     private final boolean isStatic; // Only for ALTER ADD
+ 
+     public AlterTableStatement(CFName name,
+                                Type type,
+                                ColumnIdentifier columnName,
 -                               CQL3Type validator,
++                               CQL3Type.Raw validator,
+                                CFPropDefs cfProps,
+                                Map<ColumnIdentifier, ColumnIdentifier> renames,
+                                boolean isStatic)
      {
          super(name);
          this.oType = type;
@@@ -77,16 -86,20 +85,18 @@@
          switch (oType)
          {
              case ADD:
 -                if (cfDef.isCompact)
 +                if (cfm.comparator.isDense())
-                     throw new InvalidRequestException("Cannot add new column to a compact CF");
+                     throw new InvalidRequestException("Cannot add new column to a COMPACT STORAGE table");
 -
 -                if (isStatic && !cfDef.isComposite)
++                if (isStatic && !cfm.comparator.isCompound())
+                     throw new InvalidRequestException("Static columns are not allowed in COMPACT STORAGE tables");
 -
 -                if (name != null)
 +                if (def != null)
                  {
 -                    switch (name.kind)
 +                    switch (def.kind)
                      {
 -                        case KEY_ALIAS:
 -                        case COLUMN_ALIAS:
 +                        case PARTITION_KEY:
 +                        case CLUSTERING_COLUMN:
                              throw new InvalidRequestException(String.format("Invalid column name %s because it conflicts with a PRIMARY KEY part", columnName));
-                         case REGULAR:
+                         default:
                              throw new InvalidRequestException(String.format("Invalid column name %s because it conflicts with an existing column", columnName));
                      }
                  }
@@@ -94,16 -107,31 +104,18 @@@
                  AbstractType<?> type = validator.getType();
                  if (type instanceof CollectionType)
                  {
 -                    if (!cfDef.isComposite)
 +                    if (!cfm.comparator.supportCollections())
                          throw new InvalidRequestException("Cannot use collection types with non-composite PRIMARY KEY");
 -                    if (cfDef.cfm.isSuper())
 +                    if (cfm.isSuper())
                          throw new InvalidRequestException("Cannot use collection types with Super column family");
  
 -                    Map<ByteBuffer, CollectionType> collections = cfDef.hasCollections
 -                                                                ? new HashMap<ByteBuffer, CollectionType>(cfDef.getCollectionType().defined)
 -                                                                : new HashMap<ByteBuffer, CollectionType>();
 -
 -                    collections.put(columnName.key, (CollectionType)type);
 -                    ColumnToCollectionType newColType = ColumnToCollectionType.getInstance(collections);
 -                    List<AbstractType<?>> ctypes = new ArrayList<AbstractType<?>>(((CompositeType)cfm.comparator).types);
 -                    if (cfDef.hasCollections)
 -                        ctypes.set(ctypes.size() - 1, newColType);
 -                    else
 -                        ctypes.add(newColType);
 -                    cfm.comparator = CompositeType.getInstance(ctypes);
 +                    cfm.comparator = cfm.comparator.addCollection(columnName, (CollectionType)type);
                  }
  
 -                Integer componentIndex = cfDef.isComposite
 -                                       ? ((CompositeType)meta.comparator).types.size() - (cfDef.hasCollections ? 2 : 1)
 -                                       : null;
 +                Integer componentIndex = cfm.comparator.isCompound() ? cfm.comparator.clusteringPrefixSize() : null;
-                 cfm.addColumnDefinition(ColumnDefinition.regularDef(cfm, columnName.bytes, type, componentIndex));
+                 cfm.addColumnDefinition(isStatic
 -                                        ? ColumnDefinition.staticDef(columnName.key, type, componentIndex)
 -                                        : ColumnDefinition.regularDef(columnName.key, type, componentIndex));
++                                        ? ColumnDefinition.staticDef(cfm, columnName.bytes, type, componentIndex)
++                                        : ColumnDefinition.regularDef(cfm, columnName.bytes, type, componentIndex));
                  break;
  
              case ALTER:
@@@ -161,7 -191,9 +173,8 @@@
                                                                             validator));
                          cfm.defaultValidator(validator.getType());
                          break;
 -                    case COLUMN_METADATA:
 +                    case REGULAR:
+                     case STATIC:
 -                        ColumnDefinition column = cfm.getColumnDefinition(columnName.key);
                          // Thrift allows to change a column validator so CFMetaData.validateCompatibility will let it slide
                          // if we change to an incompatible type (contrarily to the comparator case). But we don't want to
                          // allow it for CQL3 (see #5882) so validating it explicitly here. We only care about value compatibility
@@@ -180,21 -211,22 +193,22 @@@
                  break;
  
              case DROP:
 -                if (cfDef.isCompact || !cfDef.isComposite)
 -                    throw new InvalidRequestException("Cannot drop columns from a COMPACT STORAGE table");
 -                if (name == null)
 +                if (!cfm.isCQL3Table())
-                     throw new InvalidRequestException("Cannot drop columns from a non-CQL3 CF");
++                    throw new InvalidRequestException("Cannot drop columns from a non-CQL3 table");
 +                if (def == null)
-                     throw new InvalidRequestException(String.format("Cell %s was not found in table %s", columnName, columnFamily()));
+                     throw new InvalidRequestException(String.format("Column %s was not found in table %s", columnName, columnFamily()));
  
 -                switch (name.kind)
 +                switch (def.kind)
                  {
 -                    case KEY_ALIAS:
 -                    case COLUMN_ALIAS:
 +                    case PARTITION_KEY:
 +                    case CLUSTERING_COLUMN:
                          throw new InvalidRequestException(String.format("Cannot drop PRIMARY KEY part %s", columnName));
 -                    case COLUMN_METADATA:
 +                    case REGULAR:
+                     case STATIC:
                          ColumnDefinition toDelete = null;
-                         for (ColumnDefinition columnDef : cfm.regularColumns())
+                         for (ColumnDefinition columnDef : cfm.regularAndStaticColumns())
                          {
 -                            if (columnDef.name.equals(columnName.key))
 +                            if (columnDef.name.equals(columnName))
                                  toDelete = columnDef;
                          }
                          assert toDelete != null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index 2c8b2b7,d4acbae..ab2b4bc
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@@ -20,12 -20,11 +20,13 @@@ package org.apache.cassandra.cql3.state
  import java.nio.ByteBuffer;
  import java.util.*;
  
+ import com.google.common.collect.Iterables;
  import org.github.jamm.MemoryMeter;
  
++import org.apache.cassandra.config.ColumnDefinition;
  import org.apache.cassandra.cql3.*;
- import org.apache.cassandra.db.ConsistencyLevel;
- import org.apache.cassandra.db.IMutation;
- import org.apache.cassandra.db.Mutation;
+ import org.apache.cassandra.db.*;
++import org.apache.cassandra.db.composites.Composite;
  import org.apache.cassandra.exceptions.*;
  import org.apache.cassandra.service.ClientState;
  import org.apache.cassandra.service.QueryState;
@@@ -175,17 -182,103 +184,108 @@@ public class BatchStatement implements 
          StorageProxy.mutateWithTriggers(mutations, cl, mutateAtomic);
      }
  
+     private ResultMessage executeWithConditions(BatchVariables variables, ConsistencyLevel cl, ConsistencyLevel serialCf, long now)
+     throws RequestExecutionException, RequestValidationException
+     {
+         ByteBuffer key = null;
+         String ksName = null;
+         String cfName = null;
+         ColumnFamily updates = null;
+         CQL3CasConditions conditions = null;
 -        Set<ColumnIdentifier> columnsWithConditions = new LinkedHashSet<ColumnIdentifier>();
++        Set<ColumnDefinition> columnsWithConditions = new LinkedHashSet<ColumnDefinition>();
+ 
+         for (int i = 0; i < statements.size(); i++)
+         {
+             ModificationStatement statement = statements.get(i);
+             List<ByteBuffer> statementVariables = variables.getVariablesForStatement(i);
+             long timestamp = attrs.getTimestamp(now, statementVariables);
+             List<ByteBuffer> pks = statement.buildPartitionKeyNames(statementVariables);
+             if (pks.size() > 1)
+                 throw new IllegalArgumentException("Batch with conditions cannot span multiple partitions (you cannot use IN on the partition key)");
+             if (key == null)
+             {
+                 key = pks.get(0);
+                 ksName = statement.cfm.ksName;
+                 cfName = statement.cfm.cfName;
+                 conditions = new CQL3CasConditions(statement.cfm, now);
 -                updates = UnsortedColumns.factory.create(statement.cfm);
++                updates = ArrayBackedSortedColumns.factory.create(statement.cfm);
+             }
+             else if (!key.equals(pks.get(0)))
+             {
+                 throw new InvalidRequestException("Batch with conditions cannot span multiple partitions");
+             }
+ 
+             if (statement.hasConditions())
+             {
 -                ColumnNameBuilder clusteringPrefix = statement.createClusteringPrefixBuilder(statementVariables);
++                Composite clusteringPrefix = statement.createClusteringPrefix(statementVariables);
+                 statement.addUpdatesAndConditions(key, clusteringPrefix, updates, conditions, statementVariables, timestamp);
+                 // As soon as we have a ifNotExists, we set columnsWithConditions to null so that everything is in the resultSet
+                 if (statement.hasIfNotExistCondition())
+                     columnsWithConditions = null;
+                 else if (columnsWithConditions != null)
+                     Iterables.addAll(columnsWithConditions, statement.getColumnsWithConditions());
+             }
+             else
+             {
+                 // getPartitionKey will already have thrown if there is more than one key involved
+                 IMutation mut = statement.getMutations(statementVariables, false, cl, timestamp, true).iterator().next();
 -                updates.resolve(mut.getColumnFamilies().iterator().next());
++                updates.addAll(mut.getColumnFamilies().iterator().next());
+             }
+         }
+ 
+         ColumnFamily result = StorageProxy.cas(ksName, cfName, key, conditions, updates, serialCf, cl);
+         return new ResultMessage.Rows(ModificationStatement.buildCasResultSet(ksName, key, cfName, result, columnsWithConditions, true));
+     }
+ 
      public ResultMessage executeInternal(QueryState queryState) throws RequestValidationException, RequestExecutionException
      {
-         for (IMutation mutation : getMutations(Collections.<ByteBuffer>emptyList(), true, null, queryState.getTimestamp()))
+         assert !hasConditions;
 -
 -        for (IMutation mutation : getMutations(new PreparedBatchVariables(Collections.<ByteBuffer>emptyList()), true, null, queryState.getTimestamp()))
 -            mutation.apply();
++        for (IMutation mutation : getMutations(PreparedBatchVariables.EMPTY, true, null, queryState.getTimestamp()))
 +        {
 +            // We don't use counters internally.
 +            assert mutation instanceof Mutation;
 +            ((Mutation) mutation).apply();
 +        }
          return null;
      }
  
+     public interface BatchVariables
+     {
+         public List<ByteBuffer> getVariablesForStatement(int statementInBatch);
+     }
+ 
+     public static class PreparedBatchVariables implements BatchVariables
+     {
++        public static final BatchVariables EMPTY = new PreparedBatchVariables(Collections.<ByteBuffer>emptyList());
++
+         private final List<ByteBuffer> variables;
+ 
+         public PreparedBatchVariables(List<ByteBuffer> variables)
+         {
+             this.variables = variables;
+         }
+ 
+         public List<ByteBuffer> getVariablesForStatement(int statementInBatch)
+         {
+             return variables;
+         }
+     }
+ 
+     public static class BatchOfPreparedVariables implements BatchVariables
+     {
+         private final List<List<ByteBuffer>> variables;
+ 
+         public BatchOfPreparedVariables(List<List<ByteBuffer>> variables)
+         {
+             this.variables = variables;
+         }
+ 
+         public List<ByteBuffer> getVariablesForStatement(int statementInBatch)
+         {
+             return variables.get(statementInBatch);
+         }
+     }
+ 
      public String toString()
      {
          return String.format("BatchStatement(type=%s, statements=%s)", type, statements);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
index 0000000,194ff0c..56692f9
mode 000000,100644..100644
--- a/src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
@@@ -1,0 -1,164 +1,164 @@@
+ /*
+  * 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.statements;
+ 
+ import java.nio.ByteBuffer;
+ import java.util.*;
+ 
+ import org.apache.cassandra.cql3.*;
+ import org.apache.cassandra.config.CFMetaData;
+ import org.apache.cassandra.db.*;
++import org.apache.cassandra.db.composites.Composite;
+ import org.apache.cassandra.db.filter.*;
+ import org.apache.cassandra.exceptions.InvalidRequestException;
+ import org.apache.cassandra.service.CASConditions;
+ 
+ /**
+  * Processed CAS conditions on potentially multiple rows of the same partition.
+  */
+ public class CQL3CasConditions implements CASConditions
+ {
+     private final CFMetaData cfm;
+     private final long now;
+ 
+     // We index RowCondition by the prefix of the row they applied to for 2 reasons:
+     //   1) this allows to keep things sorted to build the ColumnSlice array below
+     //   2) this allows to detect when contradictory conditions are set (not exists with some other conditions on the same row)
 -    private final SortedMap<ByteBuffer, RowCondition> conditions;
++    private final SortedMap<Composite, RowCondition> conditions;
+ 
+     public CQL3CasConditions(CFMetaData cfm, long now)
+     {
+         this.cfm = cfm;
+         this.now = now;
+         this.conditions = new TreeMap<>(cfm.comparator);
+     }
+ 
 -    public void addNotExist(ColumnNameBuilder prefix) throws InvalidRequestException
++    public void addNotExist(Composite prefix) throws InvalidRequestException
+     {
 -        RowCondition previous = conditions.put(prefix.build(), new NotExistCondition(prefix, now));
++        RowCondition previous = conditions.put(prefix, new NotExistCondition(prefix, now));
+         if (previous != null && !(previous instanceof NotExistCondition))
+             throw new InvalidRequestException("Cannot mix IF conditions and IF NOT EXISTS for the same row");
+     }
+ 
 -    public void addConditions(ColumnNameBuilder prefix, Collection<ColumnCondition> conds, List<ByteBuffer> variables) throws InvalidRequestException
++    public void addConditions(Composite prefix, Collection<ColumnCondition> conds, List<ByteBuffer> variables) throws InvalidRequestException
+     {
 -        ByteBuffer b = prefix.build();
 -        RowCondition condition = conditions.get(b);
++        RowCondition condition = conditions.get(prefix);
+         if (condition == null)
+         {
+             condition = new ColumnsConditions(prefix, now);
 -            conditions.put(b, condition);
++            conditions.put(prefix, condition);
+         }
+         else if (!(condition instanceof ColumnsConditions))
+         {
+             throw new InvalidRequestException("Cannot mix IF conditions and IF NOT EXISTS for the same row");
+         }
+         ((ColumnsConditions)condition).addConditions(conds, variables);
+     }
+ 
+     public IDiskAtomFilter readFilter()
+     {
+         assert !conditions.isEmpty();
+         ColumnSlice[] slices = new ColumnSlice[conditions.size()];
+         int i = 0;
+         // We always read CQL rows entirely as on CAS failure we want to be able to distinguish between "row exists
 -        // but all values on why there were conditions are null" and "row doesn't exists", and we can't rely on the
++        // but all values for which there were conditions are null" and "row doesn't exists", and we can't rely on the
+         // row marker for that (see #6623)
 -        for (Map.Entry<ByteBuffer, RowCondition> entry : conditions.entrySet())
 -            slices[i++] = new ColumnSlice(entry.getKey(), entry.getValue().rowPrefix.buildAsEndOfRange());
++        for (Composite prefix : conditions.keySet())
++            slices[i++] = prefix.slice();
+ 
 -        return new SliceQueryFilter(slices, false, slices.length, cfm.clusteringKeyColumns().size());
++        return new SliceQueryFilter(slices, false, slices.length, cfm.clusteringColumns().size());
+     }
+ 
+     public boolean appliesTo(ColumnFamily current) throws InvalidRequestException
+     {
+         for (RowCondition condition : conditions.values())
+         {
+             if (!condition.appliesTo(current))
+                 return false;
+         }
+         return true;
+     }
+ 
+     private static abstract class RowCondition
+     {
 -        public final ColumnNameBuilder rowPrefix;
++        public final Composite rowPrefix;
+         protected final long now;
+ 
 -        protected RowCondition(ColumnNameBuilder rowPrefix, long now)
++        protected RowCondition(Composite rowPrefix, long now)
+         {
+             this.rowPrefix = rowPrefix;
+             this.now = now;
+         }
+ 
+         public abstract boolean appliesTo(ColumnFamily current) throws InvalidRequestException;
+     }
+ 
+     private static class NotExistCondition extends RowCondition
+     {
 -        private NotExistCondition(ColumnNameBuilder rowPrefix, long now)
++        private NotExistCondition(Composite rowPrefix, long now)
+         {
+             super(rowPrefix, now);
+         }
+ 
+         public boolean appliesTo(ColumnFamily current)
+         {
+             if (current == null)
+                 return true;
+ 
 -            Iterator<Column> iter = current.iterator(new ColumnSlice[]{ new ColumnSlice(rowPrefix.build(), rowPrefix.buildAsEndOfRange()) });
++            Iterator<Cell> iter = current.iterator(new ColumnSlice[]{ rowPrefix.slice() });
+             while (iter.hasNext())
+                 if (iter.next().isLive(now))
+                     return false;
+             return true;
+         }
+     }
+ 
+     private static class ColumnsConditions extends RowCondition
+     {
+         private final Map<ColumnIdentifier, ColumnCondition> conditions = new HashMap<>();
+ 
 -        private ColumnsConditions(ColumnNameBuilder rowPrefix, long now)
++        private ColumnsConditions(Composite rowPrefix, long now)
+         {
+             super(rowPrefix, now);
+         }
+ 
+         public void addConditions(Collection<ColumnCondition> conds, List<ByteBuffer> variables) throws InvalidRequestException
+         {
+             for (ColumnCondition condition : conds)
+             {
+                 // We will need the variables in appliesTo but with protocol batches, each condition in this object can have a
+                 // different list of variables. So attach them to the condition directly, it's not particulary elegant but its simpler
+                 ColumnCondition previous = conditions.put(condition.column.name, condition.attach(variables));
+                 // If 2 conditions are actually equal, let it slide
+                 if (previous != null && !previous.equalsTo(condition))
+                     throw new InvalidRequestException("Duplicate and incompatible conditions for column " + condition.column.name);
+             }
+         }
+ 
+         public boolean appliesTo(ColumnFamily current) throws InvalidRequestException
+         {
+             if (current == null)
+                 return conditions.isEmpty();
+ 
+             for (ColumnCondition condition : conditions.values())
+                 if (!condition.appliesTo(rowPrefix, current, now))
+                     return false;
+             return true;
+         }
+     }
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
index fe5508b,376fa4a..49a669f
--- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
@@@ -101,11 -84,23 +101,20 @@@ public class CreateIndexStatement exten
          properties.validate();
  
          // TODO: we could lift that limitation
 -        if (cfm.getCfDef().isCompact && cd.type != ColumnDefinition.Type.REGULAR)
 -            throw new InvalidRequestException(String.format("Secondary index on %s column %s is not yet supported for compact table", cd.type, columnName));
 +        if (cfm.comparator.isDense() && cd.kind != ColumnDefinition.Kind.REGULAR)
 +            throw new InvalidRequestException(String.format("Secondary index on %s column %s is not yet supported for compact table", cd.kind, target.column));
  
+         // It would be possible to support 2ndary index on static columns (but not without modifications of at least ExtendedFilter and
+         // CompositesIndex) and maybe we should, but that means a query like:
+         //     SELECT * FROM foo WHERE static_column = 'bar'
+         // would pull the full partition every time the static column of partition is 'bar', which sounds like offering a
+         // fair potential for foot-shooting, so I prefer leaving that to a follow up ticket once we have identified cases where
+         // such indexing is actually useful.
 -        if (cd.type == ColumnDefinition.Type.STATIC)
++        if (cd.isStatic())
+             throw new InvalidRequestException("Secondary indexes are not allowed on static columns");
+ 
 -        if (cd.getValidator().isCollection() && !properties.isCustom)
 -            throw new InvalidRequestException("Indexes on collections are no yet supported");
 -
 -        if (cd.type == ColumnDefinition.Type.PARTITION_KEY && cd.componentIndex == null)
 -            throw new InvalidRequestException(String.format("Cannot add secondary index to already primarily indexed column %s", columnName));
 +        if (cd.kind == ColumnDefinition.Kind.PARTITION_KEY && cd.isOnAllComponents())
 +            throw new InvalidRequestException(String.format("Cannot add secondary index to already primarily indexed column %s", target.column));
      }
  
      public void announceMigration() throws RequestValidationException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
index 8351e71,632194c..7cef999
--- a/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
@@@ -88,12 -89,25 +90,17 @@@ public class CreateTableStatement exten
      }
  
      // Column definitions
 -    private Map<ByteBuffer, ColumnDefinition> getColumns()
 +    private List<ColumnDefinition> getColumns(CFMetaData cfm)
      {
 -        Map<ByteBuffer, ColumnDefinition> columnDefs = new HashMap<ByteBuffer, ColumnDefinition>();
 -        Integer componentIndex = null;
 -        if (comparator instanceof CompositeType)
 -        {
 -            CompositeType ct = (CompositeType) comparator;
 -            componentIndex = ct.types.get(ct.types.size() - 1) instanceof ColumnToCollectionType
 -                           ? ct.types.size() - 2
 -                           : ct.types.size() - 1;
 -        }
 -
 +        List<ColumnDefinition> columnDefs = new ArrayList<>(columns.size());
 +        Integer componentIndex = comparator.isCompound() ? comparator.clusteringPrefixSize() : null;
          for (Map.Entry<ColumnIdentifier, AbstractType> col : columns.entrySet())
-             columnDefs.add(ColumnDefinition.regularDef(cfm, col.getKey().bytes, col.getValue(), componentIndex));
+         {
+             ColumnIdentifier id = col.getKey();
 -            columnDefs.put(id.key, staticColumns.contains(id)
 -                                   ? ColumnDefinition.staticDef(id.key, col.getValue(), componentIndex)
 -                                   : ColumnDefinition.regularDef(id.key, col.getValue(), componentIndex));
++            columnDefs.add(staticColumns.contains(id)
++                           ? ColumnDefinition.staticDef(cfm, col.getKey().bytes, col.getValue(), componentIndex)
++                           : ColumnDefinition.regularDef(cfm, col.getKey().bytes, col.getValue(), componentIndex));
+         }
  
          return columnDefs;
      }
@@@ -185,13 -201,13 +193,13 @@@
  
              properties.validate();
  
-             CreateTableStatement stmt = new CreateTableStatement(cfName, properties, ifNotExists);
+             CreateTableStatement stmt = new CreateTableStatement(cfName, properties, ifNotExists, staticColumns);
  
              Map<ByteBuffer, CollectionType> definedCollections = null;
 -            for (Map.Entry<ColumnIdentifier, CQL3Type> entry : definitions.entrySet())
 +            for (Map.Entry<ColumnIdentifier, CQL3Type.Raw> entry : definitions.entrySet())
              {
                  ColumnIdentifier id = entry.getKey();
 -                CQL3Type pt = entry.getValue();
 +                CQL3Type pt = entry.getValue().prepare(keyspace());
                  if (pt.isCollection())
                  {
                      if (definedCollections == null)
@@@ -248,11 -268,13 +258,16 @@@
                  {
                      if (definedCollections != null)
                          throw new InvalidRequestException("Collection types are not supported with COMPACT STORAGE");
-                     stmt.columnAliases.add(columnAliases.get(0).bytes);
-                     AbstractType<?> at = getTypeAndRemove(stmt.columns, columnAliases.get(0));
++
+                     ColumnIdentifier alias = columnAliases.get(0);
 -                    stmt.columnAliases.add(alias.key);
 -                    stmt.comparator = getTypeAndRemove(stmt.columns, alias);
 -                    if (stmt.comparator instanceof CounterColumnType)
 -                        throw new InvalidRequestException(String.format("counter type is not supported for PRIMARY KEY part %s", alias));
+                     if (staticColumns.contains(alias))
+                         throw new InvalidRequestException(String.format("Static column %s cannot be part of the PRIMARY KEY", alias));
++
++                    stmt.columnAliases.add(alias.bytes);
++                    AbstractType<?> at = getTypeAndRemove(stmt.columns, alias);
 +                    if (at instanceof CounterColumnType)
 +                        throw new InvalidRequestException(String.format("counter type is not supported for PRIMARY KEY part %s", stmt.columnAliases.get(0)));
 +                    stmt.comparator = new SimpleDenseCellNameType(at);
                  }
                  else
                  {
@@@ -263,7 -285,9 +278,9 @@@
  
                          AbstractType<?> type = getTypeAndRemove(stmt.columns, t);
                          if (type instanceof CounterColumnType)
 -                            throw new InvalidRequestException(String.format("counter type is not supported for PRIMARY KEY part %s", t.key));
 +                            throw new InvalidRequestException(String.format("counter type is not supported for PRIMARY KEY part %s", t));
+                         if (staticColumns.contains(t))
+                             throw new InvalidRequestException(String.format("Static column %s cannot be part of the PRIMARY KEY", t));
                          types.add(type);
                      }
  
@@@ -358,7 -396,7 +385,7 @@@
              return isReversed != null && isReversed ? ReversedType.getInstance(type) : type;
          }
  
-         public void addDefinition(ColumnIdentifier def, CQL3Type.Raw type)
 -        public void addDefinition(ColumnIdentifier def, CQL3Type type, boolean isStatic)
++        public void addDefinition(ColumnIdentifier def, CQL3Type.Raw type, boolean isStatic)
          {
              definedNames.add(def);
              definitions.put(def, type);


[5/7] Merge branch 'cassandra-2.0' into cassandra-2.1

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
index 7112b79,cd5f2a2..ece6916
--- a/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
@@@ -33,9 -31,9 +33,9 @@@ import org.apache.cassandra.utils.Pair
   */
  public class DeleteStatement extends ModificationStatement
  {
-     private DeleteStatement(int boundTerms, CFMetaData cfm, Attributes attrs)
 -    private DeleteStatement(StatementType type, CFMetaData cfm, Attributes attrs)
++    private DeleteStatement(StatementType type, int boundTerms, CFMetaData cfm, Attributes attrs)
      {
-         super(boundTerms, cfm, attrs);
 -        super(type, cfm, attrs);
++        super(type, boundTerms, cfm, attrs);
      }
  
      public boolean requireFullClusteringKey()
@@@ -43,42 -41,63 +43,45 @@@
          return false;
      }
  
-     public ColumnFamily updateForKey(ByteBuffer key, Composite prefix, UpdateParameters params)
 -    public ColumnFamily updateForKey(ByteBuffer key, ColumnNameBuilder builder, UpdateParameters params)
++    public void addUpdateForKey(ColumnFamily cf, ByteBuffer key, Composite prefix, UpdateParameters params)
      throws InvalidRequestException
      {
-         ColumnFamily cf = ArrayBackedSortedColumns.factory.create(cfm);
 -        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(cfm);
 -        addUpdateForKey(cf, key, builder, params);
 -        return cf;
 -    }
 -
 -    public void addUpdateForKey(ColumnFamily cf, ByteBuffer key, ColumnNameBuilder builder, UpdateParameters params)
 -    throws InvalidRequestException
 -    {
 -        CFDefinition cfDef = cfm.getCfDef();
          List<Operation> deletions = getOperations();
  
 -        boolean fullKey = builder.componentCount() == cfDef.clusteringColumnsCount();
 -        boolean isRange = cfDef.isCompact ? !fullKey : (!fullKey || deletions.isEmpty());
 -
 -        if (!deletions.isEmpty() && isRange)
 +        if (prefix.size() < cfm.clusteringColumns().size() && !deletions.isEmpty())
-             throw new InvalidRequestException(String.format("Missing mandatory PRIMARY KEY part %s since %s specified", getFirstEmptyKey(), deletions.iterator().next().column.name));
+         {
 -            // We only get there if we have at least one non-static columns selected, as otherwise the builder will be
 -            // the "static" builder and isRange will be false. But we may still have static columns, so pick the first
 -            // non static one for the error message so it's not confusing
++            // In general, we can't delete specific columns if not all clustering columns have been specified.
++            // However, if we delete only static colums, it's fine since we won't really use the prefix anyway.
+             for (Operation deletion : deletions)
 -                if (cfm.getCfDef().get(deletion.columnName).kind != CFDefinition.Name.Kind.STATIC)
 -                    throw new InvalidRequestException(String.format("Missing mandatory PRIMARY KEY part %s since %s specified", getFirstEmptyKey(), deletion.columnName));
 -            throw new AssertionError();
++                if (!deletion.column.isStatic())
++                    throw new InvalidRequestException(String.format("Missing mandatory PRIMARY KEY part %s since %s specified", getFirstEmptyKey(), deletion.column.name));
+         }
  
 -        if (deletions.isEmpty() && builder.componentCount() == 0)
 +        if (deletions.isEmpty())
          {
 -            // No columns specified, delete the row
 -            cf.delete(new DeletionInfo(params.timestamp, params.localDeletionTime));
 -        }
 -        else
 -        {
 -            if (isRange)
 +            // We delete the slice selected by the prefix.
 +            // However, for performance reasons, we distinguish 2 cases:
 +            //   - It's a full internal row delete
 +            //   - It's a full cell name (i.e it's a dense layout and the prefix is full)
 +            if (prefix.isEmpty())
 +            {
 +                // No columns specified, delete the row
 +                cf.delete(new DeletionInfo(params.timestamp, params.localDeletionTime));
 +            }
 +            else if (cfm.comparator.isDense() && prefix.size() == cfm.clusteringColumns().size())
              {
 -                assert deletions.isEmpty();
 -                ByteBuffer start = builder.build();
 -                ByteBuffer end = builder.buildAsEndOfRange();
 -                cf.addAtom(params.makeRangeTombstone(start, end));
 +                cf.addAtom(params.makeTombstone(cfm.comparator.create(prefix, null)));
              }
              else
              {
 -                // Delete specific columns
 -                if (cfDef.isCompact)
 -                {
 -                    ByteBuffer columnName = builder.build();
 -                    cf.addColumn(params.makeTombstone(columnName));
 -                }
 -                else
 -                {
 -                    for (Operation deletion : deletions)
 -                        deletion.execute(key, cf, builder.copy(), params);
 -                }
 +                cf.addAtom(params.makeRangeTombstone(prefix.slice()));
              }
          }
 +        else
 +        {
 +            for (Operation op : deletions)
 +                op.execute(key, cf, prefix, params);
 +        }
- 
-         return cf;
      }
  
      public static class Parsed extends ModificationStatement.Parsed
@@@ -97,9 -116,9 +100,9 @@@
              this.whereClause = whereClause;
          }
  
 -        protected ModificationStatement prepareInternal(CFDefinition cfDef, VariableSpecifications boundNames, Attributes attrs) throws InvalidRequestException
 +        protected ModificationStatement prepareInternal(CFMetaData cfm, VariableSpecifications boundNames, Attributes attrs) throws InvalidRequestException
          {
-             DeleteStatement stmt = new DeleteStatement(boundNames.size(), cfm, attrs);
 -            DeleteStatement stmt = new DeleteStatement(ModificationStatement.StatementType.DELETE, cfDef.cfm, attrs);
++            DeleteStatement stmt = new DeleteStatement(ModificationStatement.StatementType.DELETE, boundNames.size(), cfm, attrs);
  
              for (Operation.RawDeletion deletion : deletions)
              {
@@@ -109,10 -128,10 +112,10 @@@
  
                  // For compact, we only have one value except the key, so the only form of DELETE that make sense is without a column
                  // list. However, we support having the value name for coherence with the static/sparse case
-                 if (def.kind != ColumnDefinition.Kind.REGULAR && def.kind != ColumnDefinition.Kind.COMPACT_VALUE)
 -                if (name.isPrimaryKeyColumn())
 -                    throw new InvalidRequestException(String.format("Invalid identifier %s for deletion (should not be a PRIMARY KEY part)", name));
++                if (def.isPrimaryKeyColumn())
 +                    throw new InvalidRequestException(String.format("Invalid identifier %s for deletion (should not be a PRIMARY KEY part)", def.name));
  
 -                Operation op = deletion.prepare(name);
 +                Operation op = deletion.prepare(cfm.ksName, def);
                  op.collectMarkerSpecification(boundNames);
                  stmt.addOperation(op);
              }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index 6d18f1b,ac8d2e1..db0b7a9
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@@ -20,7 -20,11 +20,9 @@@ package org.apache.cassandra.cql3.state
  import java.nio.ByteBuffer;
  import java.util.*;
  
+ import com.google.common.base.Function;
+ import com.google.common.collect.Iterables;
  import org.github.jamm.MemoryMeter;
 -import org.slf4j.Logger;
 -import org.slf4j.LoggerFactory;
  
  import org.apache.cassandra.auth.Permission;
  import org.apache.cassandra.config.CFMetaData;
@@@ -50,19 -54,40 +52,36 @@@ public abstract class ModificationState
  {
      private static final ColumnIdentifier CAS_RESULT_COLUMN = new ColumnIdentifier("[applied]", false);
  
 -    private static final Logger logger = LoggerFactory.getLogger(ModificationStatement.class);
 -
 -    private static boolean loggedCounterTTL = false;
 -    private static boolean loggedCounterTimestamp = false;
 -
+     public static enum StatementType { INSERT, UPDATE, DELETE }
+     public final StatementType type;
+ 
 +    private final int boundTerms;
      public final CFMetaData cfm;
      public final Attributes attrs;
  
      private final Map<ColumnIdentifier, Restriction> processedKeys = new HashMap<ColumnIdentifier, Restriction>();
      private final List<Operation> columnOperations = new ArrayList<Operation>();
  
-     private List<Operation> columnConditions;
 -    private int boundTerms;
+     // Separating normal and static conditions makes things somewhat easier
+     private List<ColumnCondition> columnConditions;
+     private List<ColumnCondition> staticConditions;
      private boolean ifNotExists;
  
-     public ModificationStatement(int boundTerms, CFMetaData cfm, Attributes attrs)
+     private boolean hasNoClusteringColumns = true;
+     private boolean setsOnlyStaticColumns;
+ 
 -    private final Function<ColumnCondition, ColumnIdentifier> getColumnForCondition = new Function<ColumnCondition, ColumnIdentifier>()
++    private final Function<ColumnCondition, ColumnDefinition> getColumnForCondition = new Function<ColumnCondition, ColumnDefinition>()
+     {
 -        public ColumnIdentifier apply(ColumnCondition cond)
++        public ColumnDefinition apply(ColumnCondition cond)
+         {
 -            return cond.column.name;
++            return cond.column;
+         }
+     };
+ 
 -    public ModificationStatement(StatementType type, CFMetaData cfm, Attributes attrs)
++    public ModificationStatement(StatementType type, int boundTerms, CFMetaData cfm, Attributes attrs)
      {
+         this.type = type;
 +        this.boundTerms = boundTerms;
          this.cfm = cfm;
          this.attrs = attrs;
      }
@@@ -77,7 -103,7 +97,7 @@@
      }
  
      public abstract boolean requireFullClusteringKey();
-     public abstract ColumnFamily updateForKey(ByteBuffer key, Composite prefix, UpdateParameters params) throws InvalidRequestException;
 -    public abstract void addUpdateForKey(ColumnFamily updates, ByteBuffer key, ColumnNameBuilder builder, UpdateParameters params) throws InvalidRequestException;
++    public abstract void addUpdateForKey(ColumnFamily updates, ByteBuffer key, Composite prefix, UpdateParameters params) throws InvalidRequestException;
  
      public int getBoundTerms()
      {
@@@ -137,6 -176,15 +157,15 @@@
  
      public void addOperation(Operation op)
      {
 -        if (op.isStatic(cfm))
++        if (op.column.isStatic())
+         {
+             if (columnOperations.isEmpty())
+                 setsOnlyStaticColumns = true;
+         }
+         else
+         {
+             setsOnlyStaticColumns = false;
+         }
          columnOperations.add(op);
      }
  
@@@ -145,12 -193,31 +174,31 @@@
          return columnOperations;
      }
  
-     public void addCondition(Operation op)
 -    public Iterable<ColumnIdentifier> getColumnsWithConditions()
++    public Iterable<ColumnDefinition> getColumnsWithConditions()
      {
-         if (columnConditions == null)
-             columnConditions = new ArrayList<Operation>();
+         if (ifNotExists)
+             return null;
  
-         columnConditions.add(op);
 -        return Iterables.concat(columnConditions == null ? Collections.<ColumnIdentifier>emptyList() : Iterables.transform(columnConditions, getColumnForCondition),
 -                                staticConditions == null ? Collections.<ColumnIdentifier>emptyList() : Iterables.transform(staticConditions, getColumnForCondition));
++        return Iterables.concat(columnConditions == null ? Collections.<ColumnDefinition>emptyList() : Iterables.transform(columnConditions, getColumnForCondition),
++                                staticConditions == null ? Collections.<ColumnDefinition>emptyList() : Iterables.transform(staticConditions, getColumnForCondition));
+     }
+ 
+     public void addCondition(ColumnCondition cond) throws InvalidRequestException
+     {
+         List<ColumnCondition> conds = null;
 -        if (cond.column.kind == CFDefinition.Name.Kind.STATIC)
++        if (cond.column.isStatic())
+         {
+             if (staticConditions == null)
+                 staticConditions = new ArrayList<ColumnCondition>();
+             conds = staticConditions;
+         }
+         else
+         {
+             if (columnConditions == null)
+                 columnConditions = new ArrayList<ColumnCondition>();
+             conds = columnConditions;
+         }
+         conds.add(cond);
      }
  
      public void setIfNotExistCondition()
@@@ -158,15 -225,22 +206,22 @@@
          ifNotExists = true;
      }
  
-     private void addKeyValues(ColumnIdentifier name, Restriction values) throws InvalidRequestException
+     public boolean hasIfNotExistCondition()
+     {
+         return ifNotExists;
+     }
+ 
 -    private void addKeyValues(CFDefinition.Name name, Restriction values) throws InvalidRequestException
++    private void addKeyValues(ColumnDefinition def, Restriction values) throws InvalidRequestException
      {
-         if (processedKeys.put(name, values) != null)
-             throw new InvalidRequestException(String.format("Multiple definitions found for PRIMARY KEY part %s", name));
 -        if (name.kind == CFDefinition.Name.Kind.COLUMN_ALIAS)
++        if (def.kind == ColumnDefinition.Kind.CLUSTERING_COLUMN)
+             hasNoClusteringColumns = false;
 -        if (processedKeys.put(name.name, values) != null)
 -            throw new InvalidRequestException(String.format("Multiple definitions found for PRIMARY KEY part %s", name.name));
++        if (processedKeys.put(def.name, values) != null)
++            throw new InvalidRequestException(String.format("Multiple definitions found for PRIMARY KEY part %s", def.name));
      }
  
-     public void addKeyValue(ColumnIdentifier name, Term value) throws InvalidRequestException
 -    public void addKeyValue(CFDefinition.Name name, Term value) throws InvalidRequestException
++    public void addKeyValue(ColumnDefinition def, Term value) throws InvalidRequestException
      {
--        addKeyValues(name, new Restriction.EQ(value, false));
++        addKeyValues(def, new Restriction.EQ(value, false));
      }
  
      public void processWhereClause(List<Relation> whereClause, VariableSpecifications names) throws InvalidRequestException
@@@ -211,14 -286,15 +266,13 @@@
                      }
                      else
                      {
 -                        throw new InvalidRequestException(String.format("Invalid operator %s for PRIMARY KEY part %s", rel.operator(), name));
 +                        throw new InvalidRequestException(String.format("Invalid operator %s for PRIMARY KEY part %s", rel.operator(), def.name));
                      }
  
-                     addKeyValues(def.name, restriction);
 -                    addKeyValues(name, restriction);
++                    addKeyValues(def, restriction);
                      break;
-                 case COMPACT_VALUE:
-                 case REGULAR:
 -                case VALUE_ALIAS:
 -                case COLUMN_METADATA:
 -                case STATIC:
 -                    throw new InvalidRequestException(String.format("Non PRIMARY KEY %s found in where clause", name));
++                default:
 +                    throw new InvalidRequestException(String.format("Non PRIMARY KEY %s found in where clause", def.name));
              }
          }
      }
@@@ -241,8 -318,10 +295,10 @@@
                  for (ByteBuffer val : values)
                  {
                      if (val == null)
 -                        throw new InvalidRequestException(String.format("Invalid null value for partition key part %s", name));
 -                    ByteBuffer key = keyBuilder.copy().add(val).build();
 +                        throw new InvalidRequestException(String.format("Invalid null value for partition key part %s", def.name));
-                     keys.add(keyBuilder.buildWith(val).toByteBuffer());
++                    ByteBuffer key = keyBuilder.buildWith(val).toByteBuffer();
+                     ThriftValidation.validateKey(cfm, key);
+                     keys.add(key);
                  }
              }
              else
@@@ -258,19 -337,56 +314,44 @@@
          return keys;
      }
  
 -    public ColumnNameBuilder createClusteringPrefixBuilder(List<ByteBuffer> variables)
 +    public Composite createClusteringPrefix(List<ByteBuffer> variables)
      throws InvalidRequestException
      {
+         // If the only updated/deleted columns are static, then we don't need clustering columns.
+         // And in fact, unless it is an INSERT, we reject if clustering colums are provided as that
+         // suggest something unintended. For instance, given:
+         //   CREATE TABLE t (k int, v int, s int static, PRIMARY KEY (k, v))
+         // it can make sense to do:
+         //   INSERT INTO t(k, v, s) VALUES (0, 1, 2)
+         // but both
+         //   UPDATE t SET s = 3 WHERE k = 0 AND v = 1
+         //   DELETE v FROM t WHERE k = 0 AND v = 1
+         // sounds like you don't really understand what your are doing.
+         if (setsOnlyStaticColumns && columnConditions == null && (type != StatementType.INSERT || hasNoClusteringColumns))
+         {
+             // Reject if any clustering columns is set
 -            for (CFDefinition.Name name : cfm.getCfDef().clusteringColumns())
 -                if (processedKeys.get(name.name) != null)
 -                    throw new InvalidRequestException(String.format("Invalid restriction on clustering column %s since the %s statement modifies only static columns", name.name, type));
 -            return cfm.getStaticColumnNameBuilder();
++            for (ColumnDefinition def : cfm.clusteringColumns())
++                if (processedKeys.get(def.name) != null)
++                    throw new InvalidRequestException(String.format("Invalid restriction on clustering column %s since the %s statement modifies only static columns", def.name, type));
++            return cfm.comparator.staticPrefix();
+         }
+ 
+         return createClusteringPrefixBuilderInternal(variables);
+     }
+ 
 -    private ColumnNameBuilder updatePrefixFor(ByteBuffer name, ColumnNameBuilder prefix)
 -    {
 -        return isStatic(name) ? cfm.getStaticColumnNameBuilder() : prefix;
 -    }
 -
 -    public boolean isStatic(ByteBuffer name)
 -    {
 -        ColumnDefinition def = cfm.getColumnDefinition(name);
 -        return def != null && def.type == ColumnDefinition.Type.STATIC;
 -    }
 -
 -    private ColumnNameBuilder createClusteringPrefixBuilderInternal(List<ByteBuffer> variables)
++    private Composite createClusteringPrefixBuilderInternal(List<ByteBuffer> variables)
+     throws InvalidRequestException
+     {
 -        CFDefinition cfDef = cfm.getCfDef();
 -        ColumnNameBuilder builder = cfDef.getColumnNameBuilder();
 -        CFDefinition.Name firstEmptyKey = null;
 -        for (CFDefinition.Name name : cfDef.clusteringColumns())
 +        CBuilder builder = cfm.comparator.prefixBuilder();
 +        ColumnDefinition firstEmptyKey = null;
 +        for (ColumnDefinition def : cfm.clusteringColumns())
          {
 -            Restriction r = processedKeys.get(name.name);
 +            Restriction r = processedKeys.get(def.name);
              if (r == null)
              {
 -                firstEmptyKey = name;
 -                if (requireFullClusteringKey() && cfDef.isComposite && !cfDef.isCompact)
 -                    throw new InvalidRequestException(String.format("Missing mandatory PRIMARY KEY part %s", name));
 +                firstEmptyKey = def;
 +                if (requireFullClusteringKey() && !cfm.comparator.isDense() && cfm.comparator.isCompound())
 +                    throw new InvalidRequestException(String.format("Missing mandatory PRIMARY KEY part %s", def.name));
              }
              else if (firstEmptyKey != null)
              {
@@@ -303,21 -419,21 +384,20 @@@
      throws RequestExecutionException, RequestValidationException
      {
          // Lists SET operation incurs a read.
-         Set<ColumnIdentifier> toRead = null;
 -        Set<ByteBuffer> toRead = null;
++        boolean requiresRead = false;
          for (Operation op : columnOperations)
          {
              if (op.requiresRead())
              {
--                if (toRead == null)
-                     toRead = new TreeSet<ColumnIdentifier>();
-                 toRead.add(op.column.name);
 -                    toRead = new TreeSet<ByteBuffer>(UTF8Type.instance);
 -                toRead.add(op.columnName.key);
++                requiresRead = true;
++                break;
              }
          }
  
-         return toRead == null ? null : readRows(partitionKeys, clusteringPrefix, toRead, cfm, local, cl);
 -        return toRead == null ? null : readRows(partitionKeys, clusteringPrefix, toRead, (CompositeType)cfm.comparator, local, cl);
++        return requiresRead ? readRows(partitionKeys, clusteringPrefix, cfm, local, cl) : null;
      }
  
-     protected Map<ByteBuffer, CQL3Row> readRows(List<ByteBuffer> partitionKeys, Composite rowPrefix, Set<ColumnIdentifier> toRead, CFMetaData cfm, boolean local, ConsistencyLevel cl)
 -    private Map<ByteBuffer, ColumnGroupMap> readRows(List<ByteBuffer> partitionKeys, ColumnNameBuilder clusteringPrefix, Set<ByteBuffer> toRead, CompositeType composite, boolean local, ConsistencyLevel cl)
++    protected Map<ByteBuffer, CQL3Row> readRows(List<ByteBuffer> partitionKeys, Composite rowPrefix, CFMetaData cfm, boolean local, ConsistencyLevel cl)
      throws RequestExecutionException, RequestValidationException
      {
          try
@@@ -329,11 -445,16 +409,7 @@@
              throw new InvalidRequestException(String.format("Write operation require a read but consistency %s is not supported on reads", cl));
          }
  
--        ColumnSlice[] slices = new ColumnSlice[toRead.size()];
--        int i = 0;
-         for (ColumnIdentifier name : toRead)
-             slices[i++] = cfm.comparator.create(rowPrefix, name).slice();
- 
 -        for (ByteBuffer name : toRead)
 -        {
 -            ColumnNameBuilder prefix = updatePrefixFor(name, clusteringPrefix);
 -            ByteBuffer start = prefix.copy().add(name).build();
 -            ByteBuffer finish = prefix.copy().add(name).buildAsEndOfRange();
 -            slices[i++] = new ColumnSlice(start, finish);
 -        }
 -
++        ColumnSlice[] slices = new ColumnSlice[]{ rowPrefix.slice() };
          List<ReadCommand> commands = new ArrayList<ReadCommand>(partitionKeys.size());
          long now = System.currentTimeMillis();
          for (ByteBuffer key : partitionKeys)
@@@ -414,14 -536,10 +490,10 @@@
  
          // It's cleaner to use the query timestamp below, but it's in seconds while the conditions expects microseconds, so just
          // put it back in millis (we don't really lose precision because the ultimate consumer, Column.isLive, re-divide it).
-         long now = queryState.getTimestamp() * 1000;
-         CASConditions conditions = ifNotExists
-                                  ? new NotExistCondition(clusteringPrefix, now)
-                                  : new ColumnsConditions(clusteringPrefix, cfm, key, columnConditions, variables, now);
+         CQL3CasConditions conditions = new CQL3CasConditions(cfm, queryState.getTimestamp() * 1000);
 -        ColumnNameBuilder prefix = createClusteringPrefixBuilder(variables);
 -        ColumnFamily updates = UnsortedColumns.factory.create(cfm);
++        Composite prefix = createClusteringPrefix(variables);
++        ColumnFamily updates = ArrayBackedSortedColumns.factory.create(cfm);
+         addUpdatesAndConditions(key, prefix, updates, conditions, variables, getTimestamp(queryState.getTimestamp(), variables));
  
          ColumnFamily result = StorageProxy.cas(keyspace(),
                                                 columnFamily(),
@@@ -433,11 -551,39 +505,39 @@@
          return new ResultMessage.Rows(buildCasResultSet(key, result));
      }
  
 -    public void addUpdatesAndConditions(ByteBuffer key, ColumnNameBuilder clusteringPrefix, ColumnFamily updates, CQL3CasConditions conditions, List<ByteBuffer> variables, long now)
++    public void addUpdatesAndConditions(ByteBuffer key, Composite clusteringPrefix, ColumnFamily updates, CQL3CasConditions conditions, List<ByteBuffer> variables, long now)
+     throws InvalidRequestException
+     {
+         UpdateParameters updParams = new UpdateParameters(cfm, variables, now, getTimeToLive(variables), null);
+         addUpdateForKey(updates, key, clusteringPrefix, updParams);
+ 
+         if (ifNotExists)
+         {
+             // If we use ifNotExists, if the statement applies to any non static columns, then the condition is on the row of the non-static
 -            // columns and the prefix should be the rowPrefix. But if only static columns are set, then the ifNotExists apply to the existence
++            // columns and the prefix should be the clusteringPrefix. But if only static columns are set, then the ifNotExists apply to the existence
+             // of any static columns and we should use the prefix for the "static part" of the partition.
 -            conditions.addNotExist(setsOnlyStaticColumns ? cfm.getStaticColumnNameBuilder() : clusteringPrefix);
++            conditions.addNotExist(setsOnlyStaticColumns ? cfm.comparator.staticPrefix() : clusteringPrefix);
+         }
+         else
+         {
+             if (columnConditions != null)
+                 conditions.addConditions(clusteringPrefix, columnConditions, variables);
+             if (staticConditions != null)
 -                conditions.addConditions(cfm.getStaticColumnNameBuilder(), staticConditions, variables);
++                conditions.addConditions(cfm.comparator.staticPrefix(), staticConditions, variables);
+         }
+     }
+ 
      private ResultSet buildCasResultSet(ByteBuffer key, ColumnFamily cf) throws InvalidRequestException
      {
+         return buildCasResultSet(keyspace(), key, columnFamily(), cf, getColumnsWithConditions(), false);
+     }
+ 
 -    public static ResultSet buildCasResultSet(String ksName, ByteBuffer key, String cfName, ColumnFamily cf, Iterable<ColumnIdentifier> columnsWithConditions, boolean isBatch)
++    public static ResultSet buildCasResultSet(String ksName, ByteBuffer key, String cfName, ColumnFamily cf, Iterable<ColumnDefinition> columnsWithConditions, boolean isBatch)
+     throws InvalidRequestException
+     {
          boolean success = cf == null;
  
-         ColumnSpecification spec = new ColumnSpecification(keyspace(), columnFamily(), CAS_RESULT_COLUMN, BooleanType.instance);
+         ColumnSpecification spec = new ColumnSpecification(ksName, cfName, CAS_RESULT_COLUMN, BooleanType.instance);
          ResultSet.Metadata metadata = new ResultSet.Metadata(Collections.singletonList(spec));
          List<List<ByteBuffer>> rows = Collections.singletonList(Collections.singletonList(BooleanType.instance.decompose(success)));
  
@@@ -457,25 -603,40 +557,39 @@@
          List<ColumnSpecification> specs = new ArrayList<ColumnSpecification>(size);
          specs.addAll(left.metadata.names);
          specs.addAll(right.metadata.names);
-         List<ByteBuffer> row = new ArrayList<ByteBuffer>(size);
-         row.addAll(left.rows.get(0));
-         row.addAll(right.rows.get(0));
-         return new ResultSet(new ResultSet.Metadata(specs), Collections.singletonList(row));
+         List<List<ByteBuffer>> rows = new ArrayList<>(right.size());
+         for (int i = 0; i < right.size(); i++)
+         {
+             List<ByteBuffer> row = new ArrayList<ByteBuffer>(size);
+             row.addAll(left.rows.get(0));
+             row.addAll(right.rows.get(i));
+             rows.add(row);
+         }
+         return new ResultSet(new ResultSet.Metadata(specs), rows);
      }
  
-     private ResultSet buildCasFailureResultSet(ByteBuffer key, ColumnFamily cf) throws InvalidRequestException
 -    private static ResultSet buildCasFailureResultSet(ByteBuffer key, ColumnFamily cf, Iterable<ColumnIdentifier> columnsWithConditions, boolean isBatch)
++    private static ResultSet buildCasFailureResultSet(ByteBuffer key, ColumnFamily cf, Iterable<ColumnDefinition> columnsWithConditions, boolean isBatch)
+     throws InvalidRequestException
      {
 -        CFDefinition cfDef = cf.metadata().getCfDef();
 -
++        CFMetaData cfm = cf.metadata();
          Selection selection;
-         if (ifNotExists)
+         if (columnsWithConditions == null)
          {
 -            selection = Selection.wildcard(cfDef);
 +            selection = Selection.wildcard(cfm);
          }
          else
          {
-             List<ColumnDefinition> defs = new ArrayList<>(columnConditions.size());
-             for (Operation condition : columnConditions)
-                 defs.add(condition.column);
 -            List<CFDefinition.Name> names = new ArrayList<CFDefinition.Name>();
++            List<ColumnDefinition> defs = new ArrayList<>();
+             // Adding the partition key for batches to disambiguate if the conditions span multipe rows (we don't add them outside
+             // of batches for compatibility sakes).
+             if (isBatch)
+             {
 -                names.addAll(cfDef.partitionKeys());
 -                names.addAll(cfDef.clusteringColumns());
++                defs.addAll(cfm.partitionKeyColumns());
++                defs.addAll(cfm.clusteringColumns());
+             }
 -            for (ColumnIdentifier id : columnsWithConditions)
 -                names.add(cfDef.get(id));
 -            selection = Selection.forColumns(names);
++            for (ColumnDefinition def : columnsWithConditions)
++                defs.add(def);
 +            selection = Selection.forColumns(defs);
          }
  
          long now = System.currentTimeMillis();
@@@ -524,7 -681,8 +638,8 @@@
          for (ByteBuffer key: keys)
          {
              ThriftValidation.validateKey(cfm, key);
-             ColumnFamily cf = updateForKey(key, clusteringPrefix, params);
 -            ColumnFamily cf = UnsortedColumns.factory.create(cfm);
++            ColumnFamily cf = ArrayBackedSortedColumns.factory.create(cfm);
+             addUpdateForKey(cf, key, clusteringPrefix, params);
              mutations.add(makeMutation(key, cf, cl, isBatch));
          }
          return mutations;
@@@ -541,98 -699,11 +656,11 @@@
          }
          else
          {
 -            rm = new RowMutation(cfm.ksName, key, cf);
 +            mutation = new Mutation(cfm.ksName, key, cf);
          }
 -        return isCounter() ? new CounterMutation(rm, cl) : rm;
 +        return isCounter() ? new CounterMutation(mutation, cl) : mutation;
      }
  
-     private static abstract class CQL3CasConditions implements CASConditions
-     {
-         protected final Composite rowPrefix;
-         protected final long now;
- 
-         protected CQL3CasConditions(Composite rowPrefix, long now)
-         {
-             this.rowPrefix = rowPrefix;
-             this.now = now;
-         }
- 
-         public IDiskAtomFilter readFilter()
-         {
-             // We always read the row entirely as on CAS failure we want to be able to distinguish between "row exists
-             // but all values on why there were conditions are null" and "row doesn't exists", and we can't rely on the
-             // row marker for that (see #6623)
-             return new SliceQueryFilter(rowPrefix.slice(), false, 1, rowPrefix.size());
-         }
-     }
- 
-     private static class NotExistCondition extends CQL3CasConditions
-     {
-         private NotExistCondition(Composite rowPrefix, long now)
-         {
-             super(rowPrefix, now);
-         }
- 
-         public boolean appliesTo(ColumnFamily current)
-         {
-             return current == null || current.hasOnlyTombstones(now);
-         }
-     }
- 
-     private static class ColumnsConditions extends CQL3CasConditions
-     {
-         private final ColumnFamily expected;
- 
-         private ColumnsConditions(Composite rowPrefix,
-                                   CFMetaData cfm,
-                                   ByteBuffer key,
-                                   Collection<Operation> conditions,
-                                   List<ByteBuffer> variables,
-                                   long now) throws InvalidRequestException
-         {
-             super(rowPrefix, now);
-             this.expected = ArrayBackedSortedColumns.factory.create(cfm);
- 
-             // When building the conditions, we should not use a TTL. It's not useful, and if a very low ttl (1 seconds) is used, it's possible
-             // for it to expire before the actual build of the conditions which would break since we would then testing for the presence of tombstones.
-             UpdateParameters params = new UpdateParameters(cfm, variables, now, 0, null);
- 
-             // Conditions
-             for (Operation condition : conditions)
-                 condition.execute(key, expected, rowPrefix, params);
-         }
- 
-         public boolean appliesTo(ColumnFamily current)
-         {
-             if (current == null)
-                 return false;
- 
-             for (Cell e : expected)
-             {
-                 Cell c = current.getColumn(e.name());
-                 if (e.isLive(now))
-                 {
-                     if (c == null || !c.isLive(now) || !c.value().equals(e.value()))
-                         return false;
-                 }
-                 else
-                 {
-                     // If we have a tombstone in expected, it means the condition tests that the column is
-                     // null, so check that we have no value
-                     if (c != null && c.isLive(now))
-                         return false;
-                 }
-             }
-             return true;
-         }
- 
-         @Override
-         public String toString()
-         {
-             return expected.toString();
-         }
-     }
- 
      public static abstract class Parsed extends CFStatement
      {
          protected final Attributes.Raw attrs;
@@@ -680,33 -756,23 +708,21 @@@
                  }
                  else
                  {
-                     for (Pair<ColumnIdentifier, Operation.RawUpdate> entry : conditions)
+                     for (Pair<ColumnIdentifier, ColumnCondition.Raw> entry : conditions)
                      {
 -                        CFDefinition.Name name = cfDef.get(entry.left);
 -                        if (name == null)
 +                        ColumnDefinition def = metadata.getColumnDefinition(entry.left);
 +                        if (def == null)
                              throw new InvalidRequestException(String.format("Unknown identifier %s", entry.left));
  
-                         /*
-                          * Lists column names are based on a server-side generated timeuuid. So we can't allow lists
-                          * operation or that would yield unexpected results (update that should apply wouldn't). So for
-                          * now, we just refuse lists, which also save use from having to bother about the read that some
-                          * list operation involve.
-                          */
-                         if (def.type instanceof ListType)
-                             throw new InvalidRequestException(String.format("List operation (%s) are not allowed in conditional updates", def.name));
- 
-                         Operation condition = entry.right.prepare(keyspace(), def);
-                         assert !condition.requiresRead();
- 
 -                        ColumnCondition condition = entry.right.prepare(name);
++                        ColumnCondition condition = entry.right.prepare(keyspace(), def);
                          condition.collectMarkerSpecification(boundNames);
  
 -                        switch (name.kind)
 +                        switch (def.kind)
                          {
 -                            case KEY_ALIAS:
 -                            case COLUMN_ALIAS:
 +                            case PARTITION_KEY:
 +                            case CLUSTERING_COLUMN:
                                  throw new InvalidRequestException(String.format("PRIMARY KEY part %s found in SET part", entry.left));
-                             case COMPACT_VALUE:
-                             case REGULAR:
 -                            case VALUE_ALIAS:
 -                            case COLUMN_METADATA:
 -                            case STATIC:
++                            default:
                                  stmt.addCondition(condition);
                                  break;
                          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index e08b960,2636c83..868d51c
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -78,22 -80,60 +78,61 @@@ public class SelectStatement implement
      private boolean isKeyRange;
      private boolean keyIsInRelation;
      private boolean usesSecondaryIndexing;
 +    private boolean needOrderOnLastClustering;
  
 -    private Map<CFDefinition.Name, Integer> orderingIndexes;
 +    private Map<ColumnIdentifier, Integer> orderingIndexes;
  
+     private boolean selectsStaticColumns;
+     private boolean selectsOnlyStaticColumns;
+ 
      // Used by forSelection below
      private static final Parameters defaultParameters = new Parameters(Collections.<ColumnIdentifier, Boolean>emptyMap(), false, false, null, false);
  
 -    private static final Predicate<CFDefinition.Name> isStaticFilter = new Predicate<CFDefinition.Name>()
++    private static final Predicate<ColumnDefinition> isStaticFilter = new Predicate<ColumnDefinition>()
+     {
 -        public boolean apply(CFDefinition.Name name)
++        public boolean apply(ColumnDefinition def)
+         {
 -            return name.kind == CFDefinition.Name.Kind.STATIC;
++            return def.isStatic();
+         }
+     };
+ 
 -    public SelectStatement(CFDefinition cfDef, int boundTerms, Parameters parameters, Selection selection, Term limit)
 +    public SelectStatement(CFMetaData cfm, int boundTerms, Parameters parameters, Selection selection, Term limit)
      {
 -        this.cfDef = cfDef;
 +        this.cfm = cfm;
          this.boundTerms = boundTerms;
          this.selection = selection;
 -        this.keyRestrictions = new Restriction[cfDef.partitionKeyCount()];
 -        this.columnRestrictions = new Restriction[cfDef.clusteringColumnsCount()];
 +        this.keyRestrictions = new Restriction[cfm.partitionKeyColumns().size()];
 +        this.columnRestrictions = new Restriction[cfm.clusteringColumns().size()];
          this.parameters = parameters;
          this.limit = limit;
+ 
+         // Now gather a few info on whether we should bother with static columns or not for this statement
+         initStaticColumnsInfo();
+     }
+ 
+     private void initStaticColumnsInfo()
+     {
 -        if (!cfDef.cfm.hasStaticColumns())
++        if (!cfm.hasStaticColumns())
+             return;
+ 
+         // If it's a wildcard, we do select static but not only them
+         if (selection.isWildcard())
+         {
+             selectsStaticColumns = true;
+             return;
+         }
+ 
+         // Otherwise, check the selected columns
+         selectsStaticColumns = !Iterables.isEmpty(Iterables.filter(selection.getColumnsList(), isStaticFilter));
+         selectsOnlyStaticColumns = true;
 -        for (CFDefinition.Name name : selection.getColumnsList())
++        for (ColumnDefinition def : selection.getColumnsList())
+         {
 -            if (name.kind != CFDefinition.Name.Kind.KEY_ALIAS && name.kind != CFDefinition.Name.Kind.STATIC)
++            if (def.kind != ColumnDefinition.Kind.PARTITION_KEY && def.kind != ColumnDefinition.Kind.STATIC)
+             {
+                 selectsOnlyStaticColumns = false;
+                 break;
+             }
+         }
      }
  
      // Creates a simple select based on the given selection.
@@@ -378,35 -418,95 +417,89 @@@
          }
          else if (isColumnRange())
          {
 -            // For sparse, we used to ask for 'defined columns' * 'asked limit' (where defined columns includes the row marker)
 -            // to account for the grouping of columns.
 -            // Since that doesn't work for maps/sets/lists, we now use the compositesToGroup option of SliceQueryFilter.
 -            // But we must preserve backward compatibility too (for mixed version cluster that is).
 -            int toGroup = cfDef.isCompact ? -1 : cfDef.clusteringColumnsCount();
 -            List<ByteBuffer> startBounds = getRequestedBound(Bound.START, variables);
 -            List<ByteBuffer> endBounds = getRequestedBound(Bound.END, variables);
 +            int toGroup = cfm.comparator.isDense() ? -1 : cfm.clusteringColumns().size();
 +            List<Composite> startBounds = getRequestedBound(Bound.START, variables);
 +            List<Composite> endBounds = getRequestedBound(Bound.END, variables);
              assert startBounds.size() == endBounds.size();
  
+             // Handles fetching static columns. Note that for 2i, the filter is just used to restrict
+             // the part of the index to query so adding the static slice would be useless and confusing.
+             // For 2i, static columns are retrieve in CompositesSearcher with each index hit.
+             ColumnSlice staticSlice = null;
+             if (selectsStaticColumns && !usesSecondaryIndexing)
+             {
 -                ColumnNameBuilder staticPrefix = cfDef.cfm.getStaticColumnNameBuilder();
 -                // Note: we could use staticPrefix.build() for the start bound, but EMPTY_BYTE_BUFFER gives us the
++                // Note: we could use staticPrefix.start() for the start bound, but EMPTY gives us the
+                 // same effect while saving a few CPU cycles.
+                 staticSlice = isReversed
 -                            ? new ColumnSlice(staticPrefix.buildAsEndOfRange(), ByteBufferUtil.EMPTY_BYTE_BUFFER)
 -                            : new ColumnSlice(ByteBufferUtil.EMPTY_BYTE_BUFFER, staticPrefix.buildAsEndOfRange());
++                            ? new ColumnSlice(cfm.comparator.staticPrefix().end(), Composites.EMPTY)
++                            : new ColumnSlice(Composites.EMPTY, cfm.comparator.staticPrefix().end());
+ 
+                 // In the case where we only select static columns, we want to really only check the static columns.
+                 // So we return early as the rest of that method would actually make us query everything
+                 if (selectsOnlyStaticColumns)
+                     return sliceFilter(staticSlice, limit, toGroup);
+             }
+ 
              // The case where startBounds == 1 is common enough that it's worth optimizing
              if (startBounds.size() == 1)
              {
                  ColumnSlice slice = new ColumnSlice(startBounds.get(0), endBounds.get(0));
 -                if (slice.isAlwaysEmpty(cfDef.cfm.comparator, isReversed))
 +                if (slice.isAlwaysEmpty(cfm.comparator, isReversed))
-                     return null;
-                 slices = new ColumnSlice[]{slice};
+                     return staticSlice == null ? null : sliceFilter(staticSlice, limit, toGroup);
+ 
+                 return staticSlice == null
+                      ? sliceFilter(slice, limit, toGroup)
 -                     : (slice.includes(cfDef.cfm.comparator, staticSlice.finish) ? sliceFilter(new ColumnSlice(staticSlice.start, slice.finish), limit, toGroup)
 -                                                                                 : sliceFilter(new ColumnSlice[]{ staticSlice, slice }, limit, toGroup));
++                     : (slice.includes(cfm.comparator, staticSlice.finish) ? sliceFilter(new ColumnSlice(staticSlice.start, slice.finish), limit, toGroup)
++                                                                           : sliceFilter(new ColumnSlice[]{ staticSlice, slice }, limit, toGroup));
+             }
+ 
+             List<ColumnSlice> l = new ArrayList<ColumnSlice>(startBounds.size());
+             for (int i = 0; i < startBounds.size(); i++)
+             {
+                 ColumnSlice slice = new ColumnSlice(startBounds.get(i), endBounds.get(i));
 -                if (!slice.isAlwaysEmpty(cfDef.cfm.comparator, isReversed))
++                if (!slice.isAlwaysEmpty(cfm.comparator, isReversed))
+                     l.add(slice);
+             }
+ 
+             if (l.isEmpty())
+                 return staticSlice == null ? null : sliceFilter(staticSlice, limit, toGroup);
+             if (staticSlice == null)
+                 return sliceFilter(l.toArray(new ColumnSlice[l.size()]), limit, toGroup);
+ 
 -            // The slices should not overlap. We know the slices built from startBounds/endBounds don't, but
 -            // if there is a static slice, it could overlap with the 2nd slice. Check for it and correct if
 -            // that's the case
++            // The slices should not overlap. We know the slices built from startBounds/endBounds don't, but if there is
++            // a static slice, it could overlap with the 2nd slice. Check for it and correct if that's the case
+             ColumnSlice[] slices;
+             if (isReversed)
+             {
 -                if (l.get(l.size() - 1).includes(cfDef.cfm.comparator, staticSlice.start))
++                if (l.get(l.size() - 1).includes(cfm.comparator, staticSlice.start))
+                 {
+                     slices = l.toArray(new ColumnSlice[l.size()]);
 -                    slices[slices.length-1] = new ColumnSlice(slices[slices.length-1].start, ByteBufferUtil.EMPTY_BYTE_BUFFER);
++                    slices[slices.length-1] = new ColumnSlice(slices[slices.length-1].start, Composites.EMPTY);
+                 }
+                 else
+                 {
+                     slices = l.toArray(new ColumnSlice[l.size()+1]);
+                     slices[slices.length-1] = staticSlice;
+                 }
              }
              else
              {
-                 List<ColumnSlice> l = new ArrayList<ColumnSlice>(startBounds.size());
-                 for (int i = 0; i < startBounds.size(); i++)
 -                if (l.get(0).includes(cfDef.cfm.comparator, staticSlice.finish))
++                if (l.get(0).includes(cfm.comparator, staticSlice.finish))
                  {
-                     ColumnSlice slice = new ColumnSlice(startBounds.get(i), endBounds.get(i));
-                     if (!slice.isAlwaysEmpty(cfm.comparator, isReversed))
-                         l.add(slice);
+                     slices = new ColumnSlice[l.size()];
 -                    slices[0] = new ColumnSlice(ByteBufferUtil.EMPTY_BYTE_BUFFER, l.get(0).finish);
++                    slices[0] = new ColumnSlice(Composites.EMPTY, l.get(0).finish);
+                     for (int i = 1; i < l.size(); i++)
+                         slices[i] = l.get(i);
+                 }
+                 else
+                 {
+                     slices = new ColumnSlice[l.size()+1];
+                     slices[0] = staticSlice;
+                     for (int i = 0; i < l.size(); i++)
+                         slices[i] = l.get(i);
                  }
-                 if (l.isEmpty())
-                     return null;
-                 slices = l.toArray(new ColumnSlice[l.size()]);
              }
- 
-             return new SliceQueryFilter(slices, isReversed, limit, toGroup);
+             return sliceFilter(slices, limit, toGroup);
          }
          else
          {
@@@ -549,15 -659,17 +652,17 @@@
          return false;
      }
  
 -    private SortedSet<ByteBuffer> getRequestedColumns(List<ByteBuffer> variables) throws InvalidRequestException
 +    private SortedSet<CellName> getRequestedColumns(List<ByteBuffer> variables) throws InvalidRequestException
      {
+         // Note: getRequestedColumns don't handle static columns, but due to CASSANDRA-5762
+         // we always do a slice for CQL3 tables, so it's ok to ignore them here
          assert !isColumnRange();
  
 -        ColumnNameBuilder builder = cfDef.getColumnNameBuilder();
 -        Iterator<CFDefinition.Name> idIter = cfDef.clusteringColumns().iterator();
 +        CBuilder builder = cfm.comparator.prefixBuilder();
 +        Iterator<ColumnDefinition> idIter = cfm.clusteringColumns().iterator();
          for (Restriction r : columnRestrictions)
          {
 -            ColumnIdentifier id = idIter.next().name;
 +            ColumnDefinition def = idIter.next();
              assert r != null && !r.isSlice();
  
              List<ByteBuffer> values = r.values(variables);
@@@ -611,20 -726,26 +716,21 @@@
              // column (for the case where the row exists but has no columns outside the PK)
              // Two exceptions are "static CF" (non-composite non-compact CF) and "super CF"
              // that don't have marker and for which we must query all columns instead
 -            if (cfDef.isComposite && !cfDef.cfm.isSuper())
 +            if (cfm.comparator.isCompound() && !cfm.isSuper())
              {
                  // marker
 -                columns.add(builder.copy().add(ByteBufferUtil.EMPTY_BYTE_BUFFER).build());
 +                columns.add(cfm.comparator.rowMarker(prefix));
  
                  // selected columns
 -                for (ColumnIdentifier id : selection.regularAndStaticColumnsToFetch())
 -                    columns.add(builder.copy().add(id.key).build());
 +                for (ColumnDefinition def : selection.getColumnsList())
-                     if (def.kind == ColumnDefinition.Kind.REGULAR)
-                         columns.add(cfm.comparator.create(prefix, def.name));
++                    if (def.kind == ColumnDefinition.Kind.REGULAR || def.kind == ColumnDefinition.Kind.STATIC)
++                        columns.add(cfm.comparator.create(prefix, def));
              }
              else
              {
+                 // We now that we're not composite so we can ignore static columns
 -                Iterator<CFDefinition.Name> iter = cfDef.regularColumns().iterator();
 -                while (iter.hasNext())
 -                {
 -                    ColumnIdentifier name = iter.next().name;
 -                    ColumnNameBuilder b = iter.hasNext() ? builder.copy() : builder;
 -                    ByteBuffer cname = b.add(name.key).build();
 -                    columns.add(cname);
 -                }
 +                for (ColumnDefinition def : cfm.regularColumns())
-                     columns.add(cfm.comparator.create(prefix, def.name));
++                    columns.add(cfm.comparator.create(prefix, def));
              }
              return columns;
          }
@@@ -777,22 -877,23 +883,23 @@@
              return Collections.emptyList();
  
          List<IndexExpression> expressions = new ArrayList<IndexExpression>();
 -        for (CFDefinition.Name name : restrictedNames)
 +        for (ColumnDefinition def : restrictedColumns)
          {
              Restriction restriction;
 -            switch (name.kind)
 +            switch (def.kind)
              {
 -                case KEY_ALIAS:
 -                    restriction = keyRestrictions[name.position];
 +                case PARTITION_KEY:
 +                    restriction = keyRestrictions[def.position()];
                      break;
 -                case COLUMN_ALIAS:
 -                    restriction = columnRestrictions[name.position];
 +                case CLUSTERING_COLUMN:
 +                    restriction = columnRestrictions[def.position()];
                      break;
 -                case COLUMN_METADATA:
 +                case REGULAR:
+                 case STATIC:
 -                    restriction = metadataRestrictions.get(name);
 +                    restriction = metadataRestrictions.get(def.name);
                      break;
                  default:
 -                    // We don't allow restricting a VALUE_ALIAS for now in prepare.
 +                    // We don't allow restricting a COMPACT_VALUE for now in prepare.
                      throw new AssertionError();
              }
  
@@@ -903,60 -1010,140 +1010,107 @@@
      void processColumnFamily(ByteBuffer key, ColumnFamily cf, List<ByteBuffer> variables, long now, Selection.ResultSetBuilder result)
      throws InvalidRequestException
      {
 -        ByteBuffer[] keyComponents = cfDef.hasCompositeKey
 -                                   ? ((CompositeType)cfDef.cfm.getKeyValidator()).split(key)
 -                                   : new ByteBuffer[]{ key };
 -
 -        if (parameters.isDistinct)
 +        CFMetaData cfm = cf.metadata();
 +        ByteBuffer[] keyComponents = null;
 +        if (cfm.getKeyValidator() instanceof CompositeType)
          {
 -            if (!cf.hasOnlyTombstones(now))
 -            {
 -                result.newRow();
 -                // selection.getColumnsList() will contain only the partition key components - all of them.
 -                for (CFDefinition.Name name : selection.getColumnsList())
 -                    result.add(keyComponents[name.position]);
 -            }
 +            keyComponents = ((CompositeType)cfm.getKeyValidator()).split(key);
          }
 -        else if (cfDef.isCompact)
 +        else
          {
 -            // One cqlRow per column
 -            for (Column c : columnsInOrder(cf, variables))
 -            {
 -                if (c.isMarkedForDelete(now))
 -                    continue;
 +            keyComponents = new ByteBuffer[]{ key };
 +        }
  
 -                ByteBuffer[] components = null;
 -                if (cfDef.isComposite)
 -                {
 -                    components = ((CompositeType)cfDef.cfm.comparator).split(c.name());
 -                }
 -                else if (sliceRestriction != null)
 -                {
 -                    // For dynamic CF, the column could be out of the requested bounds, filter here
 -                    if (!sliceRestriction.isInclusive(Bound.START) && c.name().equals(sliceRestriction.bound(Bound.START, variables)))
 -                        continue;
 -                    if (!sliceRestriction.isInclusive(Bound.END) && c.name().equals(sliceRestriction.bound(Bound.END, variables)))
 -                        continue;
 -                }
 +        Iterator<Cell> cells = cf.getSortedColumns().iterator();
 +        if (sliceRestriction != null)
 +            cells = applySliceRestriction(cells, variables);
  
-         for (Iterator<CQL3Row> iter = cfm.comparator.CQL3RowBuilder(now).group(cells); iter.hasNext();)
 -                result.newRow();
 -                // Respect selection order
 -                for (CFDefinition.Name name : selection.getColumnsList())
++        CQL3Row.RowIterator iter = cfm.comparator.CQL3RowBuilder(now).group(cells);
++
++        // If there is static columns but there is no non-static row, then provided the select was a full
++        // partition selection (i.e. not a 2ndary index search and there was no condition on clustering columns)
++        // then we want to include the static columns in the result set (and we're done).
++        CQL3Row staticRow = iter.getStaticRow();
++        if (staticRow != null && !iter.hasNext() && !usesSecondaryIndexing && hasNoClusteringColumnsRestriction())
++        {
++            result.newRow();
++            for (ColumnDefinition def : selection.getColumnsList())
++            {
++                switch (def.kind)
+                 {
 -                    switch (name.kind)
 -                    {
 -                        case KEY_ALIAS:
 -                            result.add(keyComponents[name.position]);
 -                            break;
 -                        case COLUMN_ALIAS:
 -                            ByteBuffer val = cfDef.isComposite
 -                                           ? (name.position < components.length ? components[name.position] : null)
 -                                           : c.name();
 -                            result.add(val);
 -                            break;
 -                        case VALUE_ALIAS:
 -                            result.add(c);
 -                            break;
 -                        case COLUMN_METADATA:
 -                        case STATIC:
 -                            // This should not happen for compact CF
 -                            throw new AssertionError();
 -                        default:
 -                            throw new AssertionError();
 -                    }
++                    case PARTITION_KEY:
++                        result.add(keyComponents[def.position()]);
++                        break;
++                    case STATIC:
++                        addValue(result, def, staticRow);
++                        break;
++                    default:
++                        result.add((ByteBuffer)null);
+                 }
+             }
++            return;
+         }
 -        else if (cfDef.isComposite)
 -        {
 -            // Sparse case: group column in cqlRow when composite prefix is equal
 -            CompositeType composite = (CompositeType)cfDef.cfm.comparator;
+ 
 -            ColumnGroupMap.Builder builder = new ColumnGroupMap.Builder(composite, cfDef.hasCollections, now);
 -
 -            for (Column c : cf)
 -            {
 -                if (c.isMarkedForDelete(now))
 -                    continue;
 -
 -                builder.add(c);
 -            }
++        while (iter.hasNext())
 +        {
 +            CQL3Row cql3Row = iter.next();
  
 -            Map<CFDefinition.Name, ByteBuffer> staticValues = Collections.emptyMap();
 -            // Gather up static values first
 -            if (!builder.isEmpty() && builder.firstGroup().isStatic)
 +            // Respect requested order
 +            result.newRow();
 +            // Respect selection order
 +            for (ColumnDefinition def : selection.getColumnsList())
              {
 -                staticValues = new HashMap<>();
 -                ColumnGroupMap group = builder.firstGroup();
 -                for (CFDefinition.Name name : Iterables.filter(selection.getColumnsList(), isStaticFilter))
 -                    staticValues.put(name, name.type.isCollection() ? getCollectionValue(name, group) : getSimpleValue(name, group));
 -                builder.discardFirst();
 -
 -                // If there was static columns but there is no actual row, then provided the select was a full
 -                // partition selection (i.e. not a 2ndary index search and there was no condition on clustering columns)
 -                // then we want to include the static columns in the result set.
 -                if (!staticValues.isEmpty() && builder.isEmpty() && !usesSecondaryIndexing && hasNoClusteringColumnsRestriction())
 +                switch (def.kind)
                  {
 -                    result.newRow();
 -                    for (CFDefinition.Name name : selection.getColumnsList())
 -                    {
 -                        if (name.kind == CFDefinition.Name.Kind.KEY_ALIAS)
 -                            result.add(keyComponents[name.position]);
 -                        else
 -                            result.add(name.kind == CFDefinition.Name.Kind.STATIC ? staticValues.get(name) : null);
 -                    }
 -                    return;
 +                    case PARTITION_KEY:
 +                        result.add(keyComponents[def.position()]);
 +                        break;
 +                    case CLUSTERING_COLUMN:
 +                        result.add(cql3Row.getClusteringColumn(def.position()));
 +                        break;
 +                    case COMPACT_VALUE:
 +                        result.add(cql3Row.getColumn(null));
 +                        break;
 +                    case REGULAR:
-                         if (def.type.isCollection())
-                         {
-                             List<Cell> collection = cql3Row.getCollection(def.name);
-                             ByteBuffer value = collection == null
-                                              ? null
-                                              : ((CollectionType)def.type).serialize(collection);
-                             result.add(value);
-                         }
-                         else
-                         {
-                             result.add(cql3Row.getColumn(def.name));
-                         }
++                        addValue(result, def, cql3Row);
++                        break;
++                    case STATIC:
++                        addValue(result, def, staticRow);
 +                        break;
-                     }
                  }
+             }
 -
 -            for (ColumnGroupMap group : builder.groups())
 -                handleGroup(selection, result, keyComponents, group, staticValues);
          }
 -        else
 +    }
 +
++    private static void addValue(Selection.ResultSetBuilder result, ColumnDefinition def, CQL3Row row)
++    {
++        if (row == null)
+         {
 -            if (cf.hasOnlyTombstones(now))
 -                return;
++            result.add((ByteBuffer)null);
++            return;
++        }
+ 
 -            // Static case: One cqlRow for all columns
 -            result.newRow();
 -            for (CFDefinition.Name name : selection.getColumnsList())
 -            {
 -                if (name.kind == CFDefinition.Name.Kind.KEY_ALIAS)
 -                    result.add(keyComponents[name.position]);
 -                else
 -                    result.add(cf.getColumn(name.name.key));
 -            }
++        if (def.type.isCollection())
++        {
++            List<Cell> collection = row.getCollection(def.name);
++            ByteBuffer value = collection == null
++                             ? null
++                             : ((CollectionType)def.type).serialize(collection);
++            result.add(value);
++            return;
+         }
++
++        result.add(row.getColumn(def.name));
+     }
+ 
+     private boolean hasNoClusteringColumnsRestriction()
+     {
+         for (int i = 0; i < columnRestrictions.length; i++)
+             if (columnRestrictions[i] != null)
+                 return false;
+         return true;
+     }
+ 
      /**
       * Orders results when multiple keys are selected (using IN)
       */
@@@ -1119,23 -1331,24 +1281,24 @@@
                          hasQueriableClusteringColumnIndex = true;
                  }
  
 -                switch (name.kind)
 +                switch (def.kind)
                  {
 -                    case KEY_ALIAS:
 -                        stmt.keyRestrictions[name.position] = updateRestriction(cfm, name, stmt.keyRestrictions[name.position], rel, names);
 +                    case PARTITION_KEY:
 +                        stmt.keyRestrictions[def.position()] = updateRestriction(cfm, def, stmt.keyRestrictions[def.position()], rel, names);
                          break;
 -                    case COLUMN_ALIAS:
 -                        stmt.columnRestrictions[name.position] = updateRestriction(cfm, name, stmt.columnRestrictions[name.position], rel, names);
 +                    case CLUSTERING_COLUMN:
 +                        stmt.columnRestrictions[def.position()] = updateRestriction(cfm, def, stmt.columnRestrictions[def.position()], rel, names);
                          break;
 -                    case VALUE_ALIAS:
 -                        throw new InvalidRequestException(String.format("Predicates on the non-primary-key column (%s) of a COMPACT table are not yet supported", name.name));
 -                    case COLUMN_METADATA:
 +                    case COMPACT_VALUE:
 +                        throw new InvalidRequestException(String.format("Predicates on the non-primary-key column (%s) of a COMPACT table are not yet supported", def.name));
 +                    case REGULAR:
+                     case STATIC:
                          // We only all IN on the row key and last clustering key so far, never on non-PK columns, and this even if there's an index
 -                        Restriction r = updateRestriction(cfm, name, stmt.metadataRestrictions.get(name), rel, names);
 +                        Restriction r = updateRestriction(cfm, def, stmt.metadataRestrictions.get(def.name), rel, names);
                          if (r.isIN() && !((Restriction.IN)r).canHaveOnlyOneValue())
                              // Note: for backward compatibility reason, we conside a IN of 1 value the same as a EQ, so we let that slide.
 -                            throw new InvalidRequestException(String.format("IN predicates on non-primary-key columns (%s) is not yet supported", name));
 -                        stmt.metadataRestrictions.put(name, r);
 +                            throw new InvalidRequestException(String.format("IN predicates on non-primary-key columns (%s) is not yet supported", def.name));
 +                        stmt.metadataRestrictions.put(def.name, r);
                          break;
                  }
              }
@@@ -1222,8 -1435,11 +1385,11 @@@
              // All (or none) of the partition key columns have been specified;
              // hence there is no need to turn these restrictions into index expressions.
              if (!stmt.usesSecondaryIndexing)
 -                stmt.restrictedNames.removeAll(cfDef.partitionKeys());
 +                stmt.restrictedColumns.removeAll(cfm.partitionKeyColumns());
  
+             if (stmt.selectsOnlyStaticColumns && stmt.hasClusteringColumnsRestriction())
+                 throw new InvalidRequestException("Cannot restrict clustering columns when selecting only static columns");
+ 
              // If a clustering key column is restricted by a non-EQ relation, all preceding
              // columns must have a EQ, and all following must have no restriction. Unless
              // the column is indexed that is.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/cql3/statements/Selection.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/Selection.java
index 9c2eb0c,9760311..c506cc6
--- a/src/java/org/apache/cassandra/cql3/statements/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Selection.java
@@@ -55,30 -49,30 +55,36 @@@ public abstract class Selectio
          this.collectTTLs = collectTTLs;
      }
  
+     // Overriden by SimpleSelection when appropriate.
+     public boolean isWildcard()
+     {
+         return false;
+     }
+ 
      public ResultSet.Metadata getResultMetadata()
      {
 -        return new ResultSet.Metadata(metadata);
 +        return metadata;
      }
  
 -    public static Selection wildcard(CFDefinition cfDef)
 +    public static Selection wildcard(CFMetaData cfm)
      {
 -        List<CFDefinition.Name> all = new ArrayList<CFDefinition.Name>();
 -        for (CFDefinition.Name name : cfDef)
 -            all.add(name);
 +        List<ColumnDefinition> all = new ArrayList<ColumnDefinition>(cfm.allColumns().size());
 +        Iterators.addAll(all, cfm.allColumnsInSelectOrder());
-         return new SimpleSelection(all);
+         return new SimpleSelection(all, true);
      }
  
 -    public static Selection forColumns(List<CFDefinition.Name> columnsList)
 +    public static Selection forColumns(List<ColumnDefinition> columnsList)
      {
-         return new SimpleSelection(columnsList);
+         return new SimpleSelection(columnsList, false);
      }
  
 +    public int addColumnForOrdering(ColumnDefinition c)
 +    {
 +        columnsList.add(c);
 +        metadata.addNonSerializedColumn(c);
 +        return columnsList.size() - 1;
 +    }
 +
      private static boolean isUsingFunction(List<RawSelector> rawSelectors)
      {
          for (RawSelector rawSelector : rawSelectors)
@@@ -114,12 -108,12 +120,12 @@@
          else if (raw.selectable instanceof Selectable.WritetimeOrTTL)
          {
              Selectable.WritetimeOrTTL tot = (Selectable.WritetimeOrTTL)raw.selectable;
 -            CFDefinition.Name name = cfDef.get(tot.id);
 -            if (name == null)
 +            ColumnDefinition def = cfm.getColumnDefinition(tot.id);
 +            if (def == null)
                  throw new InvalidRequestException(String.format("Undefined name %s in selection clause", tot.id));
-             if (def.kind != ColumnDefinition.Kind.REGULAR && def.kind != ColumnDefinition.Kind.COMPACT_VALUE)
 -            if (name.isPrimaryKeyColumn())
 -                throw new InvalidRequestException(String.format("Cannot use selection function %s on PRIMARY KEY part %s", tot.isWritetime ? "writeTime" : "ttl", name));
 -            if (name.type.isCollection())
++            if (def.isPrimaryKeyColumn())
 +                throw new InvalidRequestException(String.format("Cannot use selection function %s on PRIMARY KEY part %s", tot.isWritetime ? "writeTime" : "ttl", def.name));
 +            if (def.type.isCollection())
                  throw new InvalidRequestException(String.format("Cannot use selection function %s on collections", tot.isWritetime ? "writeTime" : "ttl"));
  
              if (metadata != null)
@@@ -229,13 -195,13 +235,13 @@@
              for (RawSelector rawSelector : rawSelectors)
              {
                  assert rawSelector.selectable instanceof ColumnIdentifier;
 -                CFDefinition.Name name = cfDef.get((ColumnIdentifier)rawSelector.selectable);
 -                if (name == null)
 +                ColumnDefinition def = cfm.getColumnDefinition((ColumnIdentifier)rawSelector.selectable);
 +                if (def == null)
                      throw new InvalidRequestException(String.format("Undefined name %s in selection clause", rawSelector.selectable));
 -                names.add(name);
 -                metadata.add(rawSelector.alias == null ? name : makeAliasSpec(cfDef, name.type, rawSelector.alias));
 +                defs.add(def);
 +                metadata.add(rawSelector.alias == null ? def : makeAliasSpec(cfm, def.type, rawSelector.alias));
              }
-             return new SimpleSelection(defs, metadata);
 -            return new SimpleSelection(names, metadata, false);
++            return new SimpleSelection(defs, metadata, false);
          }
      }
  
@@@ -333,12 -313,14 +339,14 @@@
      // Special cased selection for when no function is used (this save some allocations).
      private static class SimpleSelection extends Selection
      {
-         public SimpleSelection(List<ColumnDefinition> columnsList)
+         private final boolean isWildcard;
+ 
 -        public SimpleSelection(List<CFDefinition.Name> columnsList, boolean isWildcard)
++        public SimpleSelection(List<ColumnDefinition> columnsList, boolean isWildcard)
          {
-             this(columnsList, new ArrayList<ColumnSpecification>(columnsList));
+             this(columnsList, new ArrayList<ColumnSpecification>(columnsList), isWildcard);
          }
  
-         public SimpleSelection(List<ColumnDefinition> columnsList, List<ColumnSpecification> metadata)
 -        public SimpleSelection(List<CFDefinition.Name> columnsList, List<ColumnSpecification> metadata, boolean isWildcard)
++        public SimpleSelection(List<ColumnDefinition> columnsList, List<ColumnSpecification> metadata, boolean isWildcard)
          {
              /*
               * In theory, even a simple selection could have multiple time the same column, so we
@@@ -352,20 -335,20 +361,26 @@@
          {
              return rs.current;
          }
+ 
+         @Override
+         public boolean isWildcard()
+         {
+             return isWildcard;
+         }
      }
  
 -    private interface Selector extends AssignementTestable
 +    private static abstract class Selector implements AssignementTestable
      {
 -        public ByteBuffer compute(ResultSetBuilder rs) throws InvalidRequestException;
 +        public abstract ByteBuffer compute(ResultSetBuilder rs) throws InvalidRequestException;
 +        public abstract AbstractType<?> getType();
 +
 +        public boolean isAssignableTo(String keyspace, ColumnSpecification receiver)
 +        {
 +            return getType().asCQL3Type().equals(receiver.type.asCQL3Type());
 +        }
      }
  
 -    private static class SimpleSelector implements Selector
 +    private static class SimpleSelector extends Selector
      {
          private final String columnName;
          private final int idx;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
index 6ed0e33,0e6481b..069f31f
--- a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
@@@ -35,11 -33,11 +35,11 @@@ import org.apache.cassandra.utils.Pair
   */
  public class UpdateStatement extends ModificationStatement
  {
 -    private static final Operation setToEmptyOperation = new Constants.Setter(null, new Constants.Value(ByteBufferUtil.EMPTY_BYTE_BUFFER));
 +    private static final Constants.Value EMPTY = new Constants.Value(ByteBufferUtil.EMPTY_BYTE_BUFFER);
  
-     private UpdateStatement(int boundTerms, CFMetaData cfm, Attributes attrs)
 -    private UpdateStatement(StatementType type, CFMetaData cfm, Attributes attrs)
++    private UpdateStatement(StatementType type, int boundTerms, CFMetaData cfm, Attributes attrs)
      {
-         super(boundTerms, cfm, attrs);
 -        super(type, cfm, attrs);
++        super(type, boundTerms, cfm, attrs);
      }
  
      public boolean requireFullClusteringKey()
@@@ -61,8 -61,11 +61,8 @@@
          // 'DELETE FROM t WHERE k = 1' does remove the row entirely)
          //
          // We never insert markers for Super CF as this would confuse the thrift side.
-         if (cfm.isCQL3Table())
 -        if (cfDef.isComposite && !cfDef.isCompact && !cfm.isSuper())
 -        {
 -            ByteBuffer name = builder.copy().add(ByteBufferUtil.EMPTY_BYTE_BUFFER).build();
 -            cf.addColumn(params.makeColumn(name, ByteBufferUtil.EMPTY_BYTE_BUFFER));
 -        }
++        if (cfm.isCQL3Table() && !prefix.isStatic())
 +            cf.addColumn(params.makeColumn(cfm.comparator.rowMarker(prefix), ByteBufferUtil.EMPTY_BYTE_BUFFER));
  
          List<Operation> updates = getOperations();
  
@@@ -127,9 -129,9 +127,9 @@@
              this.columnValues = columnValues;
          }
  
 -        protected ModificationStatement prepareInternal(CFDefinition cfDef, VariableSpecifications boundNames, Attributes attrs) throws InvalidRequestException
 +        protected ModificationStatement prepareInternal(CFMetaData cfm, VariableSpecifications boundNames, Attributes attrs) throws InvalidRequestException
          {
-             UpdateStatement stmt = new UpdateStatement(boundNames.size(), cfm, attrs);
 -            UpdateStatement stmt = new UpdateStatement(ModificationStatement.StatementType.INSERT, cfDef.cfm, attrs);
++            UpdateStatement stmt = new UpdateStatement(ModificationStatement.StatementType.INSERT,boundNames.size(), cfm, attrs);
  
              // Created from an INSERT
              if (stmt.isCounter())
@@@ -151,17 -153,18 +151,16 @@@
  
                  Term.Raw value = columnValues.get(i);
  
 -                switch (name.kind)
 +                switch (def.kind)
                  {
 -                    case KEY_ALIAS:
 -                    case COLUMN_ALIAS:
 -                        Term t = value.prepare(name);
 +                    case PARTITION_KEY:
 +                    case CLUSTERING_COLUMN:
 +                        Term t = value.prepare(keyspace(), def);
                          t.collectMarkerSpecification(boundNames);
-                         stmt.addKeyValue(def.name, t);
 -                        stmt.addKeyValue(name, t);
++                        stmt.addKeyValue(def, t);
                          break;
-                     case COMPACT_VALUE:
-                     case REGULAR:
 -                    case VALUE_ALIAS:
 -                    case COLUMN_METADATA:
 -                    case STATIC:
 -                        Operation operation = new Operation.SetValue(value).prepare(name);
++                    default:
 +                        Operation operation = new Operation.SetValue(value).prepare(keyspace(), def);
                          operation.collectMarkerSpecification(boundNames);
                          stmt.addOperation(operation);
                          break;
@@@ -197,26 -200,27 +196,25 @@@
              this.whereClause = whereClause;
          }
  
 -        protected ModificationStatement prepareInternal(CFDefinition cfDef, VariableSpecifications boundNames, Attributes attrs) throws InvalidRequestException
 +        protected ModificationStatement prepareInternal(CFMetaData cfm, VariableSpecifications boundNames, Attributes attrs) throws InvalidRequestException
          {
-             UpdateStatement stmt = new UpdateStatement(boundNames.size(), cfm, attrs);
 -            UpdateStatement stmt = new UpdateStatement(ModificationStatement.StatementType.UPDATE, cfDef.cfm, attrs);
++            UpdateStatement stmt = new UpdateStatement(ModificationStatement.StatementType.UPDATE, boundNames.size(), cfm, attrs);
  
              for (Pair<ColumnIdentifier, Operation.RawUpdate> entry : updates)
              {
 -                CFDefinition.Name name = cfDef.get(entry.left);
 -                if (name == null)
 +                ColumnDefinition def = cfm.getColumnDefinition(entry.left);
 +                if (def == null)
                      throw new InvalidRequestException(String.format("Unknown identifier %s", entry.left));
  
 -                Operation operation = entry.right.prepare(name);
 +                Operation operation = entry.right.prepare(keyspace(), def);
                  operation.collectMarkerSpecification(boundNames);
  
 -                switch (name.kind)
 +                switch (def.kind)
                  {
 -                    case KEY_ALIAS:
 -                    case COLUMN_ALIAS:
 +                    case PARTITION_KEY:
 +                    case CLUSTERING_COLUMN:
                          throw new InvalidRequestException(String.format("PRIMARY KEY part %s found in SET part", entry.left));
-                     case COMPACT_VALUE:
-                     case REGULAR:
 -                    case VALUE_ALIAS:
 -                    case COLUMN_METADATA:
 -                    case STATIC:
++                    default:
                          stmt.addOperation(operation);
                          break;
                  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/CFRowAdder.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/CFRowAdder.java
index 9fd5e9a,0000000..f853d17
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/CFRowAdder.java
+++ b/src/java/org/apache/cassandra/db/CFRowAdder.java
@@@ -1,110 -1,0 +1,110 @@@
 +/*
 + * 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 org.apache.cassandra.cql3.ColumnIdentifier;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.db.composites.CellName;
 +import org.apache.cassandra.db.composites.Composite;
 +import org.apache.cassandra.db.marshal.AbstractType;
 +import org.apache.cassandra.db.marshal.CollectionType;
 +import org.apache.cassandra.db.marshal.ListType;
 +import org.apache.cassandra.db.marshal.MapType;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +import org.apache.cassandra.utils.UUIDGen;
 +
 +/**
 + * Convenience object to populate a given CQL3 row in a ColumnFamily object.
 + *
 + * This is meant for when performance is not of the utmost importance. When
 + * performance matters, it might be worth allocating such builder.
 + */
 +public class CFRowAdder
 +{
 +    public final ColumnFamily cf;
 +    public final Composite prefix;
 +    public final long timestamp;
 +    private final int ldt;
 +
 +    public CFRowAdder(ColumnFamily cf, Composite prefix, long timestamp)
 +    {
 +        this.cf = cf;
 +        this.prefix = prefix;
 +        this.timestamp = timestamp;
 +        this.ldt = (int) (System.currentTimeMillis() / 1000);
 +
 +        // If a CQL3 table, add the row marker
-         if (cf.metadata().isCQL3Table())
++        if (cf.metadata().isCQL3Table() && !prefix.isStatic())
 +            cf.addColumn(new Cell(cf.getComparator().rowMarker(prefix), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp));
 +    }
 +
 +    public CFRowAdder add(String cql3ColumnName, Object value)
 +    {
 +        ColumnDefinition def = getDefinition(cql3ColumnName);
-         return add(cf.getComparator().create(prefix, def.name), def, value);
++        return add(cf.getComparator().create(prefix, def), def, value);
 +    }
 +
 +    public CFRowAdder resetCollection(String cql3ColumnName)
 +    {
 +        ColumnDefinition def = getDefinition(cql3ColumnName);
 +        assert def.type.isCollection();
-         Composite name = cf.getComparator().create(prefix, def.name);
++        Composite name = cf.getComparator().create(prefix, def);
 +        cf.addAtom(new RangeTombstone(name.start(), name.end(), timestamp - 1, ldt));
 +        return this;
 +    }
 +
 +    public CFRowAdder addMapEntry(String cql3ColumnName, Object key, Object value)
 +    {
 +        ColumnDefinition def = getDefinition(cql3ColumnName);
 +        assert def.type instanceof MapType;
 +        MapType mt = (MapType)def.type;
-         CellName name = cf.getComparator().create(prefix, def.name, mt.keys.decompose(key));
++        CellName name = cf.getComparator().create(prefix, def, mt.keys.decompose(key));
 +        return add(name, def, value);
 +    }
 +
 +    public CFRowAdder addListEntry(String cql3ColumnName, Object value)
 +    {
 +        ColumnDefinition def = getDefinition(cql3ColumnName);
 +        assert def.type instanceof ListType;
-         CellName name = cf.getComparator().create(prefix, def.name, ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes()));
++        CellName name = cf.getComparator().create(prefix, def, ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes()));
 +        return add(name, def, value);
 +    }
 +
 +    private ColumnDefinition getDefinition(String name)
 +    {
 +        return cf.metadata().getColumnDefinition(new ColumnIdentifier(name, false));
 +    }
 +
 +    private CFRowAdder add(CellName name, ColumnDefinition def, Object value)
 +    {
 +        if (value == null)
 +        {
 +            cf.addColumn(new DeletedCell(name, ldt, timestamp));
 +        }
 +        else
 +        {
 +            AbstractType valueType = def.type.isCollection()
 +                                   ? ((CollectionType) def.type).valueComparator()
 +                                   : def.type;
 +            cf.addColumn(new Cell(name, value instanceof ByteBuffer ? (ByteBuffer)value : valueType.decompose(value), timestamp));
 +        }
 +        return this;
 +    }
 +}


[3/7] Merge branch 'cassandra-2.0' into cassandra-2.1

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/CompoundSparseCellNameType.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/CompoundSparseCellNameType.java
index d0e66f8,0000000..e0cbc0f
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/CompoundSparseCellNameType.java
+++ b/src/java/org/apache/cassandra/db/composites/CompoundSparseCellNameType.java
@@@ -1,245 -1,0 +1,301 @@@
 +/*
 + * 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.composites;
 +
 +import java.nio.ByteBuffer;
 +import java.util.*;
 +
++import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.CQL3Row;
 +import org.apache.cassandra.cql3.ColumnIdentifier;
 +import org.apache.cassandra.db.marshal.AbstractType;
 +import org.apache.cassandra.db.marshal.CollectionType;
 +import org.apache.cassandra.db.marshal.ColumnToCollectionType;
 +import org.apache.cassandra.db.marshal.UTF8Type;
 +import org.apache.cassandra.utils.ByteBufferUtil;
++import org.apache.cassandra.utils.memory.AbstractAllocator;
++import org.apache.cassandra.utils.memory.PoolAllocator;
 +
 +public class CompoundSparseCellNameType extends AbstractCompoundCellNameType
 +{
 +    private static final ColumnIdentifier rowMarkerId = new ColumnIdentifier(ByteBufferUtil.EMPTY_BYTE_BUFFER, UTF8Type.instance);
-     private static final CellName rowMarkerNoPrefix = new CompoundSparseCellName(rowMarkerId);
++    private static final CellName rowMarkerNoPrefix = new CompoundSparseCellName(rowMarkerId, false);
 +
 +    // For CQL3 columns, this is always UTF8Type. However, for compatibility with super columns, we need to allow it to be non-UTF8.
 +    private final AbstractType<?> columnNameType;
 +    protected final Map<ByteBuffer, ColumnIdentifier> internedIds;
 +
++    private final Composite staticPrefix;
++
 +    public CompoundSparseCellNameType(List<AbstractType<?>> types)
 +    {
 +        this(types, UTF8Type.instance);
 +    }
 +
 +    public CompoundSparseCellNameType(List<AbstractType<?>> types, AbstractType<?> columnNameType)
 +    {
 +        this(new CompoundCType(types), columnNameType);
 +    }
 +
 +    private CompoundSparseCellNameType(CompoundCType clusteringType, AbstractType<?> columnNameType)
 +    {
 +        this(clusteringType, columnNameType, makeCType(clusteringType, columnNameType, null), new HashMap<ByteBuffer, ColumnIdentifier>());
 +    }
 +
 +    private CompoundSparseCellNameType(CompoundCType clusteringType, AbstractType<?> columnNameType, CompoundCType fullType, Map<ByteBuffer, ColumnIdentifier> internedIds)
 +    {
 +        super(clusteringType, fullType);
 +        this.columnNameType = columnNameType;
 +        this.internedIds = internedIds;
++        this.staticPrefix = makeStaticPrefix(clusteringType.size());
++    }
++
++    private static Composite makeStaticPrefix(int size)
++    {
++        ByteBuffer[] elements = new ByteBuffer[size];
++        for (int i = 0; i < size; i++)
++            elements[i] = ByteBufferUtil.EMPTY_BYTE_BUFFER;
++
++        return new CompoundComposite(elements, size, true)
++        {
++            @Override
++            public boolean isStatic()
++            {
++                return true;
++            }
++
++            @Override
++            public long unsharedHeapSize()
++            {
++                // We'll share this for a given type.
++                return 0;
++            }
++
++            @Override
++            public Composite copy(AbstractAllocator allocator)
++            {
++                return this;
++            }
++
++            @Override
++            public void free(PoolAllocator<?> allocator)
++            {
++            }
++        };
 +    }
 +
 +    protected static CompoundCType makeCType(CompoundCType clusteringType, AbstractType<?> columnNameType, ColumnToCollectionType collectionType)
 +    {
 +        List<AbstractType<?>> allSubtypes = new ArrayList<AbstractType<?>>(clusteringType.size() + (collectionType == null ? 1 : 2));
 +        for (int i = 0; i < clusteringType.size(); i++)
 +            allSubtypes.add(clusteringType.subtype(i));
 +        allSubtypes.add(columnNameType);
 +        if (collectionType != null)
 +            allSubtypes.add(collectionType);
 +        return new CompoundCType(allSubtypes);
 +    }
 +
 +    public CellNameType setSubtype(int position, AbstractType<?> newType)
 +    {
 +        if (position < clusteringSize)
 +            return new CompoundSparseCellNameType(clusteringType.setSubtype(position, newType), columnNameType, fullType.setSubtype(position, newType), internedIds);
 +
 +        if (position == clusteringSize)
 +            throw new IllegalArgumentException();
 +
 +        throw new IndexOutOfBoundsException();
 +    }
 +
 +    @Override
 +    public CellNameType addCollection(ColumnIdentifier columnName, CollectionType newCollection)
 +    {
 +        return new WithCollection(clusteringType, ColumnToCollectionType.getInstance(Collections.singletonMap(columnName.bytes, newCollection)), internedIds);
 +    }
 +
 +    public boolean isDense()
 +    {
 +        return false;
 +    }
 +
 +    public boolean supportCollections()
 +    {
 +        return true;
 +    }
 +
-     public CellName create(Composite prefix, ColumnIdentifier columnName)
++    public Composite staticPrefix()
++    {
++        return staticPrefix;
++    }
++
++    public CellName create(Composite prefix, ColumnDefinition column)
 +    {
++        return create(prefix, column.name, column.isStatic());
++    }
++
++    private CellName create(Composite prefix, ColumnIdentifier columnName, boolean isStatic)
++    {
++        if (isStatic)
++            prefix = staticPrefix();
++
 +        assert prefix.size() == clusteringSize;
 +
 +        if (prefix.isEmpty())
-             return new CompoundSparseCellName(columnName);
++            return new CompoundSparseCellName(columnName, isStatic);
 +
 +        assert prefix instanceof CompoundComposite;
 +        CompoundComposite lc = (CompoundComposite)prefix;
-         return new CompoundSparseCellName(lc.elements, clusteringSize, columnName);
++        return new CompoundSparseCellName(lc.elements, clusteringSize, columnName, isStatic);
 +    }
 +
 +    public CellName rowMarker(Composite prefix)
 +    {
++        assert !prefix.isStatic(); // static columns don't really create rows, they shouldn't have a row marker
 +        if (prefix.isEmpty())
 +            return rowMarkerNoPrefix;
 +
-         return create(prefix, rowMarkerId);
++        return create(prefix, rowMarkerId, false);
 +    }
 +
 +    protected ColumnIdentifier idFor(ByteBuffer bb)
 +    {
 +        ColumnIdentifier id = internedIds.get(bb);
 +        return id == null ? new ColumnIdentifier(bb, columnNameType) : id;
 +    }
 +
-     protected Composite makeWith(ByteBuffer[] components, int size, Composite.EOC eoc)
++    protected Composite makeWith(ByteBuffer[] components, int size, Composite.EOC eoc, boolean isStatic)
 +    {
 +        if (size < clusteringSize + 1 || eoc != Composite.EOC.NONE)
-             return new CompoundComposite(components, size).withEOC(eoc);
++            return new CompoundComposite(components, size, isStatic).withEOC(eoc);
 +
-         return new CompoundSparseCellName(components, clusteringSize, idFor(components[clusteringSize]));
++        return new CompoundSparseCellName(components, clusteringSize, idFor(components[clusteringSize]), isStatic);
 +    }
 +
-     protected Composite copyAndMakeWith(ByteBuffer[] components, int size, Composite.EOC eoc)
++    protected Composite copyAndMakeWith(ByteBuffer[] components, int size, Composite.EOC eoc, boolean isStatic)
 +    {
 +        if (size < clusteringSize + 1 || eoc != Composite.EOC.NONE)
-             return new CompoundComposite(Arrays.copyOfRange(components, 0, size), size).withEOC(eoc);
++            return new CompoundComposite(Arrays.copyOfRange(components, 0, size), size, isStatic).withEOC(eoc);
 +
 +        ByteBuffer[] clusteringColumns = Arrays.copyOfRange(components, 0, clusteringSize);
-         return new CompoundSparseCellName(clusteringColumns, idFor(components[clusteringSize]));
++        return new CompoundSparseCellName(clusteringColumns, idFor(components[clusteringSize]), isStatic);
 +    }
 +
 +    public void addCQL3Column(ColumnIdentifier id)
 +    {
 +        internedIds.put(id.bytes, id);
 +    }
 +
 +    public void removeCQL3Column(ColumnIdentifier id)
 +    {
 +        internedIds.remove(id.bytes);
 +    }
 +
 +    public CQL3Row.Builder CQL3RowBuilder(long now)
 +    {
 +        return makeSparseCQL3RowBuilder(now);
 +    }
 +
 +    public static class WithCollection extends CompoundSparseCellNameType
 +    {
 +        private final ColumnToCollectionType collectionType;
 +
 +        public WithCollection(List<AbstractType<?>> types, ColumnToCollectionType collectionType)
 +        {
 +            this(new CompoundCType(types), collectionType);
 +        }
 +
 +        WithCollection(CompoundCType clusteringType, ColumnToCollectionType collectionType)
 +        {
 +            this(clusteringType, collectionType, new HashMap<ByteBuffer, ColumnIdentifier>());
 +        }
 +
 +        private WithCollection(CompoundCType clusteringType, ColumnToCollectionType collectionType, Map<ByteBuffer, ColumnIdentifier> internedIds)
 +        {
 +            this(clusteringType, makeCType(clusteringType, UTF8Type.instance, collectionType), collectionType, internedIds);
 +        }
 +
 +        private WithCollection(CompoundCType clusteringType, CompoundCType fullCType, ColumnToCollectionType collectionType, Map<ByteBuffer, ColumnIdentifier> internedIds)
 +        {
 +            super(clusteringType, UTF8Type.instance, fullCType, internedIds);
 +            this.collectionType = collectionType;
 +        }
 +
 +        @Override
 +        public CellNameType setSubtype(int position, AbstractType<?> newType)
 +        {
 +            if (position < clusteringSize)
 +                return new WithCollection(clusteringType.setSubtype(position, newType), collectionType, internedIds);
 +
 +            throw position >= fullType.size() ? new IndexOutOfBoundsException() : new IllegalArgumentException();
 +        }
 +
 +        @Override
 +        public CellNameType addCollection(ColumnIdentifier columnName, CollectionType newCollection)
 +        {
 +            Map<ByteBuffer, CollectionType> newMap = new HashMap<>(collectionType.defined);
 +            newMap.put(columnName.bytes, newCollection);
 +            return new WithCollection(clusteringType, ColumnToCollectionType.getInstance(newMap), internedIds);
 +        }
 +
 +        @Override
-         public CellName create(Composite prefix, ColumnIdentifier columnName, ByteBuffer collectionElement)
++        public CellName create(Composite prefix, ColumnDefinition column, ByteBuffer collectionElement)
 +        {
-             // We ignore the columnName because it's just the COMPACT_VALUE name which is not store in the cell name
++            if (column.isStatic())
++                prefix = staticPrefix();
++
 +            assert prefix.size() == clusteringSize;
 +
 +            if (prefix.isEmpty())
-                 return new CompoundSparseCellName.WithCollection(columnName, collectionElement);
++                return new CompoundSparseCellName.WithCollection(column.name, collectionElement, column.isStatic());
 +
 +            assert prefix instanceof CompoundComposite;
 +            CompoundComposite lc = (CompoundComposite)prefix;
-             return new CompoundSparseCellName.WithCollection(lc.elements, clusteringSize, columnName, collectionElement);
++            return new CompoundSparseCellName.WithCollection(lc.elements, clusteringSize, column.name, collectionElement, column.isStatic());
 +        }
 +
 +        @Override
 +        public boolean hasCollections()
 +        {
 +            return true;
 +        }
 +
 +        @Override
 +        public ColumnToCollectionType collectionType()
 +        {
 +            return collectionType;
 +        }
 +
 +        @Override
-         protected Composite makeWith(ByteBuffer[] components, int size, Composite.EOC eoc)
++        protected Composite makeWith(ByteBuffer[] components, int size, Composite.EOC eoc, boolean isStatic)
 +        {
 +            if (size < fullSize)
-                 return super.makeWith(components, size, eoc);
++                return super.makeWith(components, size, eoc, isStatic);
 +
-             return new CompoundSparseCellName.WithCollection(components, clusteringSize, idFor(components[clusteringSize]), components[fullSize - 1]);
++            return new CompoundSparseCellName.WithCollection(components, clusteringSize, idFor(components[clusteringSize]), components[fullSize - 1], isStatic);
 +        }
 +
-         protected Composite copyAndMakeWith(ByteBuffer[] components, int size, Composite.EOC eoc)
++        protected Composite copyAndMakeWith(ByteBuffer[] components, int size, Composite.EOC eoc, boolean isStatic)
 +        {
 +            if (size < fullSize)
-                 return super.copyAndMakeWith(components, size, eoc);
++                return super.copyAndMakeWith(components, size, eoc, isStatic);
 +
 +            ByteBuffer[] clusteringColumns = Arrays.copyOfRange(components, 0, clusteringSize);
-             return new CompoundSparseCellName.WithCollection(clusteringColumns, idFor(components[clusteringSize]), components[clusteringSize + 1]);
++            return new CompoundSparseCellName.WithCollection(clusteringColumns, idFor(components[clusteringSize]), components[clusteringSize + 1], isStatic);
 +        }
 +    }
 +}
 +

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/SimpleDenseCellNameType.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/SimpleDenseCellNameType.java
index cafb521,0000000..1aae580
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/SimpleDenseCellNameType.java
+++ b/src/java/org/apache/cassandra/db/composites/SimpleDenseCellNameType.java
@@@ -1,78 -1,0 +1,79 @@@
 +/*
 + * 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.composites;
 +
 +import java.nio.ByteBuffer;
 +
++import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.CQL3Row;
 +import org.apache.cassandra.cql3.ColumnIdentifier;
 +import org.apache.cassandra.db.marshal.AbstractType;
 +
 +public class SimpleDenseCellNameType extends AbstractSimpleCellNameType
 +{
 +    public SimpleDenseCellNameType(AbstractType<?> type)
 +    {
 +        super(type);
 +    }
 +
 +    public int clusteringPrefixSize()
 +    {
 +        return 1;
 +    }
 +
 +    public CBuilder prefixBuilder()
 +    {
 +        // Simple dense is "all" prefix
 +        return builder();
 +    }
 +
 +    public CellNameType setSubtype(int position, AbstractType<?> newType)
 +    {
 +        if (position != 0)
 +            throw new IllegalArgumentException();
 +        return new SimpleDenseCellNameType(newType);
 +    }
 +
 +    public boolean isDense()
 +    {
 +        return true;
 +    }
 +
-     public CellName create(Composite prefix, ColumnIdentifier columnName)
++    public CellName create(Composite prefix, ColumnDefinition column)
 +    {
 +        assert prefix.size() == 1;
-         // We ignore the columnName because it's just the COMPACT_VALUE name which is not store in the cell name
++        // We ignore the column because it's just the COMPACT_VALUE name which is not store in the cell name
 +        return new SimpleDenseCellName(prefix.get(0));
 +    }
 +
 +    @Override
 +    public Composite fromByteBuffer(ByteBuffer bb)
 +    {
 +        return !bb.hasRemaining()
 +             ? Composites.EMPTY
 +             : new SimpleDenseCellName(bb);
 +    }
 +
 +    public void addCQL3Column(ColumnIdentifier id) {}
 +    public void removeCQL3Column(ColumnIdentifier id) {}
 +
 +    public CQL3Row.Builder CQL3RowBuilder(long now)
 +    {
 +        return makeDenseCQL3RowBuilder(now);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/SimpleSparseCellNameType.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/SimpleSparseCellNameType.java
index 9c99680,0000000..0f63a6a
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/SimpleSparseCellNameType.java
+++ b/src/java/org/apache/cassandra/db/composites/SimpleSparseCellNameType.java
@@@ -1,98 -1,0 +1,99 @@@
 +/*
 + * 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.composites;
 +
 +import java.nio.ByteBuffer;
 +import java.util.HashMap;
 +import java.util.Map;
 +
++import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.CQL3Row;
 +import org.apache.cassandra.cql3.ColumnIdentifier;
 +import org.apache.cassandra.db.marshal.AbstractType;
 +
 +public class SimpleSparseCellNameType extends AbstractSimpleCellNameType
 +{
 +    // Simple sparse means static thrift CF or non-clustered CQL3. This means that cell names will mainly
 +    // be those that have been declared and we can intern the whole CellName instances.
 +    private final Map<ByteBuffer, CellName> internedNames;
 +
 +    public SimpleSparseCellNameType(AbstractType<?> type)
 +    {
 +        this(type, new HashMap<ByteBuffer, CellName>());
 +    }
 +
 +    private SimpleSparseCellNameType(AbstractType<?> type, Map<ByteBuffer, CellName> internedNames)
 +    {
 +        super(type);
 +        this.internedNames = internedNames;
 +    }
 +
 +    public int clusteringPrefixSize()
 +    {
 +        return 0;
 +    }
 +
 +    public CellNameType setSubtype(int position, AbstractType<?> newType)
 +    {
 +        if (position != 0)
 +            throw new IllegalArgumentException();
 +        return new SimpleSparseCellNameType(newType, internedNames);
 +    }
 +
 +    public CBuilder prefixBuilder()
 +    {
 +        return Composites.EMPTY_BUILDER;
 +    }
 +
 +    public boolean isDense()
 +    {
 +        return false;
 +    }
 +
-     public CellName create(Composite prefix, ColumnIdentifier columnName)
++    public CellName create(Composite prefix, ColumnDefinition column)
 +    {
 +        assert prefix.isEmpty();
-         CellName cn = internedNames.get(columnName.bytes);
-         return cn == null ? new SimpleSparseCellName(columnName) : cn;
++        CellName cn = internedNames.get(column.name.bytes);
++        return cn == null ? new SimpleSparseCellName(column.name) : cn;
 +    }
 +
 +    @Override
 +    public Composite fromByteBuffer(ByteBuffer bb)
 +    {
 +        if (!bb.hasRemaining())
 +            return Composites.EMPTY;
 +
 +        CellName cn = internedNames.get(bb);
 +        return cn == null ? new SimpleSparseCellName(new ColumnIdentifier(bb, type)) : cn;
 +    }
 +
 +    public void addCQL3Column(ColumnIdentifier id)
 +    {
 +        internedNames.put(id.bytes, new SimpleSparseInternedCellName(id));
 +    }
 +
 +    public void removeCQL3Column(ColumnIdentifier id)
 +    {
 +        internedNames.remove(id.bytes);
 +    }
 +
 +    public CQL3Row.Builder CQL3RowBuilder(long now)
 +    {
 +        return makeSparseCQL3RowBuilder(now);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
index 29976f6,5aa1ea9..7326d80
--- a/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
@@@ -322,49 -320,20 +322,49 @@@ public abstract class ExtendedFilte
              return true;
          }
  
 -        private ByteBuffer extractDataValue(ColumnDefinition def, ByteBuffer rowKey, ColumnFamily data, ColumnNameBuilder builder)
 +        private static boolean collectionSatisfies(ColumnDefinition def, ColumnFamily data, Composite prefix, IndexExpression expr, ByteBuffer collectionElement)
 +        {
 +            assert def.type.isCollection();
 +
 +            CollectionType type = (CollectionType)def.type;
 +            switch (type.kind)
 +            {
 +                case LIST:
 +                    assert collectionElement != null;
-                     return type.valueComparator().compare(data.getColumn(data.getComparator().create(prefix, def.name, collectionElement)).value(), expr.value) == 0;
++                    return type.valueComparator().compare(data.getColumn(data.getComparator().create(prefix, def, collectionElement)).value(), expr.value) == 0;
 +                case SET:
-                     return data.getColumn(data.getComparator().create(prefix, def.name, expr.value)) != null;
++                    return data.getColumn(data.getComparator().create(prefix, def, expr.value)) != null;
 +                case MAP:
 +                    if (expr.operator == IndexExpression.Operator.CONTAINS_KEY)
 +                    {
-                         return data.getColumn(data.getComparator().create(prefix, def.name, expr.value)) != null;
++                        return data.getColumn(data.getComparator().create(prefix, def, expr.value)) != null;
 +                    }
 +                    else
 +                    {
 +                        assert collectionElement != null;
-                         return type.valueComparator().compare(data.getColumn(data.getComparator().create(prefix, def.name, collectionElement)).value(), expr.value) == 0;
++                        return type.valueComparator().compare(data.getColumn(data.getComparator().create(prefix, def, collectionElement)).value(), expr.value) == 0;
 +                    }
 +            }
 +            throw new AssertionError();
 +        }
 +
 +        private ByteBuffer extractDataValue(ColumnDefinition def, ByteBuffer rowKey, ColumnFamily data, Composite prefix)
          {
 -            switch (def.type)
 +            switch (def.kind)
              {
                  case PARTITION_KEY:
 -                    return def.componentIndex == null
 +                    return def.isOnAllComponents()
                           ? rowKey
 -                         : ((CompositeType)data.metadata().getKeyValidator()).split(rowKey)[def.componentIndex];
 -                case CLUSTERING_KEY:
 -                    return builder.get(def.componentIndex);
 +                         : ((CompositeType)data.metadata().getKeyValidator()).split(rowKey)[def.position()];
 +                case CLUSTERING_COLUMN:
 +                    return prefix.get(def.position());
                  case REGULAR:
 -                    ByteBuffer colName = builder == null ? def.name : builder.copy().add(def.name).build();
 -                    Column column = data.getColumn(colName);
 -                    return column == null ? null : column.value();
 +                    CellName cname = prefix == null
 +                                   ? data.getComparator().cellFromByteBuffer(def.name.bytes)
-                                    : data.getComparator().create(prefix, def.name);
++                                   : data.getComparator().create(prefix, def);
 +
 +                    Cell cell = data.getColumn(cname);
 +                    return cell == null ? null : cell.value();
                  case COMPACT_VALUE:
                      assert data.getColumnCount() == 1;
                      return data.getSortedColumns().iterator().next().value();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java
index 36504ca,0000000..c8fc56c
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java
@@@ -1,106 -1,0 +1,106 @@@
 +/*
 + * 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.index.composites;
 +
 +import java.nio.ByteBuffer;
 +import java.util.ArrayList;
 +import java.util.List;
 +
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.composites.CBuilder;
 +import org.apache.cassandra.db.composites.CellName;
 +import org.apache.cassandra.db.composites.CellNameType;
 +import org.apache.cassandra.db.composites.Composite;
 +import org.apache.cassandra.db.composites.CompoundDenseCellNameType;
 +import org.apache.cassandra.db.index.SecondaryIndex;
 +import org.apache.cassandra.db.marshal.*;
 +
 +/**
 + * Index on the collection element of the cell name of a collection.
 + *
 + * A cell indexed by this index will have the general form:
 + *   ck_0 ... ck_n c_name [col_elt] : v
 + * where ck_i are the cluster keys, c_name the CQL3 column name, col_elt the
 + * collection element that we want to index (which may or may not be there depending
 + * on whether c_name is the collection we're indexing) and v the cell value.
 + *
 + * Such a cell is indexed if c_name is the indexed collection (in which case we are guaranteed to have
 + * col_elt). The index entry will be:
 + *   - row key will be col_elt value (getIndexedValue()).
 + *   - cell name will be 'rk ck_0 ... ck_n' where rk is the row key of the initial cell.
 + */
 +public class CompositesIndexOnCollectionKey extends CompositesIndex
 +{
 +    public static CellNameType buildIndexComparator(CFMetaData baseMetadata, ColumnDefinition columnDef)
 +    {
 +        int count = 1 + baseMetadata.clusteringColumns().size(); // row key + clustering prefix
 +        List<AbstractType<?>> types = new ArrayList<AbstractType<?>>(count);
 +        types.add(SecondaryIndex.keyComparator);
 +        for (int i = 0; i < count - 1; i++)
 +            types.add(baseMetadata.comparator.subtype(i));
 +        return new CompoundDenseCellNameType(types);
 +    }
 +
 +    @Override
 +    protected AbstractType<?> getIndexKeyComparator()
 +    {
 +        return ((CollectionType)columnDef.type).nameComparator();
 +    }
 +
 +    protected ByteBuffer getIndexedValue(ByteBuffer rowKey, Cell cell)
 +    {
 +        return cell.name().get(columnDef.position() + 1);
 +    }
 +
 +    protected Composite makeIndexColumnPrefix(ByteBuffer rowKey, Composite cellName)
 +    {
 +        int count = 1 + baseCfs.metadata.clusteringColumns().size();
 +        CBuilder builder = getIndexComparator().builder();
 +        builder.add(rowKey);
 +        for (int i = 0; i < count - 1; i++)
 +            builder.add(cellName.get(i));
 +        return builder.build();
 +    }
 +
 +    public IndexedEntry decodeEntry(DecoratedKey indexedValue, Cell indexEntry)
 +    {
 +        int count = 1 + baseCfs.metadata.clusteringColumns().size();
 +        CBuilder builder = baseCfs.getComparator().builder();
 +        for (int i = 0; i < count - 1; i++)
 +            builder.add(indexEntry.name().get(i + 1));
 +        return new IndexedEntry(indexedValue, indexEntry.name(), indexEntry.timestamp(), indexEntry.name().get(0), builder.build());
 +    }
 +
 +    @Override
 +    public boolean indexes(CellName name)
 +    {
 +        // We index if the CQL3 column name is the one of the collection we index
 +        AbstractType<?> comp = baseCfs.metadata.getColumnDefinitionComparator(columnDef);
 +        return name.size() > columnDef.position()
 +            && comp.compare(name.get(columnDef.position()), columnDef.name.bytes) == 0;
 +    }
 +
 +    public boolean isStale(IndexedEntry entry, ColumnFamily data, long now)
 +    {
-         CellName name = data.getComparator().create(entry.indexedEntryPrefix, columnDef.name, entry.indexValue.key);
++        CellName name = data.getComparator().create(entry.indexedEntryPrefix, columnDef, entry.indexValue.key);
 +        Cell liveCell = data.getColumn(name);
 +        return (liveCell == null || liveCell.isMarkedForDelete(now));
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java
index 1d780cd,0000000..9536e2e
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java
@@@ -1,108 -1,0 +1,108 @@@
 +/*
 + * 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.index.composites;
 +
 +import java.nio.ByteBuffer;
 +import java.util.ArrayList;
 +import java.util.List;
 +
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.composites.CBuilder;
 +import org.apache.cassandra.db.composites.CellName;
 +import org.apache.cassandra.db.composites.CellNameType;
 +import org.apache.cassandra.db.composites.Composite;
 +import org.apache.cassandra.db.composites.CompoundDenseCellNameType;
 +import org.apache.cassandra.db.index.SecondaryIndex;
 +import org.apache.cassandra.db.marshal.*;
 +
 +/**
 + * Index the value of a collection cell.
 + *
 + * This is a lot like an index on REGULAR, except that we also need to make
 + * the collection key part of the index entry so that:
 + *   1) we don't have to scan the whole collection at query time to know the
 + *   entry is stale and if it still satisfies the query.
 + *   2) if a collection has multiple time the same value, we need one entry
 + *   for each so that if we delete one of the value only we only delete the
 + *   entry corresponding to that value.
 + */
 +public class CompositesIndexOnCollectionValue extends CompositesIndex
 +{
 +    public static CellNameType buildIndexComparator(CFMetaData baseMetadata, ColumnDefinition columnDef)
 +    {
 +        int prefixSize = columnDef.position();
 +        List<AbstractType<?>> types = new ArrayList<AbstractType<?>>(prefixSize + 2);
 +        types.add(SecondaryIndex.keyComparator);
 +        for (int i = 0; i < prefixSize; i++)
 +            types.add(baseMetadata.comparator.subtype(i));
 +        types.add(((CollectionType)columnDef.type).nameComparator()); // collection key
 +        return new CompoundDenseCellNameType(types);
 +    }
 +
 +    @Override
 +    protected AbstractType<?> getIndexKeyComparator()
 +    {
 +        return ((CollectionType)columnDef.type).valueComparator();
 +    }
 +
 +    protected ByteBuffer getIndexedValue(ByteBuffer rowKey, Cell cell)
 +    {
 +        return cell.value();
 +    }
 +
 +    protected Composite makeIndexColumnPrefix(ByteBuffer rowKey, Composite cellName)
 +    {
 +        CBuilder builder = getIndexComparator().prefixBuilder();
 +        builder.add(rowKey);
 +        for (int i = 0; i < Math.min(columnDef.position(), cellName.size()); i++)
 +            builder.add(cellName.get(i));
 +        builder.add(cellName.get(columnDef.position() + 1));
 +        return builder.build();
 +    }
 +
 +    public IndexedEntry decodeEntry(DecoratedKey indexedValue, Cell indexEntry)
 +    {
 +        int prefixSize = columnDef.position();
 +        CellName name = indexEntry.name();
 +        CBuilder builder = baseCfs.getComparator().builder();
 +        for (int i = 0; i < prefixSize; i++)
 +            builder.add(name.get(i + 1));
 +        return new IndexedEntry(indexedValue, name, indexEntry.timestamp(), name.get(0), builder.build(), name.get(prefixSize + 1));
 +    }
 +
 +    @Override
 +    public boolean indexes(CellName name)
 +    {
 +        AbstractType<?> comp = baseCfs.metadata.getColumnDefinitionComparator(columnDef);
 +        return name.size() > columnDef.position()
 +            && comp.compare(name.get(columnDef.position()), columnDef.name.bytes) == 0;
 +    }
 +
 +    public boolean isStale(IndexedEntry entry, ColumnFamily data, long now)
 +    {
-         CellName name = data.getComparator().create(entry.indexedEntryPrefix, columnDef.name, entry.indexedEntryCollectionKey);
++        CellName name = data.getComparator().create(entry.indexedEntryPrefix, columnDef, entry.indexedEntryCollectionKey);
 +        Cell liveCell = data.getColumn(name);
 +        if (liveCell == null || liveCell.isMarkedForDelete(now))
 +            return true;
 +
 +        ByteBuffer liveValue = liveCell.value();
 +        return ((CollectionType)columnDef.type).valueComparator().compare(entry.indexValue.key, liveValue) != 0;
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnRegular.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnRegular.java
index f1b0954,7159c23..fc2f9db
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnRegular.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnRegular.java
@@@ -89,12 -93,12 +89,12 @@@ public class CompositesIndexOnRegular e
  
      public boolean isStale(IndexedEntry entry, ColumnFamily data, long now)
      {
-         CellName name = data.getComparator().create(entry.indexedEntryPrefix, columnDef.name);
 -        ByteBuffer bb = entry.indexedEntryNameBuilder.copy().add(columnDef.name).build();
 -        Column liveColumn = data.getColumn(bb);
 -        if (liveColumn == null || liveColumn.isMarkedForDelete(now))
++        CellName name = data.getComparator().create(entry.indexedEntryPrefix, columnDef);
 +        Cell liveCell = data.getColumn(name);
 +        if (liveCell == null || liveCell.isMarkedForDelete(now))
              return true;
  
 -        ByteBuffer liveValue = liveColumn.value();
 -        return columnDef.getValidator().compare(entry.indexValue.key, liveValue) != 0;
 +        ByteBuffer liveValue = liveCell.value();
 +        return columnDef.type.compare(entry.indexValue.key, liveValue) != 0;
      }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
index 41c9f41,eb618f4..e09b2d1
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
@@@ -24,17 -24,14 +24,14 @@@ import java.util.*
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
 -import org.apache.cassandra.cql3.ColumnNameBuilder;
  import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.composites.CellNameType;
 +import org.apache.cassandra.db.composites.Composite;
 +import org.apache.cassandra.db.composites.Composites;
- import org.apache.cassandra.db.filter.ExtendedFilter;
- import org.apache.cassandra.db.filter.IDiskAtomFilter;
- import org.apache.cassandra.db.filter.QueryFilter;
- import org.apache.cassandra.db.filter.SliceQueryFilter;
+ import org.apache.cassandra.db.filter.*;
  import org.apache.cassandra.db.index.SecondaryIndexManager;
  import org.apache.cassandra.db.index.SecondaryIndexSearcher;
 -import org.apache.cassandra.db.marshal.CompositeType;
  import org.apache.cassandra.dht.AbstractBounds;
 -import org.apache.cassandra.thrift.IndexExpression;
  import org.apache.cassandra.utils.ByteBufferUtil;
  
  public class CompositesSearcher extends SecondaryIndexSearcher
@@@ -249,11 -235,28 +246,21 @@@
  
                          // We always query the whole CQL3 row. In the case where the original filter was a name filter this might be
                          // slightly wasteful, but this probably doesn't matter in practice and it simplify things.
-                         SliceQueryFilter dataFilter = new SliceQueryFilter(start,
-                                                                            entry.indexedEntryPrefix.end(),
-                                                                            false,
-                                                                            Integer.MAX_VALUE,
-                                                                            baseCfs.metadata.clusteringColumns().size());
 -                        ColumnSlice dataSlice = new ColumnSlice(start, entry.indexedEntryEnd());
 -                        ColumnSlice[] slices;
 -                        if (baseCfs.metadata.hasStaticColumns())
 -                        {
 -                            // If the table has static columns, we must fetch them too as they may need to be returned too.
 -                            // Note that this is potentially wasteful for 2 reasons:
 -                            //  1) we will retrieve the static parts for each indexed row, even if we have more than one row in
 -                            //     the same partition. If we were to group data queries to rows on the same slice, which would
 -                            //     speed up things in general, we would also optimize here since we would fetch static columns only
 -                            //     once for each group.
 -                            //  2) at this point we don't know if the user asked for static columns or not, so we might be fetching
 -                            //     them for nothing. We would however need to ship the list of "CQL3 columns selected" with getRangeSlice
 -                            //     to be able to know that.
 -                            // TODO: we should improve both point above
 -                            ColumnSlice staticSlice = new ColumnSlice(ByteBufferUtil.EMPTY_BYTE_BUFFER, baseCfs.metadata.getStaticColumnNameBuilder().buildAsEndOfRange());
 -                            slices = new ColumnSlice[]{ staticSlice, dataSlice };
 -                        }
 -                        else
 -                        {
 -                            slices = new ColumnSlice[]{ dataSlice };
 -                        }
 -                        SliceQueryFilter dataFilter = new SliceQueryFilter(slices, false, Integer.MAX_VALUE, baseCfs.metadata.clusteringKeyColumns().size());
++                        ColumnSlice dataSlice = new ColumnSlice(start, entry.indexedEntryPrefix.end());
++                        // If the table has static columns, we must fetch them too as they may need to be returned too.
++                        // Note that this is potentially wasteful for 2 reasons:
++                        //  1) we will retrieve the static parts for each indexed row, even if we have more than one row in
++                        //     the same partition. If we were to group data queries to rows on the same slice, which would
++                        //     speed up things in general, we would also optimize here since we would fetch static columns only
++                        //     once for each group.
++                        //  2) at this point we don't know if the user asked for static columns or not, so we might be fetching
++                        //     them for nothing. We would however need to ship the list of "CQL3 columns selected" with getRangeSlice
++                        //     to be able to know that.
++                        // TODO: we should improve both point above
++                        ColumnSlice[] slices = baseCfs.metadata.hasStaticColumns()
++                                             ? new ColumnSlice[]{ baseCfs.metadata.comparator.staticPrefix().slice(), dataSlice }
++                                             : new ColumnSlice[]{ dataSlice };
++                        SliceQueryFilter dataFilter = new SliceQueryFilter(slices, false, Integer.MAX_VALUE, baseCfs.metadata.clusteringColumns().size());
                          ColumnFamily newData = baseCfs.getColumnFamily(new QueryFilter(dk, baseCfs.name, dataFilter, filter.timestamp));
                          if (newData == null || index.isStale(entry, newData, filter.timestamp))
                          {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java
index be66d21,9250b0f..3184741
--- a/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java
@@@ -34,15 -34,23 +34,23 @@@ import java.util.List
   */
  public abstract class AbstractCompositeType extends AbstractType<ByteBuffer>
  {
+ 
      // changes bb position
 -    protected static int getShortLength(ByteBuffer bb)
 +    public static int getShortLength(ByteBuffer bb)
      {
          int length = (bb.get() & 0xFF) << 8;
          return length | (bb.get() & 0xFF);
      }
  
+     // Doesn't change bb position
+     protected static int getShortLength(ByteBuffer bb, int position)
+     {
+         int length = (bb.get(position) & 0xFF) << 8;
+         return length | (bb.get(position + 1) & 0xFF);
+     }
+ 
      // changes bb position
 -    protected static void putShortLength(ByteBuffer bb, int length)
 +    public static void putShortLength(ByteBuffer bb, int length)
      {
          bb.put((byte) ((length >> 8) & 0xFF));
          bb.put((byte) (length & 0xFF));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/marshal/CompositeType.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
index 25ffc29,65e3be1..af88853
--- a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
@@@ -25,8 -25,8 +25,9 @@@ import java.nio.ByteBuffer
  import java.nio.charset.CharacterCodingException;
  import java.util.*;
  
+ import com.google.common.collect.Iterables;
  
 +import org.apache.cassandra.db.Cell;
  import org.apache.cassandra.exceptions.ConfigurationException;
  import org.apache.cassandra.exceptions.SyntaxException;
  import org.apache.cassandra.auth.IAuthenticator;
@@@ -675,12 -674,12 +676,12 @@@ public abstract class AbstractCassandra
                  return columnDefs;
  
              // otherwise for CqlStorage, check metadata for classic thrift tables
 -            CFDefinition cfDefinition = getCfDefinition(keyspace, column_family, client);
 -            for (CFDefinition.Name column : Iterables.concat(cfDefinition.staticColumns(), cfDefinition.regularColumns()))
 +            CFMetaData cfm = getCFMetaData(keyspace, column_family, client);
-             for (ColumnDefinition def : cfm.regularColumns())
++            for (ColumnDefinition def : cfm.regularAndStaticColumns())
              {
                  ColumnDef cDef = new ColumnDef();
 -                String columnName = column.name.toString();
 -                String type = column.type.toString();
 +                String columnName = def.name.toString();
 +                String type = def.type.toString();
                  logger.debug("name: {}, type: {} ", columnName, type);
                  cDef.name = ByteBufferUtil.bytes(columnName);
                  cDef.validation_class = type;


[2/7] git commit: Add static columns in CQL3

Posted by sl...@apache.org.
Add static columns in CQL3

patch by slebresne; reviewed by iamaleksey for CASSANDRA-6561


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

Branch: refs/heads/trunk
Commit: b09d876914ad9c9fdf1af35cf48cdb98c27bbf32
Parents: b2b3055
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Jan 9 18:44:21 2014 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Feb 20 10:15:02 2014 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 doc/cql3/CQL.textile                            |  29 +-
 .../org/apache/cassandra/config/CFMetaData.java |  35 +-
 .../cassandra/config/ColumnDefinition.java      |   9 +-
 .../org/apache/cassandra/cql3/CFDefinition.java | 123 +++++--
 .../apache/cassandra/cql3/ColumnCondition.java  | 191 +++++++++++
 .../org/apache/cassandra/cql3/Constants.java    |   5 +-
 src/java/org/apache/cassandra/cql3/Cql.g        |  29 +-
 src/java/org/apache/cassandra/cql3/Lists.java   |   7 +-
 src/java/org/apache/cassandra/cql3/Maps.java    |   8 +-
 .../org/apache/cassandra/cql3/Operation.java    |  19 ++
 src/java/org/apache/cassandra/cql3/Sets.java    |   6 +-
 .../cassandra/cql3/functions/TokenFct.java      |   4 +-
 .../cql3/statements/AlterTableStatement.java    |  32 +-
 .../cql3/statements/BatchStatement.java         | 158 +++++++--
 .../cql3/statements/CQL3CasConditions.java      | 164 +++++++++
 .../cql3/statements/ColumnGroupMap.java         |  29 +-
 .../cql3/statements/CreateIndexStatement.java   |   9 +
 .../cql3/statements/CreateTableStatement.java   |  39 ++-
 .../cql3/statements/DeleteStatement.java        |  33 +-
 .../cql3/statements/ModificationStatement.java  | 331 +++++++++++--------
 .../cql3/statements/SelectStatement.java        | 258 ++++++++++++---
 .../cassandra/cql3/statements/Selection.java    |  33 +-
 .../cql3/statements/UpdateStatement.java        |  20 +-
 .../apache/cassandra/db/filter/ColumnSlice.java |   1 -
 .../db/index/composites/CompositesSearcher.java |  32 +-
 .../db/marshal/AbstractCompositeType.java       |  47 +--
 .../cassandra/db/marshal/CompositeType.java     |  72 +++-
 .../db/marshal/DynamicCompositeType.java        |   6 +
 .../hadoop/pig/AbstractCassandraStorage.java    |  11 +-
 .../apache/cassandra/hadoop/pig/CqlStorage.java |   8 +-
 .../apache/cassandra/service/CASConditions.java |   3 +-
 .../cassandra/thrift/ThriftValidation.java      |   3 +-
 33 files changed, 1384 insertions(+), 371 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 4fffb9a..bbacc4d 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -18,6 +18,7 @@
  * Fix count(*) queries in a mixed cluster (CASSANDRA-6707)
  * Improve repair tasks(snapshot, differencing) concurrency (CASSANDRA-6566)
  * Fix replaying pre-2.0 commit logs (CASSANDRA-6714)
+ * Add static columns to CQL3 (CASSANDRA-6561)
 Merged from 1.2:
  * Catch memtable flush exceptions during shutdown (CASSANDRA-6735)
  * Fix broken streams when replacing with same IP (CASSANDRA-6622)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/doc/cql3/CQL.textile
----------------------------------------------------------------------
diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile
index 03b95e0..d872bde 100644
--- a/doc/cql3/CQL.textile
+++ b/doc/cql3/CQL.textile
@@ -219,7 +219,7 @@ bc(syntax)..
                           '(' <definition> ( ',' <definition> )* ')'
                           ( WITH <option> ( AND <option>)* )?
 
-<column-definition> ::= <identifier> <type> ( PRIMARY KEY )?
+<column-definition> ::= <identifier> <type> ( STATIC )? ( PRIMARY KEY )?
                       | PRIMARY KEY '(' <partition-key> ( ',' <identifier> )* ')'
 
 <partition-key> ::= <partition-key>
@@ -288,11 +288,35 @@ In CQL, the order in which columns are defined for the @PRIMARY KEY@ matters. Th
 
 The remaining columns of the @PRIMARY KEY@ definition, if any, are called __clustering columns. On a given physical node, rows for a given partition key are stored in the order induced by the clustering columns, making the retrieval of rows in that clustering order particularly efficient (see <a href="#selectStmt"><tt>SELECT</tt></a>).
 
+h4(#createTableStatic). @STATIC@ columns
+
+Some columns can be declared as @STATIC@ in a table definition. A column that is static will be "shared" by all the rows belonging to the same partition (having the same partition key). For instance, in:
+
+bc(sample). 
+CREATE TABLE test (
+    pk int,
+    t int,
+    v text,
+    s text static,
+    PRIMARY KEY (pk, t)
+);
+INSERT INTO test(pk, t, v, s) VALUES (0, 0, 'val0', 'static0');
+INSERT INTO test(pk, t, v, s) VALUES (0, 1, 'val1', 'static1');
+SELECT * FROM test WHERE pk=0 AND t=0;
+
+the last query will return @'static1'@ as value for @s@, since @s@ is static and thus the 2nd insertion modified this "shared" value. Note however that static columns are only static within a given partition, and if in the example above both rows where from different partitions (i.e. if they had different value for @pk@), then the 2nd insertion would not have modified the value of @s@ for the first row.
+
+A few restrictions applies to when static columns are allowed:
+* tables with the @COMPACT STORAGE@ option (see below) cannot have them
+* a table without clustering columns cannot have static columns (in a table without clustering columns, every partition has only one row, and so every column is inherently static).
+* only non @PRIMARY KEY@ columns can be static
+
+
 h4(#createTableOptions). @<option>@
 
 The @CREATE TABLE@ statement supports a number of options that controls the configuration of a new table. These options can be specified after the @WITH@ keyword.
 
-The first of these option is @COMPACT STORAGE@. This option is mainly targeted towards backward compatibility for definitions created before CQL3 (see "www.datastax.com/dev/blog/thrift-to-cql3":http://www.datastax.com/dev/blog/thrift-to-cql3 for more details).  The option also provides a slightly more compact layout of data on disk but at the price of diminished flexibility and extensibility for the table.  Most notably, @COMPACT STORAGE@ tables cannot have collections and a @COMPACT STORAGE@ table with at least one clustering column supports exactly one (as in not 0 nor more than 1) column not part of the @PRIMARY KEY@ definition (which imply in particular that you cannot add nor remove columns after creation). For those reasons, @COMPACT STORAGE@ is not recommended outside of the backward compatibility reason evoked above.
+The first of these option is @COMPACT STORAGE@. This option is mainly targeted towards backward compatibility for definitions created before CQL3 (see "www.datastax.com/dev/blog/thrift-to-cql3":http://www.datastax.com/dev/blog/thrift-to-cql3 for more details).  The option also provides a slightly more compact layout of data on disk but at the price of diminished flexibility and extensibility for the table.  Most notably, @COMPACT STORAGE@ tables cannot have collections nor static columns and a @COMPACT STORAGE@ table with at least one clustering column supports exactly one (as in not 0 nor more than 1) column not part of the @PRIMARY KEY@ definition (which imply in particular that you cannot add nor remove columns after creation). For those reasons, @COMPACT STORAGE@ is not recommended outside of the backward compatibility reason evoked above.
 
 Another option is @CLUSTERING ORDER@. It allows to define the ordering of rows on disk. It takes the list of the clustering column names with, for each of them, the on-disk order (Ascending or descending). Note that this option affects "what @ORDER BY@ are allowed during @SELECT@":#selectOrderBy.
 
@@ -1116,6 +1140,7 @@ The following describes the addition/changes brought for each version of CQL.
 h3. 3.1.5
 
 * It is now possible to group clustering columns in a relatiion, see "SELECT Where clauses":#selectWhere.
+* Added support for @STATIC@ columns, see "static in CREATE TABLE":#createTableStatic.
 
 h3. 3.1.4
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/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 714a8bc..a319930 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -27,6 +27,7 @@ import java.util.*;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Objects;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.MapDifference;
 import com.google.common.collect.Maps;
 import org.apache.commons.lang3.ArrayUtils;
@@ -408,6 +409,7 @@ public final class CFMetaData
     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
     private volatile Set<ColumnDefinition> regularColumns;
+    private volatile Set<ColumnDefinition> staticColumns;
     private volatile ColumnDefinition compactValueColumn;
 
     public volatile Class<? extends AbstractCompactionStrategy> compactionStrategyClass = DEFAULT_COMPACTION_STRATEGY_CLASS;
@@ -721,6 +723,16 @@ public final class CFMetaData
         return regularColumns;
     }
 
+    public Set<ColumnDefinition> staticColumns()
+    {
+        return staticColumns;
+    }
+
+    public Iterable<ColumnDefinition> regularAndStaticColumns()
+    {
+        return Iterables.concat(staticColumns, regularColumns);
+    }
+
     public ColumnDefinition compactValueColumn()
     {
         return compactValueColumn;
@@ -1328,7 +1340,7 @@ public final class CFMetaData
         // Mixing counter with non counter columns is not supported (#2614)
         if (defaultValidator instanceof CounterColumnType)
         {
-            for (ColumnDefinition def : regularColumns)
+            for (ColumnDefinition def : regularAndStaticColumns())
                 if (!(def.getValidator() instanceof CounterColumnType))
                     throw new ConfigurationException("Cannot add a non counter column (" + getColumnDefinitionComparator(def).getString(def.name) + ") in a counter column family");
         }
@@ -1839,7 +1851,7 @@ public final class CFMetaData
         if (column_metadata.get(to) != null)
             throw new InvalidRequestException(String.format("Cannot rename column %s to %s in keyspace %s; another column of that name already exist", strFrom, strTo, cfName));
 
-        if (def.type == ColumnDefinition.Type.REGULAR)
+        if (def.type == ColumnDefinition.Type.REGULAR || def.type == ColumnDefinition.Type.STATIC)
         {
             throw new InvalidRequestException(String.format("Cannot rename non PRIMARY KEY part %s", strFrom));
         }
@@ -1883,6 +1895,7 @@ public final class CFMetaData
                      : comparator.componentsCount() - (hasCollection() ? 2 : 1);
         List<ColumnDefinition> ckCols = nullInitializedList(nbCkCols);
         Set<ColumnDefinition> regCols = new HashSet<ColumnDefinition>();
+        Set<ColumnDefinition> statCols = new HashSet<ColumnDefinition>();
         ColumnDefinition compactCol = null;
 
         for (ColumnDefinition def : column_metadata.values())
@@ -1900,6 +1913,9 @@ public final class CFMetaData
                 case REGULAR:
                     regCols.add(def);
                     break;
+                case STATIC:
+                    statCols.add(def);
+                    break;
                 case COMPACT_VALUE:
                     assert compactCol == null : "There shouldn't be more than one compact value defined: got " + compactCol + " and " + def;
                     compactCol = def;
@@ -1911,6 +1927,7 @@ public final class CFMetaData
         partitionKeyColumns = addDefaultKeyAliases(pkCols);
         clusteringKeyColumns = addDefaultColumnAliases(ckCols);
         regularColumns = regCols;
+        staticColumns = statCols;
         compactValueColumn = addDefaultValueAlias(compactCol, isDense);
     }
 
@@ -2074,6 +2091,20 @@ public final class CFMetaData
         return true;
     }
 
+    public boolean hasStaticColumns()
+    {
+        return !staticColumns.isEmpty();
+    }
+
+    public ColumnNameBuilder getStaticColumnNameBuilder()
+    {
+        assert comparator instanceof CompositeType && clusteringKeyColumns().size() > 0;
+        CompositeType.Builder builder = CompositeType.Builder.staticBuilder((CompositeType)comparator);
+        for (int i = 0; i < clusteringKeyColumns().size(); i++)
+            builder.add(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+        return builder;
+    }
+
     public void validateColumns(Iterable<Column> columns)
     {
         for (Column column : columns)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/config/ColumnDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/ColumnDefinition.java b/src/java/org/apache/cassandra/config/ColumnDefinition.java
index 7ca4d45..11340e7 100644
--- a/src/java/org/apache/cassandra/config/ColumnDefinition.java
+++ b/src/java/org/apache/cassandra/config/ColumnDefinition.java
@@ -64,7 +64,8 @@ public class ColumnDefinition
         PARTITION_KEY,
         CLUSTERING_KEY,
         REGULAR,
-        COMPACT_VALUE
+        COMPACT_VALUE,
+        STATIC
     }
 
     public final ByteBuffer name;
@@ -96,6 +97,11 @@ public class ColumnDefinition
         return new ColumnDefinition(name, validator, componentIndex, Type.REGULAR);
     }
 
+    public static ColumnDefinition staticDef(ByteBuffer name, AbstractType<?> validator, Integer componentIndex)
+    {
+        return new ColumnDefinition(name, validator, componentIndex, Type.STATIC);
+    }
+
     public static ColumnDefinition compactValueDef(ByteBuffer name, AbstractType<?> validator)
     {
         return new ColumnDefinition(name, validator, null, Type.COMPACT_VALUE);
@@ -174,6 +180,7 @@ public class ColumnDefinition
 
     public boolean isThriftCompatible()
     {
+        // componentIndex == null should always imply isStatic in practice, but there is no harm in being too careful here.
         return type == ColumnDefinition.Type.REGULAR && componentIndex == null;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/cql3/CFDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/CFDefinition.java b/src/java/org/apache/cassandra/cql3/CFDefinition.java
index 638770d..b589a95 100644
--- a/src/java/org/apache/cassandra/cql3/CFDefinition.java
+++ b/src/java/org/apache/cassandra/cql3/CFDefinition.java
@@ -42,11 +42,12 @@ public class CFDefinition implements Iterable<CFDefinition.Name>
 
     public final CFMetaData cfm;
     // LinkedHashMap because the order does matter (it is the order in the composite type)
-    public final LinkedHashMap<ColumnIdentifier, Name> keys = new LinkedHashMap<ColumnIdentifier, Name>();
-    public final LinkedHashMap<ColumnIdentifier, Name> columns = new LinkedHashMap<ColumnIdentifier, Name>();
-    public final Name value;
+    private final LinkedHashMap<ColumnIdentifier, Name> partitionKeys = new LinkedHashMap<ColumnIdentifier, Name>();
+    private final LinkedHashMap<ColumnIdentifier, Name> clusteringColumns = new LinkedHashMap<ColumnIdentifier, Name>();
+    private final Name compactValue;
     // Keep metadata lexicographically ordered so that wildcard expansion have a deterministic order
-    public final Map<ColumnIdentifier, Name> metadata = new TreeMap<ColumnIdentifier, Name>();
+    private final Map<ColumnIdentifier, Name> staticColumns = new TreeMap<ColumnIdentifier, Name>();
+    private final Map<ColumnIdentifier, Name> regularColumns = new TreeMap<ColumnIdentifier, Name>();
 
     public final boolean isComposite;
     public final boolean hasCompositeKey;
@@ -65,7 +66,7 @@ public class CFDefinition implements Iterable<CFDefinition.Name>
         for (int i = 0; i < cfm.partitionKeyColumns().size(); ++i)
         {
             ColumnIdentifier id = new ColumnIdentifier(cfm.partitionKeyColumns().get(i).name, definitionType);
-            this.keys.put(id, new Name(cfm.ksName, cfm.cfName, id, Name.Kind.KEY_ALIAS, i, cfm.getKeyValidator().getComponents().get(i)));
+            this.partitionKeys.put(id, new Name(cfm.ksName, cfm.cfName, id, Name.Kind.KEY_ALIAS, i, cfm.getKeyValidator().getComponents().get(i)));
         }
 
         this.isComposite = cfm.comparator instanceof CompositeType;
@@ -74,20 +75,25 @@ public class CFDefinition implements Iterable<CFDefinition.Name>
         for (int i = 0; i < cfm.clusteringKeyColumns().size(); ++i)
         {
             ColumnIdentifier id = new ColumnIdentifier(cfm.clusteringKeyColumns().get(i).name, definitionType);
-            this.columns.put(id, new Name(cfm.ksName, cfm.cfName, id, Name.Kind.COLUMN_ALIAS, i, cfm.comparator.getComponents().get(i)));
+            this.clusteringColumns.put(id, new Name(cfm.ksName, cfm.cfName, id, Name.Kind.COLUMN_ALIAS, i, cfm.comparator.getComponents().get(i)));
         }
 
         if (isCompact)
         {
-            this.value = createValue(cfm);
+            this.compactValue = createValue(cfm);
         }
         else
         {
-            this.value = null;
+            this.compactValue = null;
             for (ColumnDefinition def : cfm.regularColumns())
             {
                 ColumnIdentifier id = new ColumnIdentifier(def.name, cfm.getColumnDefinitionComparator(def));
-                this.metadata.put(id, new Name(cfm.ksName, cfm.cfName, id, Name.Kind.COLUMN_METADATA, def.getValidator()));
+                this.regularColumns.put(id, new Name(cfm.ksName, cfm.cfName, id, Name.Kind.COLUMN_METADATA, def.getValidator()));
+            }
+            for (ColumnDefinition def : cfm.staticColumns())
+            {
+                ColumnIdentifier id = new ColumnIdentifier(def.name, cfm.getColumnDefinitionComparator(def));
+                this.staticColumns.put(id, new Name(cfm.ksName, cfm.cfName, id, Name.Kind.STATIC, def.getValidator()));
             }
         }
     }
@@ -111,44 +117,86 @@ public class CFDefinition implements Iterable<CFDefinition.Name>
                : new Name(cfm.ksName, cfm.cfName, alias, Name.Kind.VALUE_ALIAS, cfm.getDefaultValidator());
     }
 
+    public int partitionKeyCount()
+    {
+        return partitionKeys.size();
+    }
+
+    public Collection<Name> partitionKeys()
+    {
+        return partitionKeys.values();
+    }
+
+    public int clusteringColumnsCount()
+    {
+        return clusteringColumns.size();
+    }
+
+    public Collection<Name> clusteringColumns()
+    {
+        return clusteringColumns.values();
+    }
+
+    public Collection<Name> regularColumns()
+    {
+        return regularColumns.values();
+    }
+
+    public Collection<Name> staticColumns()
+    {
+        return regularColumns.values();
+    }
+
+    public Name compactValue()
+    {
+        return compactValue;
+    }
+
     public Name get(ColumnIdentifier name)
     {
-        CFDefinition.Name kdef = keys.get(name);
-        if (kdef != null)
-            return kdef;
-        if (value != null && name.equals(value.name))
-            return value;
-        CFDefinition.Name def = columns.get(name);
+        CFDefinition.Name def = partitionKeys.get(name);
         if (def != null)
             return def;
-        return metadata.get(name);
+        if (compactValue != null && name.equals(compactValue.name))
+            return compactValue;
+        def = clusteringColumns.get(name);
+        if (def != null)
+            return def;
+        def = regularColumns.get(name);
+        if (def != null)
+            return def;
+        return staticColumns.get(name);
     }
 
     public Iterator<Name> iterator()
     {
         return new AbstractIterator<Name>()
         {
-            private final Iterator<Name> keyIter = keys.values().iterator();
-            private final Iterator<Name> columnIter = columns.values().iterator();
+            private final Iterator<Name> keyIter = partitionKeys.values().iterator();
+            private final Iterator<Name> clusteringIter = clusteringColumns.values().iterator();
             private boolean valueDone;
-            private final Iterator<Name> metadataIter = metadata.values().iterator();
+            private final Iterator<Name> staticIter = staticColumns.values().iterator();
+            private final Iterator<Name> regularIter = regularColumns.values().iterator();
 
             protected Name computeNext()
             {
                 if (keyIter.hasNext())
                     return keyIter.next();
 
-                if (columnIter.hasNext())
-                    return columnIter.next();
+                if (clusteringIter.hasNext())
+                    return clusteringIter.next();
 
-                if (value != null && !valueDone)
+                if (compactValue != null && !valueDone)
                 {
                     valueDone = true;
-                    return value;
+                    return compactValue;
                 }
 
-                if (metadataIter.hasNext())
-                    return metadataIter.next();
+                if (staticIter.hasNext())
+                    return staticIter.next();
+
+                if (regularIter.hasNext())
+                    return regularIter.next();
 
                 return endOfData();
             }
@@ -173,7 +221,7 @@ public class CFDefinition implements Iterable<CFDefinition.Name>
     {
         public static enum Kind
         {
-            KEY_ALIAS, COLUMN_ALIAS, VALUE_ALIAS, COLUMN_METADATA
+            KEY_ALIAS, COLUMN_ALIAS, VALUE_ALIAS, COLUMN_METADATA, STATIC
         }
 
         private Name(String ksName, String cfName, ColumnIdentifier name, Kind kind, AbstractType<?> type)
@@ -210,20 +258,29 @@ public class CFDefinition implements Iterable<CFDefinition.Name>
         {
             return Objects.hashCode(ksName, cfName, name, type, kind, position);
         }
+
+        public boolean isPrimaryKeyColumn()
+        {
+            return kind == Kind.KEY_ALIAS || kind == Kind.COLUMN_ALIAS;
+        }
     }
 
     @Override
     public String toString()
     {
         StringBuilder sb = new StringBuilder();
-        sb.append(Joiner.on(", ").join(keys.values()));
-        if (!columns.isEmpty())
-            sb.append(", ").append(Joiner.on(", ").join(columns.values()));
+        sb.append(Joiner.on(", ").join(partitionKeys.values()));
+        if (!clusteringColumns.isEmpty())
+            sb.append(", ").append(Joiner.on(", ").join(clusteringColumns.values()));
         sb.append(" => ");
-        if (value != null)
-            sb.append(value.name);
-        if (!metadata.isEmpty())
-            sb.append("{").append(Joiner.on(", ").join(metadata.values())).append(" }");
+        if (compactValue != null)
+            sb.append(compactValue.name);
+        sb.append("{");
+        sb.append(Joiner.on(", ").join(staticColumns.values()));
+        if (!staticColumns.isEmpty())
+            sb.append(", ");
+        sb.append(Joiner.on(", ").join(regularColumns.values()));
+        sb.append("}");
         return sb.toString();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/cql3/ColumnCondition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ColumnCondition.java b/src/java/org/apache/cassandra/cql3/ColumnCondition.java
new file mode 100644
index 0000000..797dba6
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/ColumnCondition.java
@@ -0,0 +1,191 @@
+/*
+ * 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;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterators;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.filter.ColumnSlice;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.CounterColumnType;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * A CQL3 condition.
+ */
+public class ColumnCondition
+{
+    public final CFDefinition.Name column;
+    private final Term value;
+
+    private List<ByteBuffer> variables;
+
+    private ColumnCondition(CFDefinition.Name column, Term value)
+    {
+        this.column = column;
+        this.value = value;
+    }
+
+    // The only ones we support so far
+    public static ColumnCondition equal(CFDefinition.Name column, Term value)
+    {
+        return new ColumnCondition(column, value);
+    }
+
+    // See CQL3CasConditions for why it's convenient to have this
+    public ColumnCondition attach(List<ByteBuffer> variables)
+    {
+        this.variables = variables;
+        return this;
+    }
+
+    /**
+     * Collects the column specification for the bind variables of this operation.
+     *
+     * @param boundNames the list of column specification where to collect the
+     * bind variables of this term in.
+     */
+    public void collectMarkerSpecification(VariableSpecifications boundNames)
+    {
+        value.collectMarkerSpecification(boundNames);
+    }
+
+    // Not overriding equals() because we need the variables to have been attached when this is
+    // called and so having a non standard method name might help avoid mistakes
+    public boolean equalsTo(ColumnCondition other) throws InvalidRequestException
+    {
+        return column.equals(other.column)
+            && value.bindAndGet(variables).equals(other.value.bindAndGet(other.variables));
+    }
+
+    private ColumnNameBuilder copyOrUpdatePrefix(CFMetaData cfm, ColumnNameBuilder rowPrefix)
+    {
+        return column.kind == CFDefinition.Name.Kind.STATIC ? cfm.getStaticColumnNameBuilder() : rowPrefix.copy();
+    }
+
+    /**
+     * Validates whether this condition applies to {@code current}.
+     */
+    public boolean appliesTo(ColumnNameBuilder rowPrefix, ColumnFamily current, long now) throws InvalidRequestException
+    {
+        if (column.type instanceof CollectionType)
+            return collectionAppliesTo((CollectionType)column.type, rowPrefix, current, now);
+
+        Column c = current.getColumn(copyOrUpdatePrefix(current.metadata(), rowPrefix).add(column.name.key).build());
+        ByteBuffer v = value.bindAndGet(variables);
+        return v == null
+             ? c == null || !c.isLive(now)
+             : c != null && c.isLive(now) && c.value().equals(v);
+    }
+
+    private boolean collectionAppliesTo(CollectionType type, ColumnNameBuilder rowPrefix, ColumnFamily current, final long now) throws InvalidRequestException
+    {
+        ColumnNameBuilder collectionPrefix = copyOrUpdatePrefix(current.metadata(), rowPrefix).add(column.name.key);
+        // We are testing for collection equality, so we need to have the expected values *and* only those.
+        ColumnSlice[] collectionSlice = new ColumnSlice[]{ new ColumnSlice(collectionPrefix.build(), collectionPrefix.buildAsEndOfRange()) };
+        // Filter live columns, this makes things simpler afterwards
+        Iterator<Column> iter = Iterators.filter(current.iterator(collectionSlice), new Predicate<Column>()
+        {
+            public boolean apply(Column c)
+            {
+                // we only care about live columns
+                return c.isLive(now);
+            }
+        });
+
+        Term.Terminal v = value.bind(variables);
+        if (v == null)
+            return !iter.hasNext();
+
+        switch (type.kind)
+        {
+            case LIST: return listAppliesTo(current.metadata(), iter, ((Lists.Value)v).elements);
+            case SET: return setAppliesTo(current.metadata(), iter, ((Sets.Value)v).elements);
+            case MAP: return mapAppliesTo(current.metadata(), iter, ((Maps.Value)v).map);
+        }
+        throw new AssertionError();
+    }
+
+    private static ByteBuffer collectionKey(CFMetaData cfm, Column c)
+    {
+        ByteBuffer[] bbs = ((CompositeType)cfm.comparator).split(c.name());
+        return bbs[bbs.length - 1];
+    }
+
+    private boolean listAppliesTo(CFMetaData cfm, Iterator<Column> iter, List<ByteBuffer> elements)
+    {
+        for (ByteBuffer e : elements)
+            if (!iter.hasNext() || iter.next().value().equals(e))
+                return false;
+        // We must not have more elements than expected
+        return !iter.hasNext();
+    }
+
+    private boolean setAppliesTo(CFMetaData cfm, Iterator<Column> iter, Set<ByteBuffer> elements)
+    {
+        Set<ByteBuffer> remaining = new HashSet<>(elements);
+        while (iter.hasNext())
+        {
+            if (remaining.isEmpty())
+                return false;
+
+            if (!remaining.remove(collectionKey(cfm, iter.next())))
+                return false;
+        }
+        return remaining.isEmpty();
+    }
+
+    private boolean mapAppliesTo(CFMetaData cfm, Iterator<Column> iter, Map<ByteBuffer, ByteBuffer> elements)
+    {
+        Map<ByteBuffer, ByteBuffer> remaining = new HashMap<>(elements);
+        while (iter.hasNext())
+        {
+            if (remaining.isEmpty())
+                return false;
+
+            Column c = iter.next();
+            if (!remaining.remove(collectionKey(cfm, c)).equals(c.value()))
+                return false;
+        }
+        return remaining.isEmpty();
+    }
+
+    public static class Raw
+    {
+        private final Term.Raw value;
+
+        public Raw(Term.Raw value)
+        {
+            this.value = value;
+        }
+
+        public ColumnCondition prepare(CFDefinition.Name receiver) throws InvalidRequestException
+        {
+            if (receiver.type instanceof CounterColumnType)
+                throw new InvalidRequestException("Condtions on counters are not supported");
+
+            return ColumnCondition.equal(receiver, value.prepare(receiver));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/cql3/Constants.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Constants.java b/src/java/org/apache/cassandra/cql3/Constants.java
index bcfe00d..f99fd02 100644
--- a/src/java/org/apache/cassandra/cql3/Constants.java
+++ b/src/java/org/apache/cassandra/cql3/Constants.java
@@ -296,6 +296,7 @@ public abstract class Constants
 
         public void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder prefix, UpdateParameters params) throws InvalidRequestException
         {
+            prefix = maybeUpdatePrefix(cf.metadata(), prefix);
             ByteBuffer cname = columnName == null ? prefix.build() : prefix.add(columnName.key).build();
             ByteBuffer value = t.bindAndGet(params.variables);
             cf.addColumn(value == null ? params.makeTombstone(cname) : params.makeColumn(cname, value));
@@ -315,6 +316,7 @@ public abstract class Constants
             if (bytes == null)
                 throw new InvalidRequestException("Invalid null value for counter increment");
             long increment = ByteBufferUtil.toLong(bytes);
+            prefix = maybeUpdatePrefix(cf.metadata(), prefix);
             ByteBuffer cname = columnName == null ? prefix.build() : prefix.add(columnName.key).build();
             cf.addCounter(cname, increment);
         }
@@ -337,6 +339,7 @@ public abstract class Constants
             if (increment == Long.MIN_VALUE)
                 throw new InvalidRequestException("The negation of " + increment + " overflows supported counter precision (signed 8 bytes integer)");
 
+            prefix = maybeUpdatePrefix(cf.metadata(), prefix);
             ByteBuffer cname = columnName == null ? prefix.build() : prefix.add(columnName.key).build();
             cf.addCounter(cname, -increment);
         }
@@ -356,7 +359,7 @@ public abstract class Constants
 
         public void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder prefix, UpdateParameters params) throws InvalidRequestException
         {
-            ColumnNameBuilder column = prefix.add(columnName.key);
+            ColumnNameBuilder column = maybeUpdatePrefix(cf.metadata(), prefix).add(columnName.key);
 
             if (isCollection)
                 cf.addAtom(params.makeRangeTombstone(column.build(), column.buildAsEndOfRange()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Cql.g b/src/java/org/apache/cassandra/cql3/Cql.g
index 6e7cf1c..a11a818 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -351,21 +351,22 @@ updateStatement returns [UpdateStatement.ParsedUpdate expr]
       ( usingClause[attrs] )?
       K_SET columnOperation[operations] (',' columnOperation[operations])*
       K_WHERE wclause=whereClause
-      ( K_IF conditions=updateCondition )?
+      ( K_IF conditions=updateConditions )?
       {
           return new UpdateStatement.ParsedUpdate(cf,
                                                   attrs,
                                                   operations,
                                                   wclause,
-                                                  conditions == null ? Collections.<Pair<ColumnIdentifier, Operation.RawUpdate>>emptyList() : conditions);
+                                                  conditions == null ? Collections.<Pair<ColumnIdentifier, ColumnCondition.Raw>>emptyList() : conditions);
      }
     ;
 
-updateCondition returns [List<Pair<ColumnIdentifier, Operation.RawUpdate>> conditions]
-    @init { conditions = new ArrayList<Pair<ColumnIdentifier, Operation.RawUpdate>>(); }
-    : columnOperation[conditions] ( K_AND columnOperation[conditions] )*
+updateConditions returns [List<Pair<ColumnIdentifier, ColumnCondition.Raw>> conditions]
+    @init { conditions = new ArrayList<Pair<ColumnIdentifier, ColumnCondition.Raw>>(); }
+    : columnCondition[conditions] ( K_AND columnCondition[conditions] )*
     ;
 
+
 /**
  * DELETE name1, name2
  * FROM <CF>
@@ -381,13 +382,13 @@ deleteStatement returns [DeleteStatement.Parsed expr]
       K_FROM cf=columnFamilyName
       ( usingClauseDelete[attrs] )?
       K_WHERE wclause=whereClause
-      ( K_IF conditions=updateCondition )?
+      ( K_IF conditions=updateConditions )?
       {
           return new DeleteStatement.Parsed(cf,
                                             attrs,
                                             columnDeletions,
                                             wclause,
-                                            conditions == null ? Collections.<Pair<ColumnIdentifier, Operation.RawUpdate>>emptyList() : conditions);
+                                            conditions == null ? Collections.<Pair<ColumnIdentifier, ColumnCondition.Raw>>emptyList() : conditions);
       }
     ;
 
@@ -484,7 +485,8 @@ cfamDefinition[CreateTableStatement.RawStatement expr]
     ;
 
 cfamColumns[CreateTableStatement.RawStatement expr]
-    : k=cident v=comparatorType { $expr.addDefinition(k, v); } (K_PRIMARY K_KEY { $expr.addKeyAliases(Collections.singletonList(k)); })?
+    : k=cident v=comparatorType { boolean isStatic=false; } (K_STATIC {isStatic = true;})? { $expr.addDefinition(k, v, isStatic); }
+        (K_PRIMARY K_KEY { $expr.addKeyAliases(Collections.singletonList(k)); })?
     | K_PRIMARY K_KEY '(' pkDef[expr] (',' c=cident { $expr.addColumnAlias(c); } )* ')'
     ;
 
@@ -558,10 +560,11 @@ alterTableStatement returns [AlterTableStatement expr]
         AlterTableStatement.Type type = null;
         CFPropDefs props = new CFPropDefs();
         Map<ColumnIdentifier, ColumnIdentifier> renames = new HashMap<ColumnIdentifier, ColumnIdentifier>();
+        boolean isStatic = false;
     }
     : K_ALTER K_COLUMNFAMILY cf=columnFamilyName
           ( K_ALTER id=cident K_TYPE v=comparatorType { type = AlterTableStatement.Type.ALTER; }
-          | K_ADD   id=cident v=comparatorType        { type = AlterTableStatement.Type.ADD; }
+          | K_ADD   id=cident v=comparatorType ({ isStatic=true; } K_STATIC)? { type = AlterTableStatement.Type.ADD; }
           | K_DROP  id=cident                         { type = AlterTableStatement.Type.DROP; }
           | K_WITH  properties[props]                 { type = AlterTableStatement.Type.OPTS; }
           | K_RENAME                                  { type = AlterTableStatement.Type.RENAME; }
@@ -569,7 +572,7 @@ alterTableStatement returns [AlterTableStatement expr]
                ( K_AND idn=cident K_TO toIdn=cident { renames.put(idn, toIdn); } )*
           )
     {
-        $expr = new AlterTableStatement(cf, type, id, v, props, renames);
+        $expr = new AlterTableStatement(cf, type, id, v, props, renames, isStatic);
     }
     ;
 
@@ -845,6 +848,10 @@ columnOperation[List<Pair<ColumnIdentifier, Operation.RawUpdate>> operations]
       }
     ;
 
+columnCondition[List<Pair<ColumnIdentifier, ColumnCondition.Raw>> conditions]
+    : key=cident '=' t=term { conditions.add(Pair.create(key, new ColumnCondition.Raw(t))); } // Note: we'll reject duplicates later
+    ;
+
 properties[PropertyDefinitions props]
     : property[props] (K_AND property[props])*
     ;
@@ -981,6 +988,7 @@ unreserved_function_keyword returns [String str]
         | K_CUSTOM
         | K_TRIGGER
         | K_DISTINCT
+        | K_STATIC
         ) { $str = $k.text; }
     | t=native_type { $str = t.toString(); }
     ;
@@ -1084,6 +1092,7 @@ K_NAN:         N A N;
 K_INFINITY:    I N F I N I T Y;
 
 K_TRIGGER:     T R I G G E R;
+K_STATIC:      S T A T I C;
 
 // Case-insensitive alpha characters
 fragment A: ('a'|'A');

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/cql3/Lists.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Lists.java b/src/java/org/apache/cassandra/cql3/Lists.java
index 4ca5eb3..4ad39db 100644
--- a/src/java/org/apache/cassandra/cql3/Lists.java
+++ b/src/java/org/apache/cassandra/cql3/Lists.java
@@ -266,7 +266,7 @@ public abstract class Lists
         public void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder prefix, UpdateParameters params) throws InvalidRequestException
         {
             // delete + append
-            ColumnNameBuilder column = prefix.add(columnName.key);
+            ColumnNameBuilder column = maybeUpdatePrefix(cf.metadata(), prefix).add(columnName.key);
             cf.addAtom(params.makeTombstoneForOverwrite(column.build(), column.buildAsEndOfRange()));
             Appender.doAppend(t, cf, column, params);
         }
@@ -309,6 +309,7 @@ public abstract class Lists
                 throw new InvalidRequestException(String.format("List index %d out of bound, list has size %d", idx, existingList.size()));
 
             ByteBuffer elementName = existingList.get(idx).right.name();
+            // Since we reuse the name we're read, if it's a static column, the static marker will already be set
 
             if (value == null)
             {
@@ -336,7 +337,7 @@ public abstract class Lists
 
         public void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder prefix, UpdateParameters params) throws InvalidRequestException
         {
-            doAppend(t, cf, prefix.add(columnName.key), params);
+            doAppend(t, cf, maybeUpdatePrefix(cf.metadata(), prefix).add(columnName.key), params);
         }
 
         static void doAppend(Term t, ColumnFamily cf, ColumnNameBuilder columnName, UpdateParameters params) throws InvalidRequestException
@@ -376,7 +377,7 @@ public abstract class Lists
             long time = PrecisionTime.REFERENCE_TIME - (System.currentTimeMillis() - PrecisionTime.REFERENCE_TIME);
 
             List<ByteBuffer> toAdd = ((Lists.Value)value).elements;
-            ColumnNameBuilder column = prefix.add(columnName.key);
+            ColumnNameBuilder column = maybeUpdatePrefix(cf.metadata(), prefix).add(columnName.key);
             for (int i = 0; i < toAdd.size(); i++)
             {
                 ColumnNameBuilder b = i == toAdd.size() - 1 ? column : column.copy();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/cql3/Maps.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Maps.java b/src/java/org/apache/cassandra/cql3/Maps.java
index 30d796c..c332999 100644
--- a/src/java/org/apache/cassandra/cql3/Maps.java
+++ b/src/java/org/apache/cassandra/cql3/Maps.java
@@ -244,7 +244,7 @@ public abstract class Maps
         public void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder prefix, UpdateParameters params) throws InvalidRequestException
         {
             // delete + put
-            ColumnNameBuilder column = prefix.add(columnName.key);
+            ColumnNameBuilder column = maybeUpdatePrefix(cf.metadata(), prefix).add(columnName.key);
             cf.addAtom(params.makeTombstoneForOverwrite(column.build(), column.buildAsEndOfRange()));
             Putter.doPut(t, cf, column, params);
         }
@@ -274,7 +274,7 @@ public abstract class Maps
             if (key == null)
                 throw new InvalidRequestException("Invalid null map key");
 
-            ByteBuffer cellName = prefix.add(columnName.key).add(key).build();
+            ByteBuffer cellName = maybeUpdatePrefix(cf.metadata(), prefix).add(columnName.key).add(key).build();
 
             if (value == null)
             {
@@ -302,7 +302,7 @@ public abstract class Maps
 
         public void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder prefix, UpdateParameters params) throws InvalidRequestException
         {
-            doPut(t, cf, prefix.add(columnName.key), params);
+            doPut(t, cf, maybeUpdatePrefix(cf.metadata(), prefix).add(columnName.key), params);
         }
 
         static void doPut(Term t, ColumnFamily cf, ColumnNameBuilder columnName, UpdateParameters params) throws InvalidRequestException
@@ -335,7 +335,7 @@ public abstract class Maps
                 throw new InvalidRequestException("Invalid null map key");
             assert key instanceof Constants.Value;
 
-            ByteBuffer cellName = prefix.add(columnName.key).add(((Constants.Value)key).bytes).build();
+            ByteBuffer cellName = maybeUpdatePrefix(cf.metadata(), prefix).add(columnName.key).add(((Constants.Value)key).bytes).build();
             cf.addColumn(params.makeTombstone(cellName));
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/cql3/Operation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Operation.java b/src/java/org/apache/cassandra/cql3/Operation.java
index 00dd046..6bf46b5 100644
--- a/src/java/org/apache/cassandra/cql3/Operation.java
+++ b/src/java/org/apache/cassandra/cql3/Operation.java
@@ -19,6 +19,8 @@ package org.apache.cassandra.cql3;
 
 import java.nio.ByteBuffer;
 
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.marshal.CollectionType;
 import org.apache.cassandra.db.marshal.CounterColumnType;
@@ -54,6 +56,23 @@ public abstract class Operation
         this.t = t;
     }
 
+    // Whether the colum operated on is a static column (on trunk, Operation stores the ColumnDefinition directly,
+    // not just the column name, so we'll be able to remove that lookup and check ColumnDefinition.isStatic field
+    // directly. But for 2.0, it's simpler that way).
+    public boolean isStatic(CFMetaData cfm)
+    {
+        if (columnName == null)
+            return false;
+
+        ColumnDefinition def = cfm.getColumnDefinition(columnName.key);
+        return def != null && def.type == ColumnDefinition.Type.STATIC;
+    }
+
+    protected ColumnNameBuilder maybeUpdatePrefix(CFMetaData cfm, ColumnNameBuilder prefix)
+    {
+        return isStatic(cfm) ? cfm.getStaticColumnNameBuilder() : prefix;
+    }
+
     /**
      * @return whether the operation requires a read of the previous value to be executed
      * (only lists setterByIdx, discard and discardByIdx requires that).

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/cql3/Sets.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Sets.java b/src/java/org/apache/cassandra/cql3/Sets.java
index 0fcb8bf..69bc3d3 100644
--- a/src/java/org/apache/cassandra/cql3/Sets.java
+++ b/src/java/org/apache/cassandra/cql3/Sets.java
@@ -230,7 +230,7 @@ public abstract class Sets
         public void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder prefix, UpdateParameters params) throws InvalidRequestException
         {
             // delete + add
-            ColumnNameBuilder column = prefix.add(columnName.key);
+            ColumnNameBuilder column = maybeUpdatePrefix(cf.metadata(), prefix).add(columnName.key);
             cf.addAtom(params.makeTombstoneForOverwrite(column.build(), column.buildAsEndOfRange()));
             Adder.doAdd(t, cf, column, params);
         }
@@ -245,7 +245,7 @@ public abstract class Sets
 
         public void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder prefix, UpdateParameters params) throws InvalidRequestException
         {
-            doAdd(t, cf, prefix.add(columnName.key), params);
+            doAdd(t, cf, maybeUpdatePrefix(cf.metadata(), prefix).add(columnName.key), params);
         }
 
         static void doAdd(Term t, ColumnFamily cf, ColumnNameBuilder columnName, UpdateParameters params) throws InvalidRequestException
@@ -283,7 +283,7 @@ public abstract class Sets
                                       ? Collections.singleton(((Constants.Value)value).bytes)
                                       : ((Sets.Value)value).elements;
 
-            ColumnNameBuilder column = prefix.add(columnName.key);
+            ColumnNameBuilder column = maybeUpdatePrefix(cf.metadata(), prefix).add(columnName.key);
             for (ByteBuffer bb : toDiscard)
             {
                 ByteBuffer cellName = column.copy().add(bb).build();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/cql3/functions/TokenFct.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/TokenFct.java b/src/java/org/apache/cassandra/cql3/functions/TokenFct.java
index ac6b999..4f3ff4a 100644
--- a/src/java/org/apache/cassandra/cql3/functions/TokenFct.java
+++ b/src/java/org/apache/cassandra/cql3/functions/TokenFct.java
@@ -52,9 +52,9 @@ public class TokenFct extends AbstractFunction
 
     private static AbstractType[] getKeyTypes(CFMetaData cfm)
     {
-        AbstractType[] types = new AbstractType[cfm.getCfDef().keys.size()];
+        AbstractType[] types = new AbstractType[cfm.getCfDef().partitionKeyCount()];
         int i = 0;
-        for (CFDefinition.Name name : cfm.getCfDef().keys.values())
+        for (CFDefinition.Name name : cfm.getCfDef().partitionKeys())
             types[i++] = name.type;
         return types;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
index f740ea6..85b3547 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
@@ -47,8 +47,15 @@ public class AlterTableStatement extends SchemaAlteringStatement
     public final ColumnIdentifier columnName;
     private final CFPropDefs cfProps;
     private final Map<ColumnIdentifier, ColumnIdentifier> renames;
+    private final boolean isStatic; // Only for ALTER ADD
 
-    public AlterTableStatement(CFName name, Type type, ColumnIdentifier columnName, CQL3Type validator, CFPropDefs cfProps, Map<ColumnIdentifier, ColumnIdentifier> renames)
+    public AlterTableStatement(CFName name,
+                               Type type,
+                               ColumnIdentifier columnName,
+                               CQL3Type validator,
+                               CFPropDefs cfProps,
+                               Map<ColumnIdentifier, ColumnIdentifier> renames,
+                               boolean isStatic)
     {
         super(name);
         this.oType = type;
@@ -56,6 +63,7 @@ public class AlterTableStatement extends SchemaAlteringStatement
         this.validator = validator; // used only for ADD/ALTER commands
         this.cfProps = cfProps;
         this.renames = renames;
+        this.isStatic = isStatic;
     }
 
     public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
@@ -79,7 +87,11 @@ public class AlterTableStatement extends SchemaAlteringStatement
         {
             case ADD:
                 if (cfDef.isCompact)
-                    throw new InvalidRequestException("Cannot add new column to a compact CF");
+                    throw new InvalidRequestException("Cannot add new column to a COMPACT STORAGE table");
+
+                if (isStatic && !cfDef.isComposite)
+                    throw new InvalidRequestException("Static columns are not allowed in COMPACT STORAGE tables");
+
                 if (name != null)
                 {
                     switch (name.kind)
@@ -87,7 +99,7 @@ public class AlterTableStatement extends SchemaAlteringStatement
                         case KEY_ALIAS:
                         case COLUMN_ALIAS:
                             throw new InvalidRequestException(String.format("Invalid column name %s because it conflicts with a PRIMARY KEY part", columnName));
-                        case COLUMN_METADATA:
+                        default:
                             throw new InvalidRequestException(String.format("Invalid column name %s because it conflicts with an existing column", columnName));
                     }
                 }
@@ -117,7 +129,9 @@ public class AlterTableStatement extends SchemaAlteringStatement
                 Integer componentIndex = cfDef.isComposite
                                        ? ((CompositeType)meta.comparator).types.size() - (cfDef.hasCollections ? 2 : 1)
                                        : null;
-                cfm.addColumnDefinition(ColumnDefinition.regularDef(columnName.key, type, componentIndex));
+                cfm.addColumnDefinition(isStatic
+                                        ? ColumnDefinition.staticDef(columnName.key, type, componentIndex)
+                                        : ColumnDefinition.regularDef(columnName.key, type, componentIndex));
                 break;
 
             case ALTER:
@@ -178,6 +192,7 @@ public class AlterTableStatement extends SchemaAlteringStatement
                         cfm.defaultValidator(validator.getType());
                         break;
                     case COLUMN_METADATA:
+                    case STATIC:
                         ColumnDefinition column = cfm.getColumnDefinition(columnName.key);
                         // Thrift allows to change a column validator so CFMetaData.validateCompatibility will let it slide
                         // if we change to an incompatible type (contrarily to the comparator case). But we don't want to
@@ -196,10 +211,8 @@ public class AlterTableStatement extends SchemaAlteringStatement
                 break;
 
             case DROP:
-                if (cfDef.isCompact)
-                    throw new InvalidRequestException("Cannot drop columns from a compact CF");
-                if (!cfDef.isComposite)
-                    throw new InvalidRequestException("Cannot drop columns from a non-CQL3 CF");
+                if (cfDef.isCompact || !cfDef.isComposite)
+                    throw new InvalidRequestException("Cannot drop columns from a COMPACT STORAGE table");
                 if (name == null)
                     throw new InvalidRequestException(String.format("Column %s was not found in table %s", columnName, columnFamily()));
 
@@ -209,8 +222,9 @@ public class AlterTableStatement extends SchemaAlteringStatement
                     case COLUMN_ALIAS:
                         throw new InvalidRequestException(String.format("Cannot drop PRIMARY KEY part %s", columnName));
                     case COLUMN_METADATA:
+                    case STATIC:
                         ColumnDefinition toDelete = null;
-                        for (ColumnDefinition columnDef : cfm.regularColumns())
+                        for (ColumnDefinition columnDef : cfm.regularAndStaticColumns())
                         {
                             if (columnDef.name.equals(columnName.key))
                                 toDelete = columnDef;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index 6151490..d4acbae 100644
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@ -20,11 +20,11 @@ package org.apache.cassandra.cql3.statements;
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import com.google.common.collect.Iterables;
 import org.github.jamm.MemoryMeter;
 
 import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.db.ConsistencyLevel;
-import org.apache.cassandra.db.IMutation;
+import org.apache.cassandra.db.*;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
@@ -47,6 +47,7 @@ public class BatchStatement implements CQLStatement, MeasurableForPreparedCache
     public final Type type;
     private final List<ModificationStatement> statements;
     private final Attributes attrs;
+    private final boolean hasConditions;
 
     /**
      * Creates a new BatchStatement from a list of statements and a
@@ -58,10 +59,16 @@ public class BatchStatement implements CQLStatement, MeasurableForPreparedCache
      */
     public BatchStatement(int boundTerms, Type type, List<ModificationStatement> statements, Attributes attrs)
     {
+        this(boundTerms, type, statements, attrs, false);
+    }
+
+    public BatchStatement(int boundTerms, Type type, List<ModificationStatement> statements, Attributes attrs, boolean hasConditions)
+    {
         this.boundTerms = boundTerms;
         this.type = type;
         this.statements = statements;
         this.attrs = attrs;
+        this.hasConditions = hasConditions;
     }
 
     public long measureForPreparedCache(MemoryMeter meter)
@@ -103,25 +110,15 @@ public class BatchStatement implements CQLStatement, MeasurableForPreparedCache
         return statements;
     }
 
-    private Collection<? extends IMutation> getMutations(List<ByteBuffer> variables, boolean local, ConsistencyLevel cl, long now)
-    throws RequestExecutionException, RequestValidationException
-    {
-        Map<Pair<String, ByteBuffer>, IMutation> mutations = new HashMap<Pair<String, ByteBuffer>, IMutation>();
-        for (ModificationStatement statement : statements)
-            addStatementMutations(statement, variables, local, cl, now, mutations);
-
-        return mutations.values();
-    }
-
-    private Collection<? extends IMutation> getMutations(List<List<ByteBuffer>> variables, ConsistencyLevel cl, long now)
+    private Collection<? extends IMutation> getMutations(BatchVariables variables, boolean local, ConsistencyLevel cl, long now)
     throws RequestExecutionException, RequestValidationException
     {
         Map<Pair<String, ByteBuffer>, IMutation> mutations = new HashMap<Pair<String, ByteBuffer>, IMutation>();
         for (int i = 0; i < statements.size(); i++)
         {
             ModificationStatement statement = statements.get(i);
-            List<ByteBuffer> statementVariables = variables.get(i);
-            addStatementMutations(statement, statementVariables, false, cl, now, mutations);
+            List<ByteBuffer> statementVariables = variables.getVariablesForStatement(i);
+            addStatementMutations(statement, statementVariables, local, cl, now, mutations);
         }
         return mutations.values();
     }
@@ -156,31 +153,132 @@ public class BatchStatement implements CQLStatement, MeasurableForPreparedCache
         if (options.getConsistency() == null)
             throw new InvalidRequestException("Invalid empty consistency level");
 
-        execute(getMutations(options.getValues(), false, options.getConsistency(), queryState.getTimestamp()), options.getConsistency());
-        return null;
+        return execute(new PreparedBatchVariables(options.getValues()), false, options.getConsistency(), queryState.getTimestamp());
     }
 
-    public void executeWithPerStatementVariables(ConsistencyLevel cl, QueryState queryState, List<List<ByteBuffer>> variables) throws RequestExecutionException, RequestValidationException
+    public ResultMessage executeWithPerStatementVariables(ConsistencyLevel cl, QueryState queryState, List<List<ByteBuffer>> variables) throws RequestExecutionException, RequestValidationException
     {
         if (cl == null)
             throw new InvalidRequestException("Invalid empty consistency level");
 
-        execute(getMutations(variables, cl, queryState.getTimestamp()), cl);
+        return execute(new BatchOfPreparedVariables(variables), false, cl, queryState.getTimestamp());
+    }
+
+    public ResultMessage execute(BatchVariables variables, boolean local, ConsistencyLevel cl, long now)
+    throws RequestExecutionException, RequestValidationException
+    {
+        // TODO: we don't support a serial consistency for batches in the protocol so defaulting to SERIAL for now.
+        // We'll need to fix that.
+        if (hasConditions)
+            return executeWithConditions(variables, cl, ConsistencyLevel.SERIAL, now);
+
+        executeWithoutConditions(getMutations(variables, local, cl, now), cl);
+        return null;
     }
 
-    private void execute(Collection<? extends IMutation> mutations, ConsistencyLevel cl) throws RequestExecutionException, RequestValidationException
+    private void executeWithoutConditions(Collection<? extends IMutation> mutations, ConsistencyLevel cl) throws RequestExecutionException, RequestValidationException
     {
         boolean mutateAtomic = (type == Type.LOGGED && mutations.size() > 1);
         StorageProxy.mutateWithTriggers(mutations, cl, mutateAtomic);
     }
 
+    private ResultMessage executeWithConditions(BatchVariables variables, ConsistencyLevel cl, ConsistencyLevel serialCf, long now)
+    throws RequestExecutionException, RequestValidationException
+    {
+        ByteBuffer key = null;
+        String ksName = null;
+        String cfName = null;
+        ColumnFamily updates = null;
+        CQL3CasConditions conditions = null;
+        Set<ColumnIdentifier> columnsWithConditions = new LinkedHashSet<ColumnIdentifier>();
+
+        for (int i = 0; i < statements.size(); i++)
+        {
+            ModificationStatement statement = statements.get(i);
+            List<ByteBuffer> statementVariables = variables.getVariablesForStatement(i);
+            long timestamp = attrs.getTimestamp(now, statementVariables);
+            List<ByteBuffer> pks = statement.buildPartitionKeyNames(statementVariables);
+            if (pks.size() > 1)
+                throw new IllegalArgumentException("Batch with conditions cannot span multiple partitions (you cannot use IN on the partition key)");
+            if (key == null)
+            {
+                key = pks.get(0);
+                ksName = statement.cfm.ksName;
+                cfName = statement.cfm.cfName;
+                conditions = new CQL3CasConditions(statement.cfm, now);
+                updates = UnsortedColumns.factory.create(statement.cfm);
+            }
+            else if (!key.equals(pks.get(0)))
+            {
+                throw new InvalidRequestException("Batch with conditions cannot span multiple partitions");
+            }
+
+            if (statement.hasConditions())
+            {
+                ColumnNameBuilder clusteringPrefix = statement.createClusteringPrefixBuilder(statementVariables);
+                statement.addUpdatesAndConditions(key, clusteringPrefix, updates, conditions, statementVariables, timestamp);
+                // As soon as we have a ifNotExists, we set columnsWithConditions to null so that everything is in the resultSet
+                if (statement.hasIfNotExistCondition())
+                    columnsWithConditions = null;
+                else if (columnsWithConditions != null)
+                    Iterables.addAll(columnsWithConditions, statement.getColumnsWithConditions());
+            }
+            else
+            {
+                // getPartitionKey will already have thrown if there is more than one key involved
+                IMutation mut = statement.getMutations(statementVariables, false, cl, timestamp, true).iterator().next();
+                updates.resolve(mut.getColumnFamilies().iterator().next());
+            }
+        }
+
+        ColumnFamily result = StorageProxy.cas(ksName, cfName, key, conditions, updates, serialCf, cl);
+        return new ResultMessage.Rows(ModificationStatement.buildCasResultSet(ksName, key, cfName, result, columnsWithConditions, true));
+    }
+
     public ResultMessage executeInternal(QueryState queryState) throws RequestValidationException, RequestExecutionException
     {
-        for (IMutation mutation : getMutations(Collections.<ByteBuffer>emptyList(), true, null, queryState.getTimestamp()))
+        assert !hasConditions;
+
+        for (IMutation mutation : getMutations(new PreparedBatchVariables(Collections.<ByteBuffer>emptyList()), true, null, queryState.getTimestamp()))
             mutation.apply();
         return null;
     }
 
+    public interface BatchVariables
+    {
+        public List<ByteBuffer> getVariablesForStatement(int statementInBatch);
+    }
+
+    public static class PreparedBatchVariables implements BatchVariables
+    {
+        private final List<ByteBuffer> variables;
+
+        public PreparedBatchVariables(List<ByteBuffer> variables)
+        {
+            this.variables = variables;
+        }
+
+        public List<ByteBuffer> getVariablesForStatement(int statementInBatch)
+        {
+            return variables;
+        }
+    }
+
+    public static class BatchOfPreparedVariables implements BatchVariables
+    {
+        private final List<List<ByteBuffer>> variables;
+
+        public BatchOfPreparedVariables(List<List<ByteBuffer>> variables)
+        {
+            this.variables = variables;
+        }
+
+        public List<ByteBuffer> getVariablesForStatement(int statementInBatch)
+        {
+            return variables.get(statementInBatch);
+        }
+    }
+
     public String toString()
     {
         return String.format("BatchStatement(type=%s, statements=%s)", type, statements);
@@ -212,11 +310,12 @@ public class BatchStatement implements CQLStatement, MeasurableForPreparedCache
             VariableSpecifications boundNames = getBoundVariables();
 
             List<ModificationStatement> statements = new ArrayList<ModificationStatement>(parsedStatements.size());
+            boolean hasConditions = false;
             for (ModificationStatement.Parsed parsed : parsedStatements)
             {
                 ModificationStatement stmt = parsed.prepare(boundNames);
                 if (stmt.hasConditions())
-                    throw new InvalidRequestException("Conditional updates are not allowed in batches");
+                    hasConditions = true;
 
                 if (stmt.isCounter() && type != Type.COUNTER)
                     throw new InvalidRequestException("Counter mutations are only allowed in COUNTER batches");
@@ -227,10 +326,23 @@ public class BatchStatement implements CQLStatement, MeasurableForPreparedCache
                 statements.add(stmt);
             }
 
+            if (hasConditions)
+            {
+                String ksName = null;
+                String cfName = null;
+                for (ModificationStatement stmt : statements)
+                {
+                    if (ksName != null && (!stmt.keyspace().equals(ksName) || !stmt.columnFamily().equals(cfName)))
+                        throw new InvalidRequestException("Batch with conditions cannot span multiple tables");
+                    ksName = stmt.keyspace();
+                    cfName = stmt.columnFamily();
+                }
+            }
+
             Attributes prepAttrs = attrs.prepare("[batch]", "[batch]");
             prepAttrs.collectMarkerSpecification(boundNames);
 
-            return new ParsedStatement.Prepared(new BatchStatement(boundNames.size(), type, statements, prepAttrs), boundNames);
+            return new ParsedStatement.Prepared(new BatchStatement(boundNames.size(), type, statements, prepAttrs, hasConditions), boundNames);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java b/src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
new file mode 100644
index 0000000..194ff0c
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
@@ -0,0 +1,164 @@
+/*
+ * 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.statements;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.filter.*;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.service.CASConditions;
+
+/**
+ * Processed CAS conditions on potentially multiple rows of the same partition.
+ */
+public class CQL3CasConditions implements CASConditions
+{
+    private final CFMetaData cfm;
+    private final long now;
+
+    // We index RowCondition by the prefix of the row they applied to for 2 reasons:
+    //   1) this allows to keep things sorted to build the ColumnSlice array below
+    //   2) this allows to detect when contradictory conditions are set (not exists with some other conditions on the same row)
+    private final SortedMap<ByteBuffer, RowCondition> conditions;
+
+    public CQL3CasConditions(CFMetaData cfm, long now)
+    {
+        this.cfm = cfm;
+        this.now = now;
+        this.conditions = new TreeMap<>(cfm.comparator);
+    }
+
+    public void addNotExist(ColumnNameBuilder prefix) throws InvalidRequestException
+    {
+        RowCondition previous = conditions.put(prefix.build(), new NotExistCondition(prefix, now));
+        if (previous != null && !(previous instanceof NotExistCondition))
+            throw new InvalidRequestException("Cannot mix IF conditions and IF NOT EXISTS for the same row");
+    }
+
+    public void addConditions(ColumnNameBuilder prefix, Collection<ColumnCondition> conds, List<ByteBuffer> variables) throws InvalidRequestException
+    {
+        ByteBuffer b = prefix.build();
+        RowCondition condition = conditions.get(b);
+        if (condition == null)
+        {
+            condition = new ColumnsConditions(prefix, now);
+            conditions.put(b, condition);
+        }
+        else if (!(condition instanceof ColumnsConditions))
+        {
+            throw new InvalidRequestException("Cannot mix IF conditions and IF NOT EXISTS for the same row");
+        }
+        ((ColumnsConditions)condition).addConditions(conds, variables);
+    }
+
+    public IDiskAtomFilter readFilter()
+    {
+        assert !conditions.isEmpty();
+        ColumnSlice[] slices = new ColumnSlice[conditions.size()];
+        int i = 0;
+        // We always read CQL rows entirely as on CAS failure we want to be able to distinguish between "row exists
+        // but all values on why there were conditions are null" and "row doesn't exists", and we can't rely on the
+        // row marker for that (see #6623)
+        for (Map.Entry<ByteBuffer, RowCondition> entry : conditions.entrySet())
+            slices[i++] = new ColumnSlice(entry.getKey(), entry.getValue().rowPrefix.buildAsEndOfRange());
+
+        return new SliceQueryFilter(slices, false, slices.length, cfm.clusteringKeyColumns().size());
+    }
+
+    public boolean appliesTo(ColumnFamily current) throws InvalidRequestException
+    {
+        for (RowCondition condition : conditions.values())
+        {
+            if (!condition.appliesTo(current))
+                return false;
+        }
+        return true;
+    }
+
+    private static abstract class RowCondition
+    {
+        public final ColumnNameBuilder rowPrefix;
+        protected final long now;
+
+        protected RowCondition(ColumnNameBuilder rowPrefix, long now)
+        {
+            this.rowPrefix = rowPrefix;
+            this.now = now;
+        }
+
+        public abstract boolean appliesTo(ColumnFamily current) throws InvalidRequestException;
+    }
+
+    private static class NotExistCondition extends RowCondition
+    {
+        private NotExistCondition(ColumnNameBuilder rowPrefix, long now)
+        {
+            super(rowPrefix, now);
+        }
+
+        public boolean appliesTo(ColumnFamily current)
+        {
+            if (current == null)
+                return true;
+
+            Iterator<Column> iter = current.iterator(new ColumnSlice[]{ new ColumnSlice(rowPrefix.build(), rowPrefix.buildAsEndOfRange()) });
+            while (iter.hasNext())
+                if (iter.next().isLive(now))
+                    return false;
+            return true;
+        }
+    }
+
+    private static class ColumnsConditions extends RowCondition
+    {
+        private final Map<ColumnIdentifier, ColumnCondition> conditions = new HashMap<>();
+
+        private ColumnsConditions(ColumnNameBuilder rowPrefix, long now)
+        {
+            super(rowPrefix, now);
+        }
+
+        public void addConditions(Collection<ColumnCondition> conds, List<ByteBuffer> variables) throws InvalidRequestException
+        {
+            for (ColumnCondition condition : conds)
+            {
+                // We will need the variables in appliesTo but with protocol batches, each condition in this object can have a
+                // different list of variables. So attach them to the condition directly, it's not particulary elegant but its simpler
+                ColumnCondition previous = conditions.put(condition.column.name, condition.attach(variables));
+                // If 2 conditions are actually equal, let it slide
+                if (previous != null && !previous.equalsTo(condition))
+                    throw new InvalidRequestException("Duplicate and incompatible conditions for column " + condition.column.name);
+            }
+        }
+
+        public boolean appliesTo(ColumnFamily current) throws InvalidRequestException
+        {
+            if (current == null)
+                return conditions.isEmpty();
+
+            for (ColumnCondition condition : conditions.values())
+                if (!condition.appliesTo(rowPrefix, current, now))
+                    return false;
+            return true;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/cql3/statements/ColumnGroupMap.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ColumnGroupMap.java b/src/java/org/apache/cassandra/cql3/statements/ColumnGroupMap.java
index 8974523..5c3fcb9 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ColumnGroupMap.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ColumnGroupMap.java
@@ -32,10 +32,12 @@ public class ColumnGroupMap
 {
     private final ByteBuffer[] fullPath;
     private final Map<ByteBuffer, Value> map = new HashMap<ByteBuffer, Value>();
+    public final boolean isStatic; // Whether or not the group correspond to "static" cells
 
-    private ColumnGroupMap(ByteBuffer[] fullPath)
+    private ColumnGroupMap(ByteBuffer[] fullPath, boolean isStatic)
     {
         this.fullPath = fullPath;
+        this.isStatic = isStatic;
     }
 
     private void add(ByteBuffer[] fullName, int idx, Column column)
@@ -126,7 +128,7 @@ public class ColumnGroupMap
 
             if (currentGroup == null)
             {
-                currentGroup = new ColumnGroupMap(current);
+                currentGroup = new ColumnGroupMap(current, composite.isStaticName(c.name()));
                 currentGroup.add(current, idx, c);
                 previous = current;
                 return;
@@ -135,7 +137,8 @@ public class ColumnGroupMap
             if (!isSameGroup(current))
             {
                 groups.add(currentGroup);
-                currentGroup = new ColumnGroupMap(current);
+                // Note that we know that only the first group built can be static
+                currentGroup = new ColumnGroupMap(current, false);
             }
             currentGroup.add(current, idx, c);
             previous = current;
@@ -167,5 +170,25 @@ public class ColumnGroupMap
             }
             return groups;
         }
+
+        public boolean isEmpty()
+        {
+            return currentGroup == null && groups.isEmpty();
+        }
+
+        public ColumnGroupMap firstGroup()
+        {
+            if (currentGroup != null)
+            {
+                groups.add(currentGroup);
+                currentGroup = null;
+            }
+            return groups.get(0);
+        }
+
+        public void discardFirst()
+        {
+            groups.remove(0);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
index 3ef6f5a..376fa4a 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
@@ -87,6 +87,15 @@ public class CreateIndexStatement extends SchemaAlteringStatement
         if (cfm.getCfDef().isCompact && cd.type != ColumnDefinition.Type.REGULAR)
             throw new InvalidRequestException(String.format("Secondary index on %s column %s is not yet supported for compact table", cd.type, columnName));
 
+        // It would be possible to support 2ndary index on static columns (but not without modifications of at least ExtendedFilter and
+        // CompositesIndex) and maybe we should, but that means a query like:
+        //     SELECT * FROM foo WHERE static_column = 'bar'
+        // would pull the full partition every time the static column of partition is 'bar', which sounds like offering a
+        // fair potential for foot-shooting, so I prefer leaving that to a follow up ticket once we have identified cases where
+        // such indexing is actually useful.
+        if (cd.type == ColumnDefinition.Type.STATIC)
+            throw new InvalidRequestException("Secondary indexes are not allowed on static columns");
+
         if (cd.getValidator().isCollection() && !properties.isCustom)
             throw new InvalidRequestException("Indexes on collections are no yet supported");
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09d8769/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 74f7570..632194c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
@@ -52,14 +52,16 @@ public class CreateTableStatement extends SchemaAlteringStatement
     private ByteBuffer valueAlias;
 
     private final Map<ColumnIdentifier, AbstractType> columns = new HashMap<ColumnIdentifier, AbstractType>();
+    private final Set<ColumnIdentifier> staticColumns;
     private final CFPropDefs properties;
     private final boolean ifNotExists;
 
-    public CreateTableStatement(CFName name, CFPropDefs properties, boolean ifNotExists)
+    public CreateTableStatement(CFName name, CFPropDefs properties, boolean ifNotExists, Set<ColumnIdentifier> staticColumns)
     {
         super(name);
         this.properties = properties;
         this.ifNotExists = ifNotExists;
+        this.staticColumns = staticColumns;
 
         try
         {
@@ -101,7 +103,10 @@ public class CreateTableStatement extends SchemaAlteringStatement
 
         for (Map.Entry<ColumnIdentifier, AbstractType> col : columns.entrySet())
         {
-            columnDefs.put(col.getKey().key, ColumnDefinition.regularDef(col.getKey().key, col.getValue(), componentIndex));
+            ColumnIdentifier id = col.getKey();
+            columnDefs.put(id.key, staticColumns.contains(id)
+                                   ? ColumnDefinition.staticDef(id.key, col.getValue(), componentIndex)
+                                   : ColumnDefinition.regularDef(id.key, col.getValue(), componentIndex));
         }
 
         return columnDefs;
@@ -166,6 +171,7 @@ public class CreateTableStatement extends SchemaAlteringStatement
         private final List<List<ColumnIdentifier>> keyAliases = new ArrayList<List<ColumnIdentifier>>();
         private final List<ColumnIdentifier> columnAliases = new ArrayList<ColumnIdentifier>();
         private final Map<ColumnIdentifier, Boolean> definedOrdering = new LinkedHashMap<ColumnIdentifier, Boolean>(); // Insertion ordering is important
+        private final Set<ColumnIdentifier> staticColumns = new HashSet<ColumnIdentifier>();
 
         private boolean useCompactStorage;
         private final Multiset<ColumnIdentifier> definedNames = HashMultiset.create(1);
@@ -195,7 +201,7 @@ public class CreateTableStatement extends SchemaAlteringStatement
 
             properties.validate();
 
-            CreateTableStatement stmt = new CreateTableStatement(cfName, properties, ifNotExists);
+            CreateTableStatement stmt = new CreateTableStatement(cfName, properties, ifNotExists, staticColumns);
 
             Map<ByteBuffer, CollectionType> definedCollections = null;
             for (Map.Entry<ColumnIdentifier, CQL3Type> entry : definitions.entrySet())
@@ -225,6 +231,8 @@ public class CreateTableStatement extends SchemaAlteringStatement
                 AbstractType<?> t = getTypeAndRemove(stmt.columns, alias);
                 if (t instanceof CounterColumnType)
                     throw new InvalidRequestException(String.format("counter type is not supported for PRIMARY KEY part %s", alias));
+                if (staticColumns.contains(alias))
+                    throw new InvalidRequestException(String.format("Static column %s cannot be part of the PRIMARY KEY", alias));
                 keyTypes.add(t);
             }
             stmt.keyValidator = keyTypes.size() == 1 ? keyTypes.get(0) : CompositeType.getInstance(keyTypes);
@@ -260,10 +268,13 @@ public class CreateTableStatement extends SchemaAlteringStatement
                 {
                     if (definedCollections != null)
                         throw new InvalidRequestException("Collection types are not supported with COMPACT STORAGE");
-                    stmt.columnAliases.add(columnAliases.get(0).key);
-                    stmt.comparator = getTypeAndRemove(stmt.columns, columnAliases.get(0));
+                    ColumnIdentifier alias = columnAliases.get(0);
+                    stmt.columnAliases.add(alias.key);
+                    stmt.comparator = getTypeAndRemove(stmt.columns, alias);
                     if (stmt.comparator instanceof CounterColumnType)
-                        throw new InvalidRequestException(String.format("counter type is not supported for PRIMARY KEY part %s", stmt.columnAliases.get(0)));
+                        throw new InvalidRequestException(String.format("counter type is not supported for PRIMARY KEY part %s", alias));
+                    if (staticColumns.contains(alias))
+                        throw new InvalidRequestException(String.format("Static column %s cannot be part of the PRIMARY KEY", alias));
                 }
                 else
                 {
@@ -275,6 +286,8 @@ public class CreateTableStatement extends SchemaAlteringStatement
                         AbstractType<?> type = getTypeAndRemove(stmt.columns, t);
                         if (type instanceof CounterColumnType)
                             throw new InvalidRequestException(String.format("counter type is not supported for PRIMARY KEY part %s", t.key));
+                        if (staticColumns.contains(t))
+                            throw new InvalidRequestException(String.format("Static column %s cannot be part of the PRIMARY KEY", t));
                         types.add(type);
                     }
 
@@ -298,6 +311,16 @@ public class CreateTableStatement extends SchemaAlteringStatement
                 }
             }
 
+            if (!staticColumns.isEmpty())
+            {
+                // Only CQL3 tables can have static columns
+                if (useCompactStorage)
+                    throw new InvalidRequestException("Static columns are not supported in COMPACT STORAGE tables");
+                // Static columns only make sense if we have at least one clustering column. Otherwise everything is static anyway
+                if (columnAliases.isEmpty())
+                    throw new InvalidRequestException("Static columns are only useful (and thus allowed) if the table has at least one clustering column");
+            }
+
             if (useCompactStorage && !stmt.columnAliases.isEmpty())
             {
                 if (stmt.columns.isEmpty())
@@ -373,10 +396,12 @@ public class CreateTableStatement extends SchemaAlteringStatement
             return isReversed != null && isReversed ? ReversedType.getInstance(type) : type;
         }
 
-        public void addDefinition(ColumnIdentifier def, CQL3Type type)
+        public void addDefinition(ColumnIdentifier def, CQL3Type type, boolean isStatic)
         {
             definedNames.add(def);
             definitions.put(def, type);
+            if (isStatic)
+                staticColumns.add(def);
         }
 
         public void addKeyAliases(List<ColumnIdentifier> aliases)