cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From slebre...@apache.org
Subject [2/4] Remove CFDefinition
Date Wed, 30 Oct 2013 16:10:18 GMT
http://git-wip-us.apache.org/repos/asf/cassandra/blob/5f5905d5/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 918f043..8cec64e 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -24,6 +24,7 @@ import com.google.common.base.Objects;
 import com.google.common.base.Predicate;
 import com.google.common.collect.AbstractIterator;
 import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
 
 import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.cql3.*;
@@ -57,17 +58,17 @@ public class SelectStatement implements CQLStatement
     private static final int DEFAULT_COUNT_PAGE_SIZE = 10000;
 
     private final int boundTerms;
-    public final CFDefinition cfDef;
+    public final CFMetaData cfm;
     public final Parameters parameters;
     private final Selection selection;
     private final Term limit;
 
     private final Restriction[] keyRestrictions;
     private final Restriction[] columnRestrictions;
-    private final Map<CFDefinition.Name, Restriction> metadataRestrictions = new HashMap<CFDefinition.Name, Restriction>();
+    private final Map<ColumnIdentifier, Restriction> metadataRestrictions = new HashMap<ColumnIdentifier, Restriction>();
 
-    // The name of all restricted names not covered by the key or index filter
-    private final Set<CFDefinition.Name> restrictedNames = new HashSet<CFDefinition.Name>();
+    // All restricted columns not covered by the key or index filter
+    private final Set<ColumnDefinition> restrictedColumns = new HashSet<ColumnDefinition>();
     private Restriction.Slice sliceRestriction;
 
     private boolean isReversed;
@@ -76,18 +77,18 @@ public class SelectStatement implements CQLStatement
     private boolean keyIsInRelation;
     private boolean usesSecondaryIndexing;
 
-    private Map<CFDefinition.Name, Integer> orderingIndexes;
+    private Map<ColumnDefinition, Integer> orderingIndexes;
 
     // Used by forSelection below
     private static final Parameters defaultParameters = new Parameters(Collections.<ColumnIdentifier, Boolean>emptyMap(), false, false, null, false);
 
-    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.keys.size()];
-        this.columnRestrictions = new Restriction[cfDef.columns.size()];
+        this.keyRestrictions = new Restriction[cfm.partitionKeyColumns().size()];
+        this.columnRestrictions = new Restriction[cfm.clusteringColumns().size()];
         this.parameters = parameters;
         this.limit = limit;
     }
@@ -95,9 +96,9 @@ public class SelectStatement implements CQLStatement
     // Creates a simple select based on the given selection.
     // Note that the results select statement should not be used for actual queries, but only for processing already
     // queried data through processColumnFamily.
-    static SelectStatement forSelection(CFDefinition cfDef, Selection selection)
+    static SelectStatement forSelection(CFMetaData cfm, Selection selection)
     {
-        return new SelectStatement(cfDef, 0, defaultParameters, selection, null);
+        return new SelectStatement(cfm, 0, defaultParameters, selection, null);
     }
 
     public ResultSet.Metadata getResultMetadata()
@@ -244,12 +245,12 @@ public class SelectStatement implements CQLStatement
 
     public String keyspace()
     {
-        return cfDef.cfm.ksName;
+        return cfm.ksName;
     }
 
     public String columnFamily()
     {
-        return cfDef.cfm.cfName;
+        return cfm.cfName;
     }
 
     private List<ReadCommand> getSliceCommands(List<ByteBuffer> variables, int limit, long now) throws RequestValidationException
@@ -363,7 +364,7 @@ public class SelectStatement implements CQLStatement
             // 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 = cfm.isDense() ? -1 : cfm.clusteringColumns().size();
             List<ByteBuffer> startBounds = getRequestedBound(Bound.START, variables);
             List<ByteBuffer> endBounds = getRequestedBound(Bound.END, variables);
             assert startBounds.size() == endBounds.size();
@@ -373,7 +374,7 @@ public class SelectStatement implements CQLStatement
             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};
             }
@@ -383,7 +384,7 @@ public class SelectStatement implements CQLStatement
                 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())
@@ -437,10 +438,10 @@ public class SelectStatement implements CQLStatement
     private Collection<ByteBuffer> getKeys(final List<ByteBuffer> variables) throws InvalidRequestException
     {
         List<ByteBuffer> keys = new ArrayList<ByteBuffer>();
-        ColumnNameBuilder builder = cfDef.getKeyNameBuilder();
-        for (CFDefinition.Name name : cfDef.keys.values())
+        ColumnNameBuilder builder = cfm.getKeyNameBuilder();
+        for (ColumnDefinition def : cfm.partitionKeyColumns())
         {
-            Restriction r = keyRestrictions[name.position];
+            Restriction r = keyRestrictions[def.position()];
             assert r != null && !r.isSlice();
 
             List<ByteBuffer> values = r.values(variables);
@@ -450,7 +451,7 @@ public class SelectStatement implements CQLStatement
                 for (ByteBuffer val : values)
                 {
                     if (val == null)
-                        throw new InvalidRequestException(String.format("Invalid null value for partition key part %s", name));
+                        throw new InvalidRequestException(String.format("Invalid null value for partition key part %s", def.name));
                     keys.add(builder.copy().add(val).build());
                 }
             }
@@ -461,7 +462,7 @@ public class SelectStatement implements CQLStatement
                     throw new InvalidRequestException("IN is only supported on the last column of the partition key");
                 ByteBuffer val = values.get(0);
                 if (val == null)
-                    throw new InvalidRequestException(String.format("Invalid null value for partition key part %s", name));
+                    throw new InvalidRequestException(String.format("Invalid null value for partition key part %s", def.name));
                 builder.add(val);
             }
         }
@@ -477,7 +478,7 @@ public class SelectStatement implements CQLStatement
                 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, cfm.partitionKeyColumns(), keyRestrictions, false, cfm.getKeyNameBuilder(), variables).get(0);
     }
 
     private Token getTokenBound(Bound b, List<ByteBuffer> variables, IPartitioner<?> p) throws InvalidRequestException
@@ -519,10 +520,10 @@ public class SelectStatement implements CQLStatement
 
     private boolean isColumnRange()
     {
-        // Due to CASSANDRA-5762, we always do a slice for CQL3 tables (not compact, composite).
-        // Static CF (non compact but non composite) never entails a column slice however
-        if (!cfDef.isCompact)
-            return cfDef.isComposite;
+        // Due to CASSANDRA-5762, we always do a slice for CQL3 tables (not dense, composite).
+        // Static CF (non dense but non composite) never entails a column slice however
+        if (!cfm.isDense())
+            return cfm.hasCompositeComparator();
 
         // Otherwise (i.e. for compact table where we don't have a row marker anyway and thus don't care about CASSANDRA-5762),
         // it is a range query if it has at least one the column alias for which no relation is defined or is not EQ.
@@ -538,11 +539,11 @@ public class SelectStatement implements CQLStatement
     {
         assert !isColumnRange();
 
-        ColumnNameBuilder builder = cfDef.getColumnNameBuilder();
-        Iterator<ColumnIdentifier> idIter = cfDef.columns.keySet().iterator();
+        ColumnNameBuilder builder = cfm.getColumnNameBuilder();
+        Iterator<ColumnDefinition> idIter = cfm.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);
@@ -560,7 +561,7 @@ public class SelectStatement implements CQLStatement
                 // for each value of the IN, creates all the columns corresponding to the selection.
                 if (values.isEmpty())
                     return null;
-                SortedSet<ByteBuffer> columns = new TreeSet<ByteBuffer>(cfDef.cfm.comparator);
+                SortedSet<ByteBuffer> columns = new TreeSet<ByteBuffer>(cfm.comparator);
                 Iterator<ByteBuffer> iter = values.iterator();
                 while (iter.hasNext())
                 {
@@ -569,7 +570,7 @@ public class SelectStatement implements CQLStatement
                     if (val == null)
                         throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s", id));
                     b.add(val);
-                    if (cfDef.isCompact)
+                    if (cfm.isDense())
                         columns.add(b.build());
                     else
                         columns.addAll(addSelectedColumns(b));
@@ -583,7 +584,7 @@ public class SelectStatement implements CQLStatement
 
     private SortedSet<ByteBuffer> addSelectedColumns(ColumnNameBuilder builder)
     {
-        if (cfDef.isCompact)
+        if (cfm.isDense())
         {
             return FBUtilities.singleton(builder.build());
         }
@@ -593,29 +594,29 @@ public class SelectStatement implements CQLStatement
             // non-know set of columns, so we shouldn't get there
             assert !selectACollection();
 
-            SortedSet<ByteBuffer> columns = new TreeSet<ByteBuffer>(cfDef.cfm.comparator);
+            SortedSet<ByteBuffer> columns = new TreeSet<ByteBuffer>(cfm.comparator);
 
             // We need to query the selected column as well as the marker
             // 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.hasCompositeComparator() && !cfm.isSuper())
             {
                 // marker
                 columns.add(builder.copy().add(ByteBufferUtil.EMPTY_BYTE_BUFFER).build());
 
                 // selected columns
                 for (ColumnIdentifier id : selection.regularColumnsToFetch())
-                    columns.add(builder.copy().add(id.key).build());
+                    columns.add(builder.copy().add(id).build());
             }
             else
             {
-                Iterator<ColumnIdentifier> iter = cfDef.metadata.keySet().iterator();
+                Iterator<ColumnDefinition> iter = cfm.regularColumns().iterator();
                 while (iter.hasNext())
                 {
-                    ColumnIdentifier name = iter.next();
+                    ColumnDefinition def = iter.next();
                     ColumnNameBuilder b = iter.hasNext() ? builder.copy() : builder;
-                    ByteBuffer cname = b.add(name.key).build();
+                    ByteBuffer cname = b.add(def.name).build();
                     columns.add(cname);
                 }
             }
@@ -625,12 +626,12 @@ public class SelectStatement implements CQLStatement
 
     private boolean selectACollection()
     {
-        if (!cfDef.hasCollections)
+        if (!cfm.hasCollections())
             return false;
 
-        for (CFDefinition.Name name : selection.getColumnsList())
+        for (ColumnDefinition def : selection.getColumnsList())
         {
-            if (name.type instanceof CollectionType)
+            if (def.type instanceof CollectionType)
                 return true;
         }
 
@@ -638,7 +639,7 @@ public class SelectStatement implements CQLStatement
     }
 
     private List<ByteBuffer> buildBound(Bound bound,
-                                        Collection<CFDefinition.Name> names,
+                                        Collection<ColumnDefinition> defs,
                                         Restriction[] restrictions,
                                         boolean isReversed,
                                         ColumnNameBuilder builder,
@@ -649,13 +650,13 @@ public class SelectStatement implements CQLStatement
         // to the component comparator but not to the end-of-component itself),
         // it only depends on whether the slice is reversed
         Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
-        for (CFDefinition.Name name : names)
+        for (ColumnDefinition def : defs)
         {
             // In a restriction, we always have Bound.START < Bound.END for the "base" comparator.
             // So if we're doing a reverse slice, we must inverse the bounds when giving them as start and end of the slice filter.
             // But if the actual comparator itself is reversed, we must inversed the bounds too.
-            Bound b = isReversed == isReversedType(name) ? bound : Bound.reverse(bound);
-            Restriction r = restrictions[name.position];
+            Bound b = isReversed == isReversedType(def) ? bound : Bound.reverse(bound);
+            Restriction r = restrictions[def.position()];
             if (r == null || (r.isSlice() && !((Restriction.Slice)r).hasBound(b)))
             {
                 // There wasn't any non EQ relation on that key, we select all records having the preceding component as prefix.
@@ -672,7 +673,7 @@ public class SelectStatement implements CQLStatement
                 assert slice.hasBound(b);
                 ByteBuffer val = slice.bound(b, variables);
                 if (val == null)
-                    throw new InvalidRequestException(String.format("Invalid null clustering key part %s", name));
+                    throw new InvalidRequestException(String.format("Invalid null clustering key part %s", def.name));
                 return Collections.singletonList(builder.add(val, slice.getRelation(eocBound, b)).build());
             }
             else
@@ -681,14 +682,14 @@ public class SelectStatement implements CQLStatement
                 if (values.size() != 1)
                 {
                     // IN query, we only support it on the clustering column
-                    assert name.position == names.size() - 1;
+                    assert def.position() == defs.size() - 1;
                     // The IN query might not have listed the values in comparator order, so we need to re-sort
                     // the bounds lists to make sure the slices works correctly (also, to avoid duplicates).
-                    TreeSet<ByteBuffer> s = new TreeSet<ByteBuffer>(isReversed ? cfDef.cfm.comparator.reverseComparator : cfDef.cfm.comparator);
+                    TreeSet<ByteBuffer> s = new TreeSet<ByteBuffer>(isReversed ? cfm.comparator.reverseComparator : cfm.comparator);
                     for (ByteBuffer val : values)
                     {
                         if (val == null)
-                            throw new InvalidRequestException(String.format("Invalid null clustering key part %s", name));
+                            throw new InvalidRequestException(String.format("Invalid null clustering key part %s", def.name));
                         ColumnNameBuilder copy = builder.copy().add(val);
                         // See below for why this
                         s.add((bound == Bound.END && copy.remainingCount() > 0) ? copy.buildAsEndOfRange() : copy.build());
@@ -698,7 +699,7 @@ public class SelectStatement implements CQLStatement
 
                 ByteBuffer val = values.get(0);
                 if (val == null)
-                    throw new InvalidRequestException(String.format("Invalid null clustering key part %s", name));
+                    throw new InvalidRequestException(String.format("Invalid null clustering key part %s", def.name));
                 builder.add(val);
             }
         }
@@ -714,31 +715,31 @@ public class SelectStatement implements CQLStatement
     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, cfm.clusteringColumns(), columnRestrictions, isReversed, cfm.getColumnNameBuilder(), variables);
     }
 
     public List<IndexExpression> getIndexExpressions(List<ByteBuffer> variables) throws InvalidRequestException
     {
-        if (!usesSecondaryIndexing || restrictedNames.isEmpty())
+        if (!usesSecondaryIndexing || restrictedColumns.isEmpty())
             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:
-                    restriction = metadataRestrictions.get(name);
+                case REGULAR:
+                    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();
             }
 
@@ -751,10 +752,10 @@ public class SelectStatement implements CQLStatement
                     {
                         ByteBuffer value = slice.bound(b, variables);
                         if (value == null)
-                            throw new InvalidRequestException(String.format("Unsupported null value for indexed column %s", name));
+                            throw new InvalidRequestException(String.format("Unsupported null value for indexed column %s", def.name));
                         if (value.remaining() > 0xFFFF)
                             throw new InvalidRequestException("Index expression values may not be larger than 64K");
-                        expressions.add(new IndexExpression(name.name.key, slice.getIndexOperator(b), value));
+                        expressions.add(new IndexExpression(def.name.bytes, slice.getIndexOperator(b), value));
                     }
                 }
             }
@@ -767,10 +768,10 @@ public class SelectStatement implements CQLStatement
 
                 ByteBuffer value = values.get(0);
                 if (value == null)
-                    throw new InvalidRequestException(String.format("Unsupported null value for indexed column %s", name));
+                    throw new InvalidRequestException(String.format("Unsupported null value for indexed column %s", def.name));
                 if (value.remaining() > 0xFFFF)
                     throw new InvalidRequestException("Index expression values may not be larger than 64K");
-                expressions.add(new IndexExpression(name.name.key, IndexExpression.Operator.EQ, value));
+                expressions.add(new IndexExpression(def.name.bytes, IndexExpression.Operator.EQ, value));
             }
         }
         return expressions;
@@ -788,11 +789,10 @@ public class SelectStatement implements CQLStatement
         if (last == null || last.isSlice())
             return cf.getSortedColumns();
 
-        ColumnNameBuilder builder = cfDef.getColumnNameBuilder();
+        ColumnNameBuilder builder = cfm.getColumnNameBuilder();
         for (int i = 0; i < columnRestrictions.length - 1; i++)
             builder.add(columnRestrictions[i].values(variables).get(0));
 
-
         List<ByteBuffer> values = last.values(variables);
         final List<ByteBuffer> requested = new ArrayList<ByteBuffer>(values.size());
         Iterator<ByteBuffer> iter = values.iterator();
@@ -851,8 +851,8 @@ public class SelectStatement implements CQLStatement
     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)
+        ByteBuffer[] keyComponents = cfm.getKeyValidator() instanceof CompositeType
+                                   ? ((CompositeType)cfm.getKeyValidator()).split(key)
                                    : new ByteBuffer[]{ key };
 
         if (parameters.isDistinct)
@@ -861,11 +861,11 @@ public class SelectStatement implements CQLStatement
             {
                 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]);
+                for (ColumnDefinition def : selection.getColumnsList())
+                    result.add(keyComponents[def.position()]);
             }
         }
-        else if (cfDef.isCompact)
+        else if (cfm.isDense())
         {
             // One cqlRow per column
             for (Column c : columnsInOrder(cf, variables))
@@ -874,9 +874,9 @@ public class SelectStatement implements CQLStatement
                     continue;
 
                 ByteBuffer[] components = null;
-                if (cfDef.isComposite)
+                if (cfm.hasCompositeComparator())
                 {
-                    components = ((CompositeType)cfDef.cfm.comparator).split(c.name());
+                    components = ((CompositeType)cfm.comparator).split(c.name());
                 }
                 else if (sliceRestriction != null)
                 {
@@ -889,23 +889,23 @@ public class SelectStatement implements CQLStatement
 
                 result.newRow();
                 // Respect selection order
-                for (CFDefinition.Name name : selection.getColumnsList())
+                for (ColumnDefinition def : selection.getColumnsList())
                 {
-                    switch (name.kind)
+                    switch (def.kind)
                     {
-                        case KEY_ALIAS:
-                            result.add(keyComponents[name.position]);
+                        case PARTITION_KEY:
+                            result.add(keyComponents[def.position()]);
                             break;
-                        case COLUMN_ALIAS:
-                            ByteBuffer val = cfDef.isComposite
-                                           ? (name.position < components.length ? components[name.position] : null)
+                        case CLUSTERING_COLUMN:
+                            ByteBuffer val = cfm.hasCompositeComparator()
+                                           ? (def.position() < components.length ? components[def.position()] : null)
                                            : c.name();
                             result.add(val);
                             break;
-                        case VALUE_ALIAS:
+                        case COMPACT_VALUE:
                             result.add(c);
                             break;
-                        case COLUMN_METADATA:
+                        case REGULAR:
                             // This should not happen for compact CF
                             throw new AssertionError();
                         default:
@@ -914,12 +914,12 @@ public class SelectStatement implements CQLStatement
                 }
             }
         }
-        else if (cfDef.isComposite)
+        else if (cfm.hasCompositeComparator())
         {
             // Sparse case: group column in cqlRow when composite prefix is equal
-            CompositeType composite = (CompositeType)cfDef.cfm.comparator;
+            CompositeType composite = (CompositeType)cfm.comparator;
 
-            ColumnGroupMap.Builder builder = new ColumnGroupMap.Builder(composite, cfDef.hasCollections, now);
+            ColumnGroupMap.Builder builder = new ColumnGroupMap.Builder(composite, cfm.hasCollections(), now);
 
             for (Column c : cf)
             {
@@ -939,12 +939,12 @@ public class SelectStatement implements CQLStatement
 
             // Static case: One cqlRow for all columns
             result.newRow();
-            for (CFDefinition.Name name : selection.getColumnsList())
+            for (ColumnDefinition def : selection.getColumnsList())
             {
-                if (name.kind == CFDefinition.Name.Kind.KEY_ALIAS)
-                    result.add(keyComponents[name.position]);
+                if (def.kind == ColumnDefinition.Kind.PARTITION_KEY)
+                    result.add(keyComponents[def.position()]);
                 else
-                    result.add(cf.getColumn(name.name.key));
+                    result.add(cf.getColumn(def.name.bytes));
             }
         }
     }
@@ -967,7 +967,7 @@ public class SelectStatement implements CQLStatement
         // because there is no point of using composite comparator if there is only one order condition
         if (parameters.orderings.size() == 1)
         {
-            CFDefinition.Name ordering = cfDef.get(parameters.orderings.keySet().iterator().next());
+            ColumnDefinition ordering = cfm.getColumnDefinition(parameters.orderings.keySet().iterator().next());
             Collections.sort(cqlRows.rows, new SingleColumnComparator(orderingIndexes.get(ordering), ordering.type));
             return;
         }
@@ -981,7 +981,7 @@ public class SelectStatement implements CQLStatement
         int idx = 0;
         for (ColumnIdentifier identifier : parameters.orderings.keySet())
         {
-            CFDefinition.Name orderingColumn = cfDef.get(identifier);
+            ColumnDefinition orderingColumn = cfm.getColumnDefinition(identifier);
             types.add(orderingColumn.type);
             positions[idx++] = orderingIndexes.get(orderingColumn);
         }
@@ -993,40 +993,40 @@ public class SelectStatement implements CQLStatement
     {
         // Respect requested order
         result.newRow();
-        for (CFDefinition.Name name : selection.getColumnsList())
+        for (ColumnDefinition def : selection.getColumnsList())
         {
-            switch (name.kind)
+            switch (def.kind)
             {
-                case KEY_ALIAS:
-                    result.add(keyComponents[name.position]);
+                case PARTITION_KEY:
+                    result.add(keyComponents[def.position()]);
                     break;
-                case COLUMN_ALIAS:
-                    result.add(columns.getKeyComponent(name.position));
+                case CLUSTERING_COLUMN:
+                    result.add(columns.getKeyComponent(def.position()));
                     break;
-                case VALUE_ALIAS:
+                case COMPACT_VALUE:
                     // This should not happen for SPARSE
                     throw new AssertionError();
-                case COLUMN_METADATA:
-                    if (name.type.isCollection())
+                case REGULAR:
+                    if (def.type.isCollection())
                     {
-                        List<Pair<ByteBuffer, Column>> collection = columns.getCollection(name.name.key);
+                        List<Pair<ByteBuffer, Column>> collection = columns.getCollection(def.name.bytes);
                         ByteBuffer value = collection == null
                                          ? null
-                                         : ((CollectionType)name.type).serialize(collection);
+                                         : ((CollectionType)def.type).serialize(collection);
                         result.add(value);
                     }
                     else
                     {
-                        result.add(columns.getSimple(name.name.key));
+                        result.add(columns.getSimple(def.name.bytes));
                     }
                     break;
             }
         }
     }
 
-    private static boolean isReversedType(CFDefinition.Name name)
+    private static boolean isReversedType(ColumnDefinition def)
     {
-        return name.type instanceof ReversedType;
+        return def.type instanceof ReversedType;
     }
 
     private boolean columnFilterIsIdentity()
@@ -1059,8 +1059,6 @@ public class SelectStatement implements CQLStatement
         {
             CFMetaData cfm = ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
 
-            CFDefinition cfDef = cfm.getCfDef();
-
             VariableSpecifications names = getBoundsVariables();
 
             // Select clause
@@ -1068,11 +1066,11 @@ public class SelectStatement implements CQLStatement
                 throw new InvalidRequestException("Only COUNT(*) and COUNT(1) operations are currently supported.");
 
             Selection selection = selectClause.isEmpty()
-                                ? Selection.wildcard(cfDef)
-                                : Selection.fromSelectors(cfDef, selectClause);
+                                ? Selection.wildcard(cfm)
+                                : Selection.fromSelectors(cfm, selectClause);
 
             if (parameters.isDistinct)
-                validateDistinctSelection(selection.getColumnsList(), cfDef.keys.values());
+                validateDistinctSelection(selection.getColumnsList(), cfm.partitionKeyColumns());
 
             Term prepLimit = null;
             if (limit != null)
@@ -1081,7 +1079,7 @@ public class SelectStatement implements CQLStatement
                 prepLimit.collectMarkerSpecification(names);
             }
 
-            SelectStatement stmt = new SelectStatement(cfDef, names.size(), parameters, selection, prepLimit);
+            SelectStatement stmt = new SelectStatement(cfm, names.size(), parameters, selection, prepLimit);
 
             /*
              * WHERE clause. For a given entity, rules are:
@@ -1095,8 +1093,8 @@ public class SelectStatement implements CQLStatement
             boolean hasQueriableClusteringColumnIndex = false;
             for (Relation rel : whereClause)
             {
-                CFDefinition.Name name = cfDef.get(rel.getEntity());
-                if (name == null)
+                ColumnDefinition def = cfm.getColumnDefinition(rel.getEntity());
+                if (def == null)
                 {
                     if (containsAlias(rel.getEntity()))
                         throw new InvalidRequestException(String.format("Aliases aren't allowed in where clause ('%s')", rel));
@@ -1104,32 +1102,31 @@ public class SelectStatement implements CQLStatement
                         throw new InvalidRequestException(String.format("Undefined name %s in where clause ('%s')", rel.getEntity(), rel));
                 }
 
-                ColumnDefinition def = cfDef.cfm.getColumnDefinition(name.name.key);
-                stmt.restrictedNames.add(name);
+                stmt.restrictedColumns.add(def);
                 if (def.isIndexed() && rel.operator() == Relation.Type.EQ)
                 {
                     hasQueriableIndex = true;
-                    if (name.kind == CFDefinition.Name.Kind.COLUMN_ALIAS)
+                    if (def.kind == ColumnDefinition.Kind.CLUSTERING_COLUMN)
                         hasQueriableClusteringColumnIndex = true;
                 }
 
-                switch (name.kind)
+                switch (def.kind)
                 {
-                    case KEY_ALIAS:
-                        stmt.keyRestrictions[name.position] = updateRestriction(name, stmt.keyRestrictions[name.position], rel, names);
+                    case PARTITION_KEY:
+                        stmt.keyRestrictions[def.position()] = updateRestriction(def, stmt.keyRestrictions[def.position()], rel, names);
                         break;
-                    case COLUMN_ALIAS:
-                        stmt.columnRestrictions[name.position] = updateRestriction(name, stmt.columnRestrictions[name.position], rel, names);
+                    case CLUSTERING_COLUMN:
+                        stmt.columnRestrictions[def.position()] = updateRestriction(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:
                         // 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(name, stmt.metadataRestrictions.get(name), rel, names);
+                        Restriction r = updateRestriction(def, stmt.metadataRestrictions.get(def), 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;
                 }
             }
@@ -1145,12 +1142,12 @@ public class SelectStatement implements CQLStatement
             // If a component of the partition key is restricted by a relation, all preceding
             // components must have a EQ. Only the last partition key component can be in IN relation.
             boolean canRestrictFurtherComponents = true;
-            CFDefinition.Name previous = null;
+            ColumnDefinition previous = null;
             stmt.keyIsInRelation = false;
-            Iterator<CFDefinition.Name> iter = cfDef.keys.values().iterator();
+            Iterator<ColumnDefinition> iter = cfm.partitionKeyColumns().iterator();
             for (int i = 0; i < stmt.keyRestrictions.length; i++)
             {
-                CFDefinition.Name cname = iter.next();
+                ColumnDefinition cdef = iter.next();
                 Restriction restriction = stmt.keyRestrictions[i];
 
                 if (restriction == null)
@@ -1167,7 +1164,7 @@ public class SelectStatement implements CQLStatement
                             stmt.isKeyRange = true;
                             break;
                         }
-                        throw new InvalidRequestException(String.format("Partition key part %s must be restricted since preceding part is", cname));
+                        throw new InvalidRequestException(String.format("Partition key part %s must be restricted since preceding part is", cdef.name));
                     }
 
                     stmt.isKeyRange = true;
@@ -1180,7 +1177,7 @@ public class SelectStatement implements CQLStatement
                         stmt.usesSecondaryIndexing = true;
                         break;
                     }
-                    throw new InvalidRequestException(String.format("partition key part %s cannot be restricted (preceding part %s is either not restricted or by a non-EQ relation)", cname, previous));
+                    throw new InvalidRequestException(String.format("partition key part %s cannot be restricted (preceding part %s is either not restricted or by a non-EQ relation)", cdef.name, previous));
                 }
                 else if (restriction.isOnToken())
                 {
@@ -1198,7 +1195,7 @@ public class SelectStatement implements CQLStatement
                     {
                         // We only support IN for the last name so far
                         if (i != stmt.keyRestrictions.length - 1)
-                            throw new InvalidRequestException(String.format("Partition KEY part %s cannot be restricted by IN relation (only the last part of the partition key can)", cname));
+                            throw new InvalidRequestException(String.format("Partition KEY part %s cannot be restricted by IN relation (only the last part of the partition key can)", cdef.name));
                         stmt.keyIsInRelation = true;
                     }
                 }
@@ -1208,23 +1205,23 @@ public class SelectStatement implements CQLStatement
                     // Note: In theory we could allow it for 2ndary index queries with ALLOW FILTERING, but that would probably require some special casing
                     throw new InvalidRequestException("Only EQ and IN relation are supported on the partition key (unless you use the token() function)");
                 }
-                previous = cname;
+                previous = cdef;
             }
 
             // 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.restrictedColumns.removeAll(cfm.partitionKeyColumns());
 
             // 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.
             canRestrictFurtherComponents = true;
             previous = null;
-            iter = cfDef.columns.values().iterator();
+            iter = cfm.clusteringColumns().iterator();
             for (int i = 0; i < stmt.columnRestrictions.length; i++)
             {
-                CFDefinition.Name cname = iter.next();
+                ColumnDefinition cdef = iter.next();
                 Restriction restriction = stmt.columnRestrictions[i];
 
                 if (restriction == null)
@@ -1238,7 +1235,7 @@ public class SelectStatement implements CQLStatement
                         stmt.usesSecondaryIndexing = true; // handle gaps and non-keyrange cases.
                         break;
                     }
-                    throw new InvalidRequestException(String.format("PRIMARY KEY part %s cannot be restricted (preceding part %s is either not restricted or by a non-EQ relation)", cname, previous));
+                    throw new InvalidRequestException(String.format("PRIMARY KEY part %s cannot be restricted (preceding part %s is either not restricted or by a non-EQ relation)", cdef.name, previous));
                 }
                 else if (restriction.isSlice())
                 {
@@ -1246,7 +1243,7 @@ public class SelectStatement implements CQLStatement
                     Restriction.Slice slice = (Restriction.Slice)restriction;
                     // For non-composite slices, we don't support internally the difference between exclusive and
                     // inclusive bounds, so we deal with it manually.
-                    if (!cfDef.isComposite && (!slice.isInclusive(Bound.START) || !slice.isInclusive(Bound.END)))
+                    if (!cfm.hasCompositeComparator() && (!slice.isInclusive(Bound.START) || !slice.isInclusive(Bound.END)))
                         stmt.sliceRestriction = slice;
                 }
                 else if (restriction.isIN())
@@ -1254,12 +1251,12 @@ public class SelectStatement implements CQLStatement
                     // We only support IN for the last name and for compact storage so far
                     // TODO: #3885 allows us to extend to non compact as well, but that remains to be done
                     if (i != stmt.columnRestrictions.length - 1)
-                        throw new InvalidRequestException(String.format("PRIMARY KEY part %s cannot be restricted by IN relation", cname));
+                        throw new InvalidRequestException(String.format("PRIMARY KEY part %s cannot be restricted by IN relation", cdef.name));
                     else if (stmt.selectACollection())
-                        throw new InvalidRequestException(String.format("Cannot restrict PRIMARY KEY part %s by IN relation as a collection is selected by the query", cname));
+                        throw new InvalidRequestException(String.format("Cannot restrict PRIMARY KEY part %s by IN relation as a collection is selected by the query", cdef.name));
                 }
 
-                previous = cname;
+                previous = cdef;
             }
 
             // Covers indexes on the first clustering column (among others).
@@ -1267,7 +1264,7 @@ public class SelectStatement implements CQLStatement
                 stmt.usesSecondaryIndexing = true;
 
             if (!stmt.usesSecondaryIndexing)
-                stmt.restrictedNames.removeAll(cfDef.columns.values());
+                stmt.restrictedColumns.removeAll(cfm.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.
@@ -1294,11 +1291,11 @@ public class SelectStatement implements CQLStatement
                 // queried automatically, and then removing it from the resultSet afterwards if needed)
                 if (stmt.keyIsInRelation)
                 {
-                    stmt.orderingIndexes = new HashMap<CFDefinition.Name, Integer>();
+                    stmt.orderingIndexes = new HashMap<ColumnDefinition, Integer>();
                     for (ColumnIdentifier column : stmt.parameters.orderings.keySet())
                     {
-                        final CFDefinition.Name name = cfDef.get(column);
-                        if (name == null)
+                        final ColumnDefinition def = cfm.getColumnDefinition(column);
+                        if (def == null)
                         {
                             if (containsAlias(column))
                                 throw new InvalidRequestException(String.format("Aliases are not allowed in order by clause ('%s')", column));
@@ -1308,13 +1305,14 @@ public class SelectStatement implements CQLStatement
 
                         if (selectClause.isEmpty()) // wildcard
                         {
-                            stmt.orderingIndexes.put(name, Iterables.indexOf(cfDef, new Predicate<CFDefinition.Name>()
-                                                                                    {
-                                                                                        public boolean apply(CFDefinition.Name n)
-                                                                                        {
-                                                                                            return name.equals(n);
-                                                                                        }
-                                                                                    }));
+                            stmt.orderingIndexes.put(def, Iterators.indexOf(cfm.allColumnsInSelectOrder(),
+                                                                            new Predicate<ColumnDefinition>()
+                                                                            {
+                                                                                public boolean apply(ColumnDefinition n)
+                                                                                {
+                                                                                    return def.equals(n);
+                                                                                }
+                                                                            }));
                         }
                         else
                         {
@@ -1322,9 +1320,9 @@ public class SelectStatement implements CQLStatement
                             for (int i = 0; i < selectClause.size(); i++)
                             {
                                 RawSelector selector = selectClause.get(i);
-                                if (name.name.equals(selector.selectable))
+                                if (def.name.equals(selector.selectable))
                                 {
-                                    stmt.orderingIndexes.put(name, i);
+                                    stmt.orderingIndexes.put(def, i);
                                     hasColumn = true;
                                     break;
                                 }
@@ -1336,15 +1334,15 @@ public class SelectStatement implements CQLStatement
                     }
                 }
 
-                Boolean[] reversedMap = new Boolean[cfDef.columns.size()];
+                Boolean[] reversedMap = new Boolean[cfm.clusteringColumns().size()];
                 int i = 0;
                 for (Map.Entry<ColumnIdentifier, Boolean> entry : stmt.parameters.orderings.entrySet())
                 {
                     ColumnIdentifier column = entry.getKey();
                     boolean reversed = entry.getValue();
 
-                    CFDefinition.Name name = cfDef.get(column);
-                    if (name == null)
+                    ColumnDefinition def = cfm.getColumnDefinition(column);
+                    if (def == null)
                     {
                         if (containsAlias(column))
                             throw new InvalidRequestException(String.format("Aliases are not allowed in order by clause ('%s')", column));
@@ -1352,13 +1350,13 @@ public class SelectStatement implements CQLStatement
                             throw new InvalidRequestException(String.format("Order by on unknown column %s", column));
                     }
 
-                    if (name.kind != CFDefinition.Name.Kind.COLUMN_ALIAS)
+                    if (def.kind != ColumnDefinition.Kind.CLUSTERING_COLUMN)
                         throw new InvalidRequestException(String.format("Order by is currently only supported on the clustered columns of the PRIMARY KEY, got %s", column));
 
-                    if (i++ != name.position)
+                    if (i++ != def.position())
                         throw new InvalidRequestException(String.format("Order by currently only support the ordering of columns following their declared order in the PRIMARY KEY"));
 
-                    reversedMap[name.position] = (reversed != isReversedType(name));
+                    reversedMap[def.position()] = (reversed != isReversedType(def));
                 }
 
                 // Check that all boolean in reversedMap, if set, agrees
@@ -1387,7 +1385,7 @@ public class SelectStatement implements CQLStatement
                 // We will potentially filter data if either:
                 //  - Have more than one IndexExpression
                 //  - Have no index expression and the column filter is not the identity
-                if (stmt.restrictedNames.size() > 1 || (stmt.restrictedNames.isEmpty() && !stmt.columnFilterIsIdentity()))
+                if (stmt.restrictedColumns.size() > 1 || (stmt.restrictedColumns.isEmpty() && !stmt.columnFilterIsIdentity()))
                     throw new InvalidRequestException("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. "
                                                     + "If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING");
             }
@@ -1395,16 +1393,16 @@ public class SelectStatement implements CQLStatement
             return new ParsedStatement.Prepared(stmt, names);
         }
 
-        private void validateDistinctSelection(Collection<CFDefinition.Name> requestedColumns, Collection<CFDefinition.Name> partitionKey)
+        private void validateDistinctSelection(Collection<ColumnDefinition> requestedColumns, Collection<ColumnDefinition> partitionKey)
         throws InvalidRequestException
         {
-            for (CFDefinition.Name name : requestedColumns)
-                if (!partitionKey.contains(name))
-                    throw new InvalidRequestException(String.format("SELECT DISTINCT queries must only request partition key columns (not %s)", name));
+            for (ColumnDefinition def : requestedColumns)
+                if (!partitionKey.contains(def))
+                    throw new InvalidRequestException(String.format("SELECT DISTINCT queries must only request partition key columns (not %s)", def.name));
 
-            for (CFDefinition.Name name : partitionKey)
-                if (!requestedColumns.contains(name))
-                    throw new InvalidRequestException(String.format("SELECT DISTINCT queries must request all the partition key columns (missing %s)", name));
+            for (ColumnDefinition def : partitionKey)
+                if (!requestedColumns.contains(def))
+                    throw new InvalidRequestException(String.format("SELECT DISTINCT queries must request all the partition key columns (missing %s)", def.name));
         }
 
         private boolean containsAlias(final ColumnIdentifier name)
@@ -1423,16 +1421,16 @@ public class SelectStatement implements CQLStatement
             return new ColumnSpecification(keyspace(), columnFamily(), new ColumnIdentifier("[limit]", true), Int32Type.instance);
         }
 
-        Restriction updateRestriction(CFDefinition.Name name, Restriction restriction, Relation newRel, VariableSpecifications boundNames) throws InvalidRequestException
+        Restriction updateRestriction(ColumnDefinition def, Restriction restriction, Relation newRel, VariableSpecifications boundNames) throws InvalidRequestException
         {
-            ColumnSpecification receiver = name;
+            ColumnSpecification receiver = def;
             if (newRel.onToken)
             {
-                if (name.kind != CFDefinition.Name.Kind.KEY_ALIAS)
-                    throw new InvalidRequestException(String.format("The token() function is only supported on the partition key, found on %s", name));
+                if (def.kind != ColumnDefinition.Kind.PARTITION_KEY)
+                    throw new InvalidRequestException(String.format("The token() function is only supported on the partition key, found on %s", def.name));
 
-                receiver = new ColumnSpecification(name.ksName,
-                                                   name.cfName,
+                receiver = new ColumnSpecification(def.ksName,
+                                                   def.cfName,
                                                    new ColumnIdentifier("partition key token", true),
                                                    StorageService.getPartitioner().getTokenValidator());
             }
@@ -1442,7 +1440,7 @@ public class SelectStatement implements CQLStatement
                 case EQ:
                     {
                         if (restriction != null)
-                            throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes an Equal", name));
+                            throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes an Equal", def.name));
                         Term t = newRel.getValue().prepare(receiver);
                         t.collectMarkerSpecification(boundNames);
                         restriction = new Restriction.EQ(t, newRel.onToken);
@@ -1450,7 +1448,7 @@ public class SelectStatement implements CQLStatement
                     break;
                 case IN:
                     if (restriction != null)
-                        throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes a IN", name));
+                        throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes a IN", def.name));
 
                     if (newRel.getInValues() == null)
                     {
@@ -1480,10 +1478,10 @@ public class SelectStatement implements CQLStatement
                         if (restriction == null)
                             restriction = new Restriction.Slice(newRel.onToken);
                         else if (!restriction.isSlice())
-                            throw new InvalidRequestException(String.format("%s cannot be restricted by both an equal and an inequal relation", name));
+                            throw new InvalidRequestException(String.format("%s cannot be restricted by both an equal and an inequal relation", def.name));
                         Term t = newRel.getValue().prepare(receiver);
                         t.collectMarkerSpecification(boundNames);
-                        ((Restriction.Slice)restriction).setBound(name.name, newRel.operator(), t);
+                        ((Restriction.Slice)restriction).setBound(def.name, newRel.operator(), t);
                     }
                     break;
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5f5905d5/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..7f13d29 100644
--- a/src/java/org/apache/cassandra/cql3/statements/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Selection.java
@@ -21,9 +21,13 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
+import com.google.common.collect.Iterators;
+
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.functions.Functions;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.CounterColumn;
 import org.apache.cassandra.db.ExpiringColumn;
 import org.apache.cassandra.db.Column;
@@ -36,12 +40,12 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 
 public abstract class Selection
 {
-    private final List<CFDefinition.Name> columnsList;
+    private final List<ColumnDefinition> columnsList;
     private final List<ColumnSpecification> metadata;
     private final boolean collectTimestamps;
     private final boolean collectTTLs;
 
-    protected Selection(List<CFDefinition.Name> columnsList, List<ColumnSpecification> metadata, boolean collectTimestamps, boolean collectTTLs)
+    protected Selection(List<ColumnDefinition> columnsList, List<ColumnSpecification> metadata, boolean collectTimestamps, boolean collectTTLs)
     {
         this.columnsList = columnsList;
         this.metadata = metadata;
@@ -54,15 +58,14 @@ public abstract class Selection
         return new ResultSet.Metadata(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);
     }
 
-    public static Selection forColumns(List<CFDefinition.Name> columnsList)
+    public static Selection forColumns(List<ColumnDefinition> columnsList)
     {
         return new SimpleSelection(columnsList);
     }
@@ -77,54 +80,54 @@ public abstract class Selection
         return false;
     }
 
-    private static int addAndGetIndex(CFDefinition.Name name, List<CFDefinition.Name> l)
+    private static int addAndGetIndex(ColumnDefinition def, List<ColumnDefinition> l)
     {
-        int idx = l.indexOf(name);
+        int idx = l.indexOf(def);
         if (idx < 0)
         {
             idx = l.size();
-            l.add(name);
+            l.add(def);
         }
         return idx;
     }
 
-    private static Selector makeSelector(CFDefinition cfDef, RawSelector raw, List<CFDefinition.Name> names, List<ColumnSpecification> metadata) throws InvalidRequestException
+    private static Selector makeSelector(CFMetaData cfm, RawSelector raw, List<ColumnDefinition> defs, List<ColumnSpecification> metadata) throws InvalidRequestException
     {
         if (raw.selectable instanceof ColumnIdentifier)
         {
-            CFDefinition.Name name = cfDef.get((ColumnIdentifier)raw.selectable);
-            if (name == null)
+            ColumnDefinition def = cfm.getColumnDefinition((ColumnIdentifier)raw.selectable);
+            if (def == null)
                 throw new InvalidRequestException(String.format("Undefined name %s in selection clause", raw.selectable));
             if (metadata != null)
-                metadata.add(raw.alias == null ? name : makeAliasSpec(cfDef, name.type, raw.alias));
-            return new SimpleSelector(name.toString(), addAndGetIndex(name, names), name.type);
+                metadata.add(raw.alias == null ? def : makeAliasSpec(cfm, def.type, raw.alias));
+            return new SimpleSelector(def.name.toString(), addAndGetIndex(def, defs), def.type);
         }
         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 (name.kind != CFDefinition.Name.Kind.COLUMN_METADATA && name.kind != CFDefinition.Name.Kind.VALUE_ALIAS)
-                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.kind != ColumnDefinition.Kind.REGULAR && def.kind != ColumnDefinition.Kind.COMPACT_VALUE)
+                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)
-                metadata.add(makeWritetimeOrTTLSpec(cfDef, tot, raw.alias));
-            return new WritetimeOrTTLSelector(name.toString(), addAndGetIndex(name, names), tot.isWritetime);
+                metadata.add(makeWritetimeOrTTLSpec(cfm, tot, raw.alias));
+            return new WritetimeOrTTLSelector(def.name.toString(), addAndGetIndex(def, defs), tot.isWritetime);
         }
         else
         {
             Selectable.WithFunction withFun = (Selectable.WithFunction)raw.selectable;
             List<Selector> args = new ArrayList<Selector>(withFun.args.size());
             for (Selectable rawArg : withFun.args)
-                args.add(makeSelector(cfDef, new RawSelector(rawArg, null), names, null));
+                args.add(makeSelector(cfm, new RawSelector(rawArg, null), defs, null));
 
-            AbstractType<?> returnType = Functions.getReturnType(withFun.functionName, cfDef.cfm.ksName, cfDef.cfm.cfName);
+            AbstractType<?> returnType = Functions.getReturnType(withFun.functionName, cfm.ksName, cfm.cfName);
             if (returnType == null)
                 throw new InvalidRequestException(String.format("Unknown function '%s'", withFun.functionName));
-            ColumnSpecification spec = makeFunctionSpec(cfDef, withFun, returnType, raw.alias);
+            ColumnSpecification spec = makeFunctionSpec(cfm, withFun, returnType, raw.alias);
             Function fun = Functions.get(withFun.functionName, args, spec);
             if (metadata != null)
                 metadata.add(spec);
@@ -132,15 +135,15 @@ public abstract class Selection
         }
     }
 
-    private static ColumnSpecification makeWritetimeOrTTLSpec(CFDefinition cfDef, Selectable.WritetimeOrTTL tot, ColumnIdentifier alias)
+    private static ColumnSpecification makeWritetimeOrTTLSpec(CFMetaData cfm, Selectable.WritetimeOrTTL tot, ColumnIdentifier alias)
     {
-        return new ColumnSpecification(cfDef.cfm.ksName,
-                                       cfDef.cfm.cfName,
+        return new ColumnSpecification(cfm.ksName,
+                                       cfm.cfName,
                                        alias == null ? new ColumnIdentifier(tot.toString(), true) : alias,
                                        tot.isWritetime ? LongType.instance : Int32Type.instance);
     }
 
-    private static ColumnSpecification makeFunctionSpec(CFDefinition cfDef,
+    private static ColumnSpecification makeFunctionSpec(CFMetaData cfm,
                                                         Selectable.WithFunction fun,
                                                         AbstractType<?> returnType,
                                                         ColumnIdentifier alias) throws InvalidRequestException
@@ -148,31 +151,31 @@ public abstract class Selection
         if (returnType == null)
             throw new InvalidRequestException(String.format("Unknown function %s called in selection clause", fun.functionName));
 
-        return new ColumnSpecification(cfDef.cfm.ksName,
-                                       cfDef.cfm.cfName,
+        return new ColumnSpecification(cfm.ksName,
+                                       cfm.cfName,
                                        alias == null ? new ColumnIdentifier(fun.toString(), true) : alias,
                                        returnType);
     }
 
-    private static ColumnSpecification makeAliasSpec(CFDefinition cfDef, AbstractType<?> type, ColumnIdentifier alias)
+    private static ColumnSpecification makeAliasSpec(CFMetaData cfm, AbstractType<?> type, ColumnIdentifier alias)
     {
-        return new ColumnSpecification(cfDef.cfm.ksName, cfDef.cfm.cfName, alias, type);
+        return new ColumnSpecification(cfm.ksName, cfm.cfName, alias, type);
     }
 
-    public static Selection fromSelectors(CFDefinition cfDef, List<RawSelector> rawSelectors) throws InvalidRequestException
+    public static Selection fromSelectors(CFMetaData cfm, List<RawSelector> rawSelectors) throws InvalidRequestException
     {
         boolean usesFunction = isUsingFunction(rawSelectors);
 
         if (usesFunction)
         {
-            List<CFDefinition.Name> names = new ArrayList<CFDefinition.Name>();
+            List<ColumnDefinition> defs = new ArrayList<ColumnDefinition>();
             List<ColumnSpecification> metadata = new ArrayList<ColumnSpecification>(rawSelectors.size());
             List<Selector> selectors = new ArrayList<Selector>(rawSelectors.size());
             boolean collectTimestamps = false;
             boolean collectTTLs = false;
             for (RawSelector rawSelector : rawSelectors)
             {
-                Selector selector = makeSelector(cfDef, rawSelector, names, metadata);
+                Selector selector = makeSelector(cfm, rawSelector, defs, metadata);
                 selectors.add(selector);
                 if (selector instanceof WritetimeOrTTLSelector)
                 {
@@ -180,22 +183,22 @@ public abstract class Selection
                     collectTTLs |= !((WritetimeOrTTLSelector)selector).isWritetime;
                 }
             }
-            return new SelectionWithFunctions(names, metadata, selectors, collectTimestamps, collectTTLs);
+            return new SelectionWithFunctions(defs, metadata, selectors, collectTimestamps, collectTTLs);
         }
         else
         {
-            List<CFDefinition.Name> names = new ArrayList<CFDefinition.Name>(rawSelectors.size());
+            List<ColumnDefinition> defs = new ArrayList<ColumnDefinition>(rawSelectors.size());
             List<ColumnSpecification> metadata = new ArrayList<ColumnSpecification>(rawSelectors.size());
             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(names, metadata);
+            return new SimpleSelection(defs, metadata);
         }
     }
 
@@ -207,10 +210,10 @@ public abstract class Selection
     public List<ColumnIdentifier> regularColumnsToFetch()
     {
         List<ColumnIdentifier> toFetch = new ArrayList<ColumnIdentifier>();
-        for (CFDefinition.Name name : columnsList)
+        for (ColumnDefinition def : columnsList)
         {
-            if (name.kind == CFDefinition.Name.Kind.COLUMN_METADATA)
-                toFetch.add(name.name);
+            if (def.kind == ColumnDefinition.Kind.REGULAR)
+                toFetch.add(def.name);
         }
         return toFetch;
     }
@@ -218,7 +221,7 @@ public abstract class Selection
     /**
      * @return the list of CQL3 columns value this SelectionClause needs.
      */
-    public List<CFDefinition.Name> getColumnsList()
+    public List<ColumnDefinition> getColumnsList()
     {
         return columnsList;
     }
@@ -307,12 +310,12 @@ 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)
+        public SimpleSelection(List<ColumnDefinition> columnsList)
         {
             this(columnsList, new ArrayList<ColumnSpecification>(columnsList));
         }
 
-        public SimpleSelection(List<CFDefinition.Name> columnsList, List<ColumnSpecification> metadata)
+        public SimpleSelection(List<ColumnDefinition> columnsList, List<ColumnSpecification> metadata)
         {
             /*
              * In theory, even a simple selection could have multiple time the same column, so we
@@ -444,7 +447,7 @@ public abstract class Selection
     {
         private final List<Selector> selectors;
 
-        public SelectionWithFunctions(List<CFDefinition.Name> columnsList, List<ColumnSpecification> metadata, List<Selector> selectors, boolean collectTimestamps, boolean collectTTLs)
+        public SelectionWithFunctions(List<ColumnDefinition> columnsList, List<ColumnSpecification> metadata, List<Selector> selectors, boolean collectTimestamps, boolean collectTTLs)
         {
             super(columnsList, metadata, collectTimestamps, collectTTLs);
             this.selectors = selectors;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5f5905d5/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 12348df..7a8340d 100644
--- a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
@@ -22,7 +22,9 @@ import java.util.*;
 
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
@@ -48,8 +50,6 @@ public class UpdateStatement extends ModificationStatement
     public void addUpdateForKey(ColumnFamily cf, ByteBuffer key, ColumnNameBuilder builder, UpdateParameters params)
     throws InvalidRequestException
     {
-        CFDefinition cfDef = cfm.getCfDef();
-
         // Inserting the CQL row marker (see #4361)
         // We always need to insert a marker, because of the following situation:
         //   CREATE TABLE t ( k int PRIMARY KEY, c text );
@@ -61,7 +61,7 @@ public class UpdateStatement extends ModificationStatement
         // '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 (cfDef.isComposite && !cfDef.isCompact && !cfm.isSuper())
+        if (cfm.hasCompositeComparator() && !cfm.isDense() && !cfm.isSuper())
         {
             ByteBuffer name = builder.copy().add(ByteBufferUtil.EMPTY_BYTE_BUFFER).build();
             cf.addColumn(params.makeColumn(name, ByteBufferUtil.EMPTY_BYTE_BUFFER));
@@ -69,23 +69,24 @@ public class UpdateStatement extends ModificationStatement
 
         List<Operation> updates = getOperations();
 
-        if (cfDef.isCompact)
+        if (cfm.isDense())
         {
             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", cfm.clusteringColumns().iterator().next()));
 
-            if (cfDef.value == null)
+            // An empty name for the compact value is what we use to recognize the case where there is not column
+            // outside the PK, see CreateStatement.
+            if (!cfm.compactValueColumn().name.bytes.hasRemaining())
             {
-                // compact + no compact value implies there is no column outside the PK. So no operation could
-                // have passed through validation
+                // There is no column outside the PK. So no operation could have passed through validation
                 assert updates.isEmpty();
                 setToEmptyOperation.execute(key, cf, builder.copy(), params);
             }
             else
             {
-                // compact means we don't have a row marker, so don't accept to set only the PK. See CASSANDRA-5648.
+                // dense 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", cfm.compactValueColumn().name));
 
                 for (Operation update : updates)
                     update.execute(key, cf, builder.copy(), params);
@@ -129,9 +130,9 @@ public class UpdateStatement extends ModificationStatement
             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(), cfDef.cfm, attrs);
+            UpdateStatement stmt = new UpdateStatement(boundNames.size(), cfm, attrs);
 
             // Created from an INSERT
             if (stmt.isCounter())
@@ -143,27 +144,27 @@ public class UpdateStatement extends ModificationStatement
 
             for (int i = 0; i < columnNames.size(); i++)
             {
-                CFDefinition.Name name = cfDef.get(columnNames.get(i));
-                if (name == null)
+                ColumnDefinition def = cfm.getColumnDefinition(columnNames.get(i));
+                if (def == null)
                     throw new InvalidRequestException(String.format("Unknown identifier %s", columnNames.get(i)));
 
                 for (int j = 0; j < i; j++)
-                    if (name.name.equals(columnNames.get(j)))
-                        throw new InvalidRequestException(String.format("Multiple definitions found for column %s", name));
+                    if (def.name.equals(columnNames.get(j)))
+                        throw new InvalidRequestException(String.format("Multiple definitions found for column %s", def.name));
 
                 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(def);
                         t.collectMarkerSpecification(boundNames);
-                        stmt.addKeyValue(name.name, t);
+                        stmt.addKeyValue(def.name, t);
                         break;
-                    case VALUE_ALIAS:
-                    case COLUMN_METADATA:
-                        Operation operation = new Operation.SetValue(value).prepare(name);
+                    case COMPACT_VALUE:
+                    case REGULAR:
+                        Operation operation = new Operation.SetValue(value).prepare(def);
                         operation.collectMarkerSpecification(boundNames);
                         stmt.addOperation(operation);
                         break;
@@ -199,26 +200,26 @@ public class UpdateStatement extends ModificationStatement
             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(), cfDef.cfm, attrs);
+            UpdateStatement stmt = new UpdateStatement(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(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 VALUE_ALIAS:
-                    case COLUMN_METADATA:
+                    case COMPACT_VALUE:
+                    case REGULAR:
                         stmt.addOperation(operation);
                         break;
                 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5f5905d5/src/java/org/apache/cassandra/db/BatchlogManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BatchlogManager.java b/src/java/org/apache/cassandra/db/BatchlogManager.java
index 0968df2..bba3ffe 100644
--- a/src/java/org/apache/cassandra/db/BatchlogManager.java
+++ b/src/java/org/apache/cassandra/db/BatchlogManager.java
@@ -284,7 +284,7 @@ public class BatchlogManager implements BatchlogManagerMBean
 
     private static ByteBuffer columnName(String name)
     {
-        return CFMetaData.BatchlogCf.getCfDef().getColumnNameBuilder().add(UTF8Type.instance.decompose(name)).build();
+        return CFMetaData.BatchlogCf.getColumnNameBuilder().add(UTF8Type.instance.decompose(name)).build();
     }
 
     // force flush + compaction to reclaim space from the replayed batches

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5f5905d5/src/java/org/apache/cassandra/db/Column.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Column.java b/src/java/org/apache/cassandra/db/Column.java
index b0d22fb..e103cd3 100644
--- a/src/java/org/apache/cassandra/db/Column.java
+++ b/src/java/org/apache/cassandra/db/Column.java
@@ -30,7 +30,6 @@ import java.util.List;
 import com.google.common.collect.AbstractIterator;
 
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.cql3.CFDefinition;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.util.DataOutputBuffer;
@@ -298,15 +297,8 @@ public class Column implements OnDiskAtom
     public void validateFields(CFMetaData metadata) throws MarshalException
     {
         validateName(metadata);
-        CFDefinition cfdef = metadata.getCfDef();
 
-        // If this is a CQL table, we need to pull out the CQL column name to look up the correct column type.
-        // (Note that COMPACT composites are handled by validateName, above.)
-        ByteBuffer internalName = (cfdef.isComposite && !cfdef.isCompact)
-                                ? ((CompositeType) metadata.comparator).extractLastComponent(name)
-                                : name;
-
-        AbstractType<?> valueValidator = metadata.getValueValidator(internalName);
+        AbstractType<?> valueValidator = metadata.getValueValidatorFromCellName(name());
         if (valueValidator != null)
             valueValidator.validate(value());
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5f5905d5/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java b/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
index 7f8693e..89d6683 100644
--- a/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
@@ -180,7 +180,7 @@ public abstract class ExtendedFilter
              * We also don't want to do for paging ranges as the actual filter depends on the row key (it would
              * probably be possible to make it work but we won't really use it so we don't bother).
              */
-            if (cfs.getComparator() instanceof CompositeType || dataRange instanceof DataRange.Paging)
+            if (cfs.metadata.hasCompositeComparator() || dataRange instanceof DataRange.Paging)
                 return null;
 
             IDiskAtomFilter filter = dataRange.columnFilter(null); // ok since not a paging range
@@ -251,7 +251,7 @@ public abstract class ExtendedFilter
              * 2) We don't yet allow non-indexed range slice with filters in CQL3 (i.e. this will never be
              * called by CFS.filter() for composites).
              */
-            assert !(cfs.getComparator() instanceof CompositeType);
+            assert !cfs.metadata.hasCompositeComparator();
 
             if (!needsExtraQuery(rowKey.key, data))
                 return null;
@@ -302,7 +302,7 @@ public abstract class ExtendedFilter
                 else
                 {
                     dataValue = extractDataValue(def, rowKey.key, data, builder);
-                    validator = def.getValidator();
+                    validator = def.type;
                 }
 
                 if (dataValue == null)
@@ -317,16 +317,16 @@ public abstract class ExtendedFilter
 
         private ByteBuffer extractDataValue(ColumnDefinition def, ByteBuffer rowKey, ColumnFamily data, ColumnNameBuilder builder)
         {
-            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 builder.get(def.position());
                 case REGULAR:
-                    ByteBuffer colName = builder == null ? def.name : builder.copy().add(def.name).build();
+                    ByteBuffer colName = builder == null ? def.name.bytes : builder.copy().add(def.name).build();
                     Column column = data.getColumn(colName);
                     return column == null ? null : column.value();
                 case COMPACT_VALUE:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5f5905d5/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java b/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
index ebd4730..699b391 100644
--- a/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
@@ -50,7 +50,7 @@ public abstract class AbstractSimplePerColumnSecondaryIndex extends PerColumnSec
         CFMetaData indexedCfMetadata = CFMetaData.newIndexMetadata(baseCfs.metadata, columnDef, indexComparator);
         indexCfs = ColumnFamilyStore.createColumnFamilyStore(baseCfs.keyspace,
                                                              indexedCfMetadata.cfName,
-                                                             new LocalPartitioner(columnDef.getValidator()),
+                                                             new LocalPartitioner(columnDef.type),
                                                              indexedCfMetadata);
     }
 
@@ -66,7 +66,7 @@ public abstract class AbstractSimplePerColumnSecondaryIndex extends PerColumnSec
                              baseCfs.name,
                              getExpressionComparator().getString(expr.column),
                              expr.operator,
-                             baseCfs.metadata.getColumnDefinition(expr.column).getValidator().getString(expr.value));
+                             baseCfs.metadata.getColumnDefinition(expr.column).type.getString(expr.value));
     }
 
     public void delete(ByteBuffer rowKey, Column column)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5f5905d5/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/SecondaryIndex.java b/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
index 10f0e95..7a822dc 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
@@ -118,13 +118,13 @@ public abstract class SecondaryIndex
     public void setIndexBuilt()
     {
         for (ColumnDefinition columnDef : columnDefs)
-            SystemKeyspace.setIndexBuilt(baseCfs.keyspace.getName(), getNameForSystemKeyspace(columnDef.name));
+            SystemKeyspace.setIndexBuilt(baseCfs.keyspace.getName(), getNameForSystemKeyspace(columnDef.name.bytes));
     }
 
     public void setIndexRemoved()
     {
         for (ColumnDefinition columnDef : columnDefs)
-            SystemKeyspace.setIndexRemoved(baseCfs.keyspace.getName(), getNameForSystemKeyspace(columnDef.name));
+            SystemKeyspace.setIndexRemoved(baseCfs.keyspace.getName(), getNameForSystemKeyspace(columnDef.name.bytes));
     }
 
     /**
@@ -204,7 +204,7 @@ public abstract class SecondaryIndex
         boolean allAreBuilt = true;
         for (ColumnDefinition cdef : columnDefs)
         {
-            if (!SystemKeyspace.isIndexBuilt(baseCfs.keyspace.getName(), getNameForSystemKeyspace(cdef.name)))
+            if (!SystemKeyspace.isIndexBuilt(baseCfs.keyspace.getName(), getNameForSystemKeyspace(cdef.name.bytes)))
             {
                 allAreBuilt = false;
                 break;
@@ -268,8 +268,8 @@ public abstract class SecondaryIndex
     public DecoratedKey getIndexKeyFor(ByteBuffer value)
     {
         // FIXME: this imply one column definition per index
-        ByteBuffer name = columnDefs.iterator().next().name;
-        return new DecoratedKey(new LocalToken(baseCfs.metadata.getColumnDefinition(name).getValidator(), value), value);
+        ByteBuffer name = columnDefs.iterator().next().name.bytes;
+        return new DecoratedKey(new LocalToken(baseCfs.metadata.getColumnDefinition(name).type, value), value);
     }
 
     /**
@@ -282,7 +282,7 @@ public abstract class SecondaryIndex
     {
         for (ColumnDefinition columnDef : columnDefs)
         {
-            if (baseCfs.getComparator().compare(columnDef.name, name) == 0)
+            if (baseCfs.getComparator().compare(columnDef.name.bytes, name) == 0)
                 return true;
         }
         return false;


Mime
View raw message