You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ty...@apache.org on 2014/11/05 00:27:03 UTC

[1/3] git commit: Fix non-text comparator col names in cql3 queries

Repository: cassandra
Updated Branches:
  refs/heads/trunk 254cd85a5 -> 9bf17e15a


Fix non-text comparator col names in cql3 queries

Patch by Tyler Hobbs; reviewed by Aleksey Yeschenko for CASSANDRA-8178


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

Branch: refs/heads/trunk
Commit: 11e657189addf1a7da3c5f32719f1cedbcf3d2e3
Parents: e019760
Author: Tyler Hobbs <ty...@datastax.com>
Authored: Tue Nov 4 16:42:05 2014 -0600
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Tue Nov 4 16:42:05 2014 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/cql3/ColumnIdentifier.java | 66 +++++++++++++++
 src/java/org/apache/cassandra/cql3/Cql.g        | 85 +++++++++++---------
 .../cassandra/cql3/MultiColumnRelation.java     | 12 +--
 .../org/apache/cassandra/cql3/Operation.java    | 29 +++----
 .../cassandra/cql3/SingleColumnRelation.java    | 12 +--
 .../cql3/statements/AlterTableStatement.java    | 31 ++++---
 .../cql3/statements/CreateIndexStatement.java   | 10 ++-
 .../cql3/statements/DeleteStatement.java        |  9 ++-
 .../cql3/statements/ModificationStatement.java  | 21 ++---
 .../cassandra/cql3/statements/RawSelector.java  |  4 +-
 .../cql3/statements/SelectStatement.java        | 31 +++----
 .../cassandra/cql3/statements/Selectable.java   | 45 +++++++++++
 .../cassandra/cql3/statements/Selection.java    | 26 +++---
 .../cql3/statements/UpdateStatement.java        | 17 ++--
 .../cassandra/cql3/ThriftCompatibilityTest.java | 58 +++++++++++++
 16 files changed, 332 insertions(+), 125 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/11e65718/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 2fe3a39..d5f607f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.0.12:
+ * Correctly handle non-text column names in cql3 (CASSANDRA-8178)
  * Fix deletion for indexes on primary key columns (CASSANDRA-8206)
  * Add 'nodetool statusgossip' (CASSANDRA-8125)
  * Improve client notification that nodes are ready for requests (CASSANDRA-7510)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/11e65718/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
index a8a25cf..f284436 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
@@ -20,8 +20,12 @@ package org.apache.cassandra.cql3;
 import java.util.Locale;
 import java.nio.ByteBuffer;
 
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.statements.Selectable;
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
@@ -44,6 +48,12 @@ public class ColumnIdentifier implements Selectable
         this.text = type.getString(key);
     }
 
+    private ColumnIdentifier(ByteBuffer key, String text)
+    {
+        this.key = key;
+        this.text = text;
+    }
+
     @Override
     public final int hashCode()
     {
@@ -64,4 +74,60 @@ public class ColumnIdentifier implements Selectable
     {
         return text;
     }
+
+    /**
+     * Because Thrift-created tables may have a non-text comparator, we cannot determine the proper 'key' until
+     * we know the comparator. ColumnIdentifier.Raw is a placeholder that can be converted to a real ColumnIdentifier
+     * once the comparator is known with prepare(). This should only be used with identifiers that are actual
+     * column names. See CASSANDRA-8178 for more background.
+     */
+    public static class Raw implements Selectable.Raw
+    {
+        private final String rawText;
+        private final String text;
+
+        public Raw(String rawText, boolean keepCase)
+        {
+            this.rawText = rawText;
+            this.text =  keepCase ? rawText : rawText.toLowerCase();
+        }
+
+        public ColumnIdentifier prepare(CFMetaData cfm)
+        {
+            if (cfm.getIsDense() || cfm.comparator instanceof CompositeType || cfm.comparator instanceof UTF8Type)
+                return new ColumnIdentifier(text, true);
+
+            // We have a Thrift-created table with a non-text comparator.  We need to parse column names with the comparator
+            // to get the correct ByteBuffer representation.  However, this doesn't apply to key aliases, so we need to
+            // make a special check for those and treat them normally.  See CASSANDRA-8178.
+            ByteBuffer bufferName = ByteBufferUtil.bytes(text);
+            for (ColumnDefinition def : cfm.partitionKeyColumns())
+            {
+                if (def.name.equals(bufferName))
+                    return new ColumnIdentifier(text, true);
+            }
+            return new ColumnIdentifier(cfm.comparator.fromString(rawText), text);
+        }
+
+        @Override
+        public final int hashCode()
+        {
+            return text.hashCode();
+        }
+
+        @Override
+        public final boolean equals(Object o)
+        {
+            if(!(o instanceof ColumnIdentifier.Raw))
+                return false;
+            ColumnIdentifier.Raw that = (ColumnIdentifier.Raw)o;
+            return text.equals(that.text);
+        }
+
+        @Override
+        public String toString()
+        {
+            return text;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/11e65718/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 06dbc3a..43b1c01 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -142,9 +142,9 @@ options {
         return res;
     }
 
-    public void addRawUpdate(List<Pair<ColumnIdentifier, Operation.RawUpdate>> operations, ColumnIdentifier key, Operation.RawUpdate update)
+    public void addRawUpdate(List<Pair<ColumnIdentifier.Raw, Operation.RawUpdate>> operations, ColumnIdentifier.Raw key, Operation.RawUpdate update)
     {
-        for (Pair<ColumnIdentifier, Operation.RawUpdate> p : operations)
+        for (Pair<ColumnIdentifier.Raw, Operation.RawUpdate> p : operations)
         {
             if (p.left.equals(key) && !p.right.isCompatibleWith(update))
                 addRecognitionError("Multiple incompatible setting of column " + key);
@@ -250,11 +250,11 @@ selectStatement returns [SelectStatement.RawStatement expr]
         boolean isCount = false;
         ColumnIdentifier countAlias = null;
         Term.Raw limit = null;
-        Map<ColumnIdentifier, Boolean> orderings = new LinkedHashMap<ColumnIdentifier, Boolean>();
+        Map<ColumnIdentifier.Raw, Boolean> orderings = new LinkedHashMap<ColumnIdentifier.Raw, Boolean>();
         boolean allowFiltering = false;
     }
     : K_SELECT ( ( K_DISTINCT { isDistinct = true; } )? sclause=selectClause
-               | (K_COUNT '(' sclause=selectCountClause ')' { isCount = true; } (K_AS c=cident { countAlias = c; })?) )
+               | (K_COUNT '(' sclause=selectCountClause ')' { isCount = true; } (K_AS c=ident { countAlias = c; })?) )
       K_FROM cf=columnFamilyName
       ( K_WHERE wclause=whereClause )?
       ( K_ORDER K_BY orderByClause[orderings] ( ',' orderByClause[orderings] )* )?
@@ -277,19 +277,19 @@ selectClause returns [List<RawSelector> expr]
 
 selector returns [RawSelector s]
     @init{ ColumnIdentifier alias = null; }
-    : us=unaliasedSelector (K_AS c=cident { alias = c; })? { $s = new RawSelector(us, alias); }
+    : us=unaliasedSelector (K_AS c=ident { alias = c; })? { $s = new RawSelector(us, alias); }
     ;
 
-unaliasedSelector returns [Selectable s]
+unaliasedSelector returns [Selectable.Raw s]
     : c=cident                                  { $s = c; }
-    | K_WRITETIME '(' c=cident ')'              { $s = new Selectable.WritetimeOrTTL(c, true); }
-    | K_TTL       '(' c=cident ')'              { $s = new Selectable.WritetimeOrTTL(c, false); }
-    | f=functionName args=selectionFunctionArgs { $s = new Selectable.WithFunction(f, args); }
+    | K_WRITETIME '(' c=cident ')'              { $s = new Selectable.WritetimeOrTTL.Raw(c, true); }
+    | K_TTL       '(' c=cident ')'              { $s = new Selectable.WritetimeOrTTL.Raw(c, false); }
+    | f=functionName args=selectionFunctionArgs { $s = new Selectable.WithFunction.Raw(f, args); }
     ;
 
-selectionFunctionArgs returns [List<Selectable> a]
+selectionFunctionArgs returns [List<Selectable.Raw> a]
     : '(' ')' { $a = Collections.emptyList(); }
-    | '(' s1=unaliasedSelector { List<Selectable> args = new ArrayList<Selectable>(); args.add(s1); }
+    | '(' s1=unaliasedSelector { List<Selectable.Raw> args = new ArrayList<Selectable.Raw>(); args.add(s1); }
           ( ',' sn=unaliasedSelector { args.add(sn); } )*
        ')' { $a = args; }
     ;
@@ -304,9 +304,9 @@ whereClause returns [List<Relation> clause]
     : relation[$clause] (K_AND relation[$clause])*
     ;
 
-orderByClause[Map<ColumnIdentifier, Boolean> orderings]
+orderByClause[Map<ColumnIdentifier.Raw, Boolean> orderings]
     @init{
-        ColumnIdentifier orderBy = null;
+        ColumnIdentifier.Raw orderBy = null;
         boolean reversed = false;
     }
     : c=cident { orderBy = c; } (K_ASC | K_DESC { reversed = true; })? { orderings.put(c, reversed); }
@@ -321,7 +321,7 @@ orderByClause[Map<ColumnIdentifier, Boolean> orderings]
 insertStatement returns [UpdateStatement.ParsedInsert expr]
     @init {
         Attributes.Raw attrs = new Attributes.Raw();
-        List<ColumnIdentifier> columnNames  = new ArrayList<ColumnIdentifier>();
+        List<ColumnIdentifier.Raw> columnNames  = new ArrayList<ColumnIdentifier.Raw>();
         List<Term.Raw> values = new ArrayList<Term.Raw>();
         boolean ifNotExists = false;
     }
@@ -359,7 +359,7 @@ usingClauseObjective[Attributes.Raw attrs]
 updateStatement returns [UpdateStatement.ParsedUpdate expr]
     @init {
         Attributes.Raw attrs = new Attributes.Raw();
-        List<Pair<ColumnIdentifier, Operation.RawUpdate>> operations = new ArrayList<Pair<ColumnIdentifier, Operation.RawUpdate>>();
+        List<Pair<ColumnIdentifier.Raw, Operation.RawUpdate>> operations = new ArrayList<Pair<ColumnIdentifier.Raw, Operation.RawUpdate>>();
     }
     : K_UPDATE cf=columnFamilyName
       ( usingClause[attrs] )?
@@ -371,12 +371,12 @@ updateStatement returns [UpdateStatement.ParsedUpdate expr]
                                                   attrs,
                                                   operations,
                                                   wclause,
-                                                  conditions == null ? Collections.<Pair<ColumnIdentifier, ColumnCondition.Raw>>emptyList() : conditions);
+                                                  conditions == null ? Collections.<Pair<ColumnIdentifier.Raw, ColumnCondition.Raw>>emptyList() : conditions);
      }
     ;
 
-updateConditions returns [List<Pair<ColumnIdentifier, ColumnCondition.Raw>> conditions]
-    @init { conditions = new ArrayList<Pair<ColumnIdentifier, ColumnCondition.Raw>>(); }
+updateConditions returns [List<Pair<ColumnIdentifier.Raw, ColumnCondition.Raw>> conditions]
+    @init { conditions = new ArrayList<Pair<ColumnIdentifier.Raw, ColumnCondition.Raw>>(); }
     : columnCondition[conditions] ( K_AND columnCondition[conditions] )*
     ;
 
@@ -404,7 +404,7 @@ deleteStatement returns [DeleteStatement.Parsed expr]
                                             attrs,
                                             columnDeletions,
                                             wclause,
-                                            conditions == null ? Collections.<Pair<ColumnIdentifier, ColumnCondition.Raw>>emptyList() : conditions,
+                                            conditions == null ? Collections.<Pair<ColumnIdentifier.Raw, ColumnCondition.Raw>>emptyList() : conditions,
                                             ifExists);
       }
     ;
@@ -502,14 +502,14 @@ cfamDefinition[CreateTableStatement.RawStatement expr]
     ;
 
 cfamColumns[CreateTableStatement.RawStatement expr]
-    : k=cident v=comparatorType { boolean isStatic=false; } (K_STATIC {isStatic = true;})? { $expr.addDefinition(k, v, isStatic); }
+    : k=ident 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); } )* ')'
+    | K_PRIMARY K_KEY '(' pkDef[expr] (',' c=ident { $expr.addColumnAlias(c); } )* ')'
     ;
 
 pkDef[CreateTableStatement.RawStatement expr]
-    : k=cident { $expr.addKeyAliases(Collections.singletonList(k)); }
-    | '(' { List<ColumnIdentifier> l = new ArrayList<ColumnIdentifier>(); } k1=cident { l.add(k1); } ( ',' kn=cident { l.add(kn); } )* ')' { $expr.addKeyAliases(l); }
+    : k=ident { $expr.addKeyAliases(Collections.singletonList(k)); }
+    | '(' { List<ColumnIdentifier> l = new ArrayList<ColumnIdentifier>(); } k1=ident { l.add(k1); } ( ',' kn=ident { l.add(kn); } )* ')' { $expr.addKeyAliases(l); }
     ;
 
 cfamProperty[CreateTableStatement.RawStatement expr]
@@ -520,7 +520,7 @@ cfamProperty[CreateTableStatement.RawStatement expr]
 
 cfamOrdering[CreateTableStatement.RawStatement expr]
     @init{ boolean reversed=false; }
-    : k=cident (K_ASC | K_DESC { reversed=true;} ) { $expr.setOrdering(k, reversed); }
+    : k=ident (K_ASC | K_DESC { reversed=true;} ) { $expr.setOrdering(k, reversed); }
     ;
 
 /**
@@ -576,7 +576,7 @@ alterTableStatement returns [AlterTableStatement expr]
     @init {
         AlterTableStatement.Type type = null;
         CFPropDefs props = new CFPropDefs();
-        Map<ColumnIdentifier, ColumnIdentifier> renames = new HashMap<ColumnIdentifier, ColumnIdentifier>();
+        Map<ColumnIdentifier.Raw, ColumnIdentifier.Raw> renames = new HashMap<ColumnIdentifier.Raw, ColumnIdentifier.Raw>();
         boolean isStatic = false;
     }
     : K_ALTER K_COLUMNFAMILY cf=columnFamilyName
@@ -740,8 +740,17 @@ userOption[UserOptions opts]
 
 /** DEFINITIONS **/
 
-// Column Identifiers
-cident returns [ColumnIdentifier id]
+// Column Identifiers.  These need to be treated differently from other
+// identifiers because the underlying comparator is not necessarily text. See
+// CASSANDRA-8178 for details.
+cident returns [ColumnIdentifier.Raw id]
+    : t=IDENT              { $id = new ColumnIdentifier.Raw($t.text, false); }
+    | t=QUOTED_NAME        { $id = new ColumnIdentifier.Raw($t.text, true); }
+    | k=unreserved_keyword { $id = new ColumnIdentifier.Raw(k, false); }
+    ;
+
+// Identifiers that do not refer to columns.
+ident returns [ColumnIdentifier id]
     : t=IDENT              { $id = new ColumnIdentifier($t.text, false); }
     | t=QUOTED_NAME        { $id = new ColumnIdentifier($t.text, true); }
     | k=unreserved_keyword { $id = new ColumnIdentifier(k, false); }
@@ -803,14 +812,14 @@ value returns [Term.Raw value]
     : c=constant           { $value = c; }
     | l=collection_literal { $value = l; }
     | K_NULL               { $value = Constants.NULL_LITERAL; }
-    | ':' id=cident        { $value = newBindVariables(id); }
+    | ':' id=ident         { $value = newBindVariables(id); }
     | QMARK                { $value = newBindVariables(null); }
     ;
 
 intValue returns [Term.Raw value]
     :
     | t=INTEGER     { $value = Constants.Literal.integer($t.text); }
-    | ':' id=cident { $value = newBindVariables(id); }
+    | ':' id=ident  { $value = newBindVariables(id); }
     | QMARK         { $value = newBindVariables(null); }
     ;
 
@@ -833,7 +842,7 @@ term returns [Term.Raw term]
     | '(' c=comparatorType ')' t=term  { $term = new TypeCast(c, t); }
     ;
 
-columnOperation[List<Pair<ColumnIdentifier, Operation.RawUpdate>> operations]
+columnOperation[List<Pair<ColumnIdentifier.Raw, Operation.RawUpdate>> operations]
     : key=cident '=' t=term ('+' c=cident )?
       {
           if (c == null)
@@ -867,7 +876,7 @@ columnOperation[List<Pair<ColumnIdentifier, Operation.RawUpdate>> operations]
       }
     ;
 
-columnCondition[List<Pair<ColumnIdentifier, ColumnCondition.Raw>> conditions]
+columnCondition[List<Pair<ColumnIdentifier.Raw, ColumnCondition.Raw>> conditions]
     // Note: we'll reject duplicates later
     : key=cident '=' t=term { conditions.add(Pair.create(key, ColumnCondition.Raw.simpleEqual(t))); }
     | key=cident '[' element=term ']' '=' t=term { conditions.add(Pair.create(key, ColumnCondition.Raw.collectionEqual(t, element))); } 
@@ -878,7 +887,7 @@ properties[PropertyDefinitions props]
     ;
 
 property[PropertyDefinitions props]
-    : k=cident '=' (simple=propertyValue { try { $props.addProperty(k.toString(), simple); } catch (SyntaxException e) { addRecognitionError(e.getMessage()); } }
+    : k=ident '=' (simple=propertyValue { try { $props.addProperty(k.toString(), simple); } catch (SyntaxException e) { addRecognitionError(e.getMessage()); } }
                    |   map=map_literal   { try { $props.addProperty(k.toString(), convertPropertyMap(map)); } catch (SyntaxException e) { addRecognitionError(e.getMessage()); } })
     ;
 
@@ -899,7 +908,7 @@ relation[List<Relation> clauses]
     : name=cident type=relationType t=term { $clauses.add(new SingleColumnRelation(name, type, t)); }
     | K_TOKEN l=tupleOfIdentifiers type=relationType t=term
         {
-            for (ColumnIdentifier id : l)
+            for (ColumnIdentifier.Raw id : l)
                 $clauses.add(new SingleColumnRelation(id, type, t, true));
         }
     | name=cident K_IN marker=inMarker
@@ -931,11 +940,11 @@ relation[List<Relation> clauses]
 
 inMarker returns [AbstractMarker.INRaw marker]
     : QMARK { $marker = newINBindVariables(null); }
-    | ':' name=cident { $marker = newINBindVariables(name); }
+    | ':' name=ident { $marker = newINBindVariables(name); }
     ;
 
-tupleOfIdentifiers returns [List<ColumnIdentifier> ids]
-    @init { $ids = new ArrayList<ColumnIdentifier>(); }
+tupleOfIdentifiers returns [List<ColumnIdentifier.Raw> ids]
+    @init { $ids = new ArrayList<ColumnIdentifier.Raw>(); }
     : '(' n1=cident { $ids.add(n1); } (',' ni=cident { $ids.add(ni); })* ')'
     ;
 
@@ -956,7 +965,7 @@ tupleOfTupleLiterals returns [List<Tuples.Literal> literals]
 
 markerForTuple returns [Tuples.Raw marker]
     : QMARK { $marker = newTupleBindVariables(null); }
-    | ':' name=cident { $marker = newTupleBindVariables(name); }
+    | ':' name=ident { $marker = newTupleBindVariables(name); }
     ;
 
 tupleOfMarkersForTuples returns [List<Tuples.Raw> markers]
@@ -966,7 +975,7 @@ tupleOfMarkersForTuples returns [List<Tuples.Raw> markers]
 
 inMarkerForTuple returns [Tuples.INRaw marker]
     : QMARK { $marker = newTupleINBindVariables(null); }
-    | ':' name=cident { $marker = newTupleINBindVariables(name); }
+    | ':' name=ident { $marker = newTupleINBindVariables(name); }
     ;
 
 comparatorType returns [CQL3Type t]

http://git-wip-us.apache.org/repos/asf/cassandra/blob/11e65718/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java b/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
index fda60df..d79bba5 100644
--- a/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
+++ b/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
@@ -29,7 +29,7 @@ import java.util.List;
  */
 public class MultiColumnRelation extends Relation
 {
-    private final List<ColumnIdentifier> entities;
+    private final List<ColumnIdentifier.Raw> entities;
 
     /** A Tuples.Literal or Tuples.Raw marker */
     private final Term.MultiColumnRaw valuesOrMarker;
@@ -39,7 +39,7 @@ public class MultiColumnRelation extends Relation
 
     private final Tuples.INRaw inMarker;
 
-    private MultiColumnRelation(List<ColumnIdentifier> entities, Type relationType, Term.MultiColumnRaw valuesOrMarker, List<? extends Term.MultiColumnRaw> inValues, Tuples.INRaw inMarker)
+    private MultiColumnRelation(List<ColumnIdentifier.Raw> entities, Type relationType, Term.MultiColumnRaw valuesOrMarker, List<? extends Term.MultiColumnRaw> inValues, Tuples.INRaw inMarker)
     {
         this.entities = entities;
         this.relationType = relationType;
@@ -56,7 +56,7 @@ public class MultiColumnRelation extends Relation
      * @param relationType the relation operator
      * @param valuesOrMarker a Tuples.Literal instance or a Tuples.Raw marker
      */
-    public static MultiColumnRelation createNonInRelation(List<ColumnIdentifier> entities, Type relationType, Term.MultiColumnRaw valuesOrMarker)
+    public static MultiColumnRelation createNonInRelation(List<ColumnIdentifier.Raw> entities, Type relationType, Term.MultiColumnRaw valuesOrMarker)
     {
         assert relationType != Relation.Type.IN;
         return new MultiColumnRelation(entities, relationType, valuesOrMarker, null, null);
@@ -68,7 +68,7 @@ public class MultiColumnRelation extends Relation
      * @param entities the columns on the LHS of the relation
      * @param inValues a list of Tuples.Literal instances or a Tuples.Raw markers
      */
-    public static MultiColumnRelation createInRelation(List<ColumnIdentifier> entities, List<? extends Term.MultiColumnRaw> inValues)
+    public static MultiColumnRelation createInRelation(List<ColumnIdentifier.Raw> entities, List<? extends Term.MultiColumnRaw> inValues)
     {
         return new MultiColumnRelation(entities, Relation.Type.IN, null, inValues, null);
     }
@@ -79,12 +79,12 @@ public class MultiColumnRelation extends Relation
      * @param entities the columns on the LHS of the relation
      * @param inMarker a single IN marker
      */
-    public static MultiColumnRelation createSingleMarkerInRelation(List<ColumnIdentifier> entities, Tuples.INRaw inMarker)
+    public static MultiColumnRelation createSingleMarkerInRelation(List<ColumnIdentifier.Raw> entities, Tuples.INRaw inMarker)
     {
         return new MultiColumnRelation(entities, Relation.Type.IN, null, null, inMarker);
     }
 
-    public List<ColumnIdentifier> getEntities()
+    public List<ColumnIdentifier.Raw> getEntities()
     {
         return entities;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/11e65718/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 6bf46b5..1792a2d 100644
--- a/src/java/org/apache/cassandra/cql3/Operation.java
+++ b/src/java/org/apache/cassandra/cql3/Operation.java
@@ -149,7 +149,7 @@ public abstract class Operation
         /**
          * The name of the column affected by this delete operation.
          */
-        public ColumnIdentifier affectedColumn();
+        public ColumnIdentifier.Raw affectedColumn();
 
         /**
          * This method validates the operation (i.e. validate it is well typed)
@@ -162,7 +162,7 @@ public abstract class Operation
          * @param receiver the "column" this operation applies to.
          * @return the prepared delete operation.
          */
-        public Operation prepare(ColumnSpecification receiver) throws InvalidRequestException;
+        public Operation prepare(ColumnSpecification receiver, CFMetaData cfm) throws InvalidRequestException;
     }
 
     public static class SetValue implements RawUpdate
@@ -372,57 +372,58 @@ public abstract class Operation
 
     public static class ColumnDeletion implements RawDeletion
     {
-        private final ColumnIdentifier id;
+        private final ColumnIdentifier.Raw id;
 
-        public ColumnDeletion(ColumnIdentifier id)
+        public ColumnDeletion(ColumnIdentifier.Raw id)
         {
             this.id = id;
         }
 
-        public ColumnIdentifier affectedColumn()
+        public ColumnIdentifier.Raw affectedColumn()
         {
             return id;
         }
 
-        public Operation prepare(ColumnSpecification receiver) throws InvalidRequestException
+        public Operation prepare(ColumnSpecification receiver, CFMetaData cfm) throws InvalidRequestException
         {
             // No validation, deleting a column is always "well typed"
-            return new Constants.Deleter(id, receiver.type instanceof CollectionType);
+            return new Constants.Deleter(id.prepare(cfm), receiver.type instanceof CollectionType);
         }
     }
 
     public static class ElementDeletion implements RawDeletion
     {
-        private final ColumnIdentifier id;
+        private final ColumnIdentifier.Raw id;
         private final Term.Raw element;
 
-        public ElementDeletion(ColumnIdentifier id, Term.Raw element)
+        public ElementDeletion(ColumnIdentifier.Raw id, Term.Raw element)
         {
             this.id = id;
             this.element = element;
         }
 
-        public ColumnIdentifier affectedColumn()
+        public ColumnIdentifier.Raw affectedColumn()
         {
             return id;
         }
 
-        public Operation prepare(ColumnSpecification receiver) throws InvalidRequestException
+        public Operation prepare(ColumnSpecification receiver, CFMetaData cfm) throws InvalidRequestException
         {
             if (!(receiver.type instanceof CollectionType))
                 throw new InvalidRequestException(String.format("Invalid deletion operation for non collection column %s", receiver));
 
+            ColumnIdentifier preparedId = id.prepare(cfm);
             switch (((CollectionType)receiver.type).kind)
             {
                 case LIST:
                     Term idx = element.prepare(Lists.indexSpecOf(receiver));
-                    return new Lists.DiscarderByIndex(id, idx);
+                    return new Lists.DiscarderByIndex(preparedId, idx);
                 case SET:
                     Term elt = element.prepare(Sets.valueSpecOf(receiver));
-                    return new Sets.Discarder(id, elt);
+                    return new Sets.Discarder(preparedId, elt);
                 case MAP:
                     Term key = element.prepare(Maps.keySpecOf(receiver));
-                    return new Maps.DiscarderByKey(id, key);
+                    return new Maps.DiscarderByKey(preparedId, key);
             }
             throw new AssertionError();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/11e65718/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
index ee95da0..d63493e 100644
--- a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
+++ b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
@@ -26,12 +26,12 @@ import java.util.List;
  */
 public class SingleColumnRelation extends Relation
 {
-    private final ColumnIdentifier entity;
+    private final ColumnIdentifier.Raw entity;
     private final Term.Raw value;
     private final List<Term.Raw> inValues;
     public final boolean onToken;
 
-    private SingleColumnRelation(ColumnIdentifier entity, Type type, Term.Raw value, List<Term.Raw> inValues, boolean onToken)
+    private SingleColumnRelation(ColumnIdentifier.Raw entity, Type type, Term.Raw value, List<Term.Raw> inValues, boolean onToken)
     {
         this.entity = entity;
         this.relationType = type;
@@ -47,22 +47,22 @@ public class SingleColumnRelation extends Relation
      * @param type the type that describes how this entity relates to the value.
      * @param value the value being compared.
      */
-    public SingleColumnRelation(ColumnIdentifier entity, Type type, Term.Raw value)
+    public SingleColumnRelation(ColumnIdentifier.Raw entity, Type type, Term.Raw value)
     {
         this(entity, type, value, null, false);
     }
 
-    public SingleColumnRelation(ColumnIdentifier entity, Type type, Term.Raw value, boolean onToken)
+    public SingleColumnRelation(ColumnIdentifier.Raw entity, Type type, Term.Raw value, boolean onToken)
     {
         this(entity, type, value, null, onToken);
     }
 
-    public static SingleColumnRelation createInRelation(ColumnIdentifier entity, List<Term.Raw> inValues)
+    public static SingleColumnRelation createInRelation(ColumnIdentifier.Raw entity, List<Term.Raw> inValues)
     {
         return new SingleColumnRelation(entity, Type.IN, null, inValues, false);
     }
 
-    public ColumnIdentifier getEntity()
+    public ColumnIdentifier.Raw getEntity()
     {
         return entity;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/11e65718/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 698c8b8..32f949f 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
@@ -44,22 +44,22 @@ public class AlterTableStatement extends SchemaAlteringStatement
 
     public final Type oType;
     public final CQL3Type validator;
-    public final ColumnIdentifier columnName;
+    public final ColumnIdentifier.Raw rawColumnName;
     private final CFPropDefs cfProps;
-    private final Map<ColumnIdentifier, ColumnIdentifier> renames;
+    private final Map<ColumnIdentifier.Raw, ColumnIdentifier.Raw> renames;
     private final boolean isStatic; // Only for ALTER ADD
 
     public AlterTableStatement(CFName name,
                                Type type,
-                               ColumnIdentifier columnName,
+                               ColumnIdentifier.Raw columnName,
                                CQL3Type validator,
                                CFPropDefs cfProps,
-                               Map<ColumnIdentifier, ColumnIdentifier> renames,
+                               Map<ColumnIdentifier.Raw, ColumnIdentifier.Raw> renames,
                                boolean isStatic)
     {
         super(name);
         this.oType = type;
-        this.columnName = columnName;
+        this.rawColumnName = columnName;
         this.validator = validator; // used only for ADD/ALTER commands
         this.cfProps = cfProps;
         this.renames = renames;
@@ -82,10 +82,19 @@ public class AlterTableStatement extends SchemaAlteringStatement
         CFMetaData cfm = meta.clone();
 
         CFDefinition cfDef = meta.getCfDef();
-        CFDefinition.Name name = columnName == null ? null : cfDef.get(columnName);
+
+        ColumnIdentifier columnName = null;
+        CFDefinition.Name name = null;
+        if (rawColumnName != null)
+        {
+            columnName = rawColumnName.prepare(cfm);
+            name = cfDef.get(columnName);
+        }
+
         switch (oType)
         {
             case ADD:
+                assert columnName != null;
                 if (cfDef.isCompact)
                     throw new InvalidRequestException("Cannot add new column to a COMPACT STORAGE table");
 
@@ -152,6 +161,7 @@ public class AlterTableStatement extends SchemaAlteringStatement
                 break;
 
             case ALTER:
+                assert columnName != null;
                 if (name == null)
                     throw new InvalidRequestException(String.format("Column %s was not found in table %s", columnName, columnFamily()));
 
@@ -228,6 +238,7 @@ public class AlterTableStatement extends SchemaAlteringStatement
                 break;
 
             case DROP:
+                assert columnName != null;
                 if (cfDef.isCompact || !cfDef.isComposite)
                     throw new InvalidRequestException("Cannot drop columns from a COMPACT STORAGE table");
                 if (name == null)
@@ -260,10 +271,10 @@ public class AlterTableStatement extends SchemaAlteringStatement
                 cfProps.applyToCFMetadata(cfm);
                 break;
             case RENAME:
-                for (Map.Entry<ColumnIdentifier, ColumnIdentifier> entry : renames.entrySet())
+                for (Map.Entry<ColumnIdentifier.Raw, ColumnIdentifier.Raw> entry : renames.entrySet())
                 {
-                    ColumnIdentifier from = entry.getKey();
-                    ColumnIdentifier to = entry.getValue();
+                    ColumnIdentifier from = entry.getKey().prepare(cfm);
+                    ColumnIdentifier to = entry.getValue().prepare(cfm);
                     cfm.renameColumn(from.key, from.toString(), to.key, to.toString());
                 }
                 break;
@@ -278,7 +289,7 @@ public class AlterTableStatement extends SchemaAlteringStatement
         return String.format("AlterTableStatement(name=%s, type=%s, column=%s, validator=%s)",
                              cfName,
                              oType,
-                             columnName,
+                             rawColumnName,
                              validator);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/11e65718/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 8b40978..e173e8c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
@@ -40,19 +40,19 @@ public class CreateIndexStatement extends SchemaAlteringStatement
     private static final Logger logger = LoggerFactory.getLogger(CreateIndexStatement.class);
 
     private final String indexName;
-    private final ColumnIdentifier columnName;
+    private final ColumnIdentifier.Raw rawColumnName;
     private final IndexPropDefs properties;
     private final boolean ifNotExists;
 
     public CreateIndexStatement(CFName name,
                                 String indexName,
-                                ColumnIdentifier columnName,
+                                ColumnIdentifier.Raw rawColumnName,
                                 IndexPropDefs properties,
                                 boolean ifNotExists)
     {
         super(name);
         this.indexName = indexName;
-        this.columnName = columnName;
+        this.rawColumnName = rawColumnName;
         this.properties = properties;
         this.ifNotExists = ifNotExists;
     }
@@ -68,6 +68,7 @@ public class CreateIndexStatement extends SchemaAlteringStatement
         if (cfm.getDefaultValidator().isCommutative())
             throw new InvalidRequestException("Secondary indexes are not supported on counter tables");
 
+        ColumnIdentifier columnName = rawColumnName.prepare(cfm);
         ColumnDefinition cd = cfm.getColumnDefinition(columnName.key);
 
         if (cd == null)
@@ -105,8 +106,9 @@ public class CreateIndexStatement extends SchemaAlteringStatement
 
     public boolean announceMigration() throws RequestValidationException
     {
-        logger.debug("Updating column {} definition for index {}", columnName, indexName);
         CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), columnFamily()).clone();
+        ColumnIdentifier columnName = rawColumnName.prepare(cfm);
+        logger.debug("Updating column {} definition for index {}", columnName, indexName);
         ColumnDefinition cd = cfm.getColumnDefinition(columnName.key);
 
         if (cd.getIndexType() != null && ifNotExists)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/11e65718/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 6c1c6ed..e3aa8ce 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
@@ -120,7 +120,7 @@ public class DeleteStatement extends ModificationStatement
                       Attributes.Raw attrs,
                       List<Operation.RawDeletion> deletions,
                       List<Relation> whereClause,
-                      List<Pair<ColumnIdentifier, ColumnCondition.Raw>> conditions,
+                      List<Pair<ColumnIdentifier.Raw, ColumnCondition.Raw>> conditions,
                       boolean ifExists)
         {
             super(name, attrs, conditions, false, ifExists);
@@ -134,16 +134,17 @@ public class DeleteStatement extends ModificationStatement
 
             for (Operation.RawDeletion deletion : deletions)
             {
-                CFDefinition.Name name = cfDef.get(deletion.affectedColumn());
+                ColumnIdentifier id = deletion.affectedColumn().prepare(cfDef.cfm);
+                CFDefinition.Name name = cfDef.get(id);
                 if (name == null)
-                    throw new InvalidRequestException(String.format("Unknown identifier %s", deletion.affectedColumn()));
+                    throw new InvalidRequestException(String.format("Unknown identifier %s", id));
 
                 // 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.isPrimaryKeyColumn())
                     throw new InvalidRequestException(String.format("Invalid identifier %s for deletion (should not be a PRIMARY KEY part)", name));
 
-                Operation op = deletion.prepare(name);
+                Operation op = deletion.prepare(name, cfDef.cfm);
                 op.collectMarkerSpecification(boundNames);
                 stmt.addOperation(op);
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/11e65718/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 adb0084..c098c92 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -22,6 +22,7 @@ import java.util.*;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Iterables;
+import org.apache.cassandra.db.marshal.AbstractType;
 import org.github.jamm.MemoryMeter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -271,9 +272,10 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
             if (rel.onToken)
                 throw new InvalidRequestException(String.format("The token function cannot be used in WHERE clauses for UPDATE and DELETE statements: %s", relation));
 
-            CFDefinition.Name name = cfDef.get(rel.getEntity());
+            ColumnIdentifier id = rel.getEntity().prepare(cfm);
+            CFDefinition.Name name = cfDef.get(id);
             if (name == null)
-                throw new InvalidRequestException(String.format("Unknown key identifier %s", rel.getEntity()));
+                throw new InvalidRequestException(String.format("Unknown key identifier %s", id));
 
             switch (name.kind)
             {
@@ -760,15 +762,15 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
     public static abstract class Parsed extends CFStatement
     {
         protected final Attributes.Raw attrs;
-        protected final List<Pair<ColumnIdentifier, ColumnCondition.Raw>> conditions;
+        protected final List<Pair<ColumnIdentifier.Raw, ColumnCondition.Raw>> conditions;
         private final boolean ifNotExists;
         private final boolean ifExists;
 
-        protected Parsed(CFName name, Attributes.Raw attrs, List<Pair<ColumnIdentifier, ColumnCondition.Raw>> conditions, boolean ifNotExists, boolean ifExists)
+        protected Parsed(CFName name, Attributes.Raw attrs, List<Pair<ColumnIdentifier.Raw, ColumnCondition.Raw>> conditions, boolean ifNotExists, boolean ifExists)
         {
             super(name);
             this.attrs = attrs;
-            this.conditions = conditions == null ? Collections.<Pair<ColumnIdentifier, ColumnCondition.Raw>>emptyList() : conditions;
+            this.conditions = conditions == null ? Collections.<Pair<ColumnIdentifier.Raw, ColumnCondition.Raw>>emptyList() : conditions;
             this.ifNotExists = ifNotExists;
             this.ifExists = ifExists;
         }
@@ -815,11 +817,12 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
                 }
                 else
                 {
-                    for (Pair<ColumnIdentifier, ColumnCondition.Raw> entry : conditions)
+                    for (Pair<ColumnIdentifier.Raw, ColumnCondition.Raw> entry : conditions)
                     {
-                        CFDefinition.Name name = cfDef.get(entry.left);
+                        ColumnIdentifier id = entry.left.prepare(cfDef.cfm);
+                        CFDefinition.Name name = cfDef.get(id);
                         if (name == null)
-                            throw new InvalidRequestException(String.format("Unknown identifier %s", entry.left));
+                            throw new InvalidRequestException(String.format("Unknown identifier %s", id));
 
                         ColumnCondition condition = entry.right.prepare(name);
                         condition.collectMarkerSpecification(boundNames);
@@ -828,7 +831,7 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
                         {
                             case KEY_ALIAS:
                             case COLUMN_ALIAS:
-                                throw new InvalidRequestException(String.format("PRIMARY KEY column '%s' cannot have IF conditions", entry.left));
+                                throw new InvalidRequestException(String.format("PRIMARY KEY column '%s' cannot have IF conditions", id));
                             case VALUE_ALIAS:
                             case COLUMN_METADATA:
                             case STATIC:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/11e65718/src/java/org/apache/cassandra/cql3/statements/RawSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/RawSelector.java b/src/java/org/apache/cassandra/cql3/statements/RawSelector.java
index 01fe9e4..0194239 100644
--- a/src/java/org/apache/cassandra/cql3/statements/RawSelector.java
+++ b/src/java/org/apache/cassandra/cql3/statements/RawSelector.java
@@ -22,10 +22,10 @@ import org.apache.cassandra.cql3.ColumnIdentifier;
 
 public class RawSelector
 {
-    public final Selectable selectable;
+    public final Selectable.Raw selectable;
     public final ColumnIdentifier alias;
 
-    public RawSelector(Selectable selectable, ColumnIdentifier alias)
+    public RawSelector(Selectable.Raw selectable, ColumnIdentifier alias)
     {
         this.selectable = selectable;
         this.alias = alias;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/11e65718/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 aadd0bd..77d94e3 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -99,7 +99,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
     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 Parameters defaultParameters = new Parameters(Collections.<ColumnIdentifier.Raw, Boolean>emptyMap(), false, false, null, false);
 
     private static final Predicate<CFDefinition.Name> isStaticFilter = new Predicate<CFDefinition.Name>()
     {
@@ -1249,7 +1249,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         // 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());
+            CFDefinition.Name ordering = cfDef.get(parameters.orderings.keySet().iterator().next().prepare(cfDef.cfm));
             Collections.sort(cqlRows.rows, new SingleColumnComparator(orderingIndexes.get(ordering), ordering.type));
             return;
         }
@@ -1261,9 +1261,9 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         int[] positions = new int[parameters.orderings.size()];
 
         int idx = 0;
-        for (ColumnIdentifier identifier : parameters.orderings.keySet())
+        for (ColumnIdentifier.Raw identifier : parameters.orderings.keySet())
         {
-            CFDefinition.Name orderingColumn = cfDef.get(identifier);
+            CFDefinition.Name orderingColumn = cfDef.get(identifier.prepare(cfDef.cfm));
             types.add(orderingColumn.type);
             positions[idx++] = orderingIndexes.get(orderingColumn);
         }
@@ -1412,8 +1412,9 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 {
                     MultiColumnRelation rel = (MultiColumnRelation) relation;
                     List<CFDefinition.Name> names = new ArrayList<>(rel.getEntities().size());
-                    for (ColumnIdentifier entity : rel.getEntities())
+                    for (ColumnIdentifier.Raw rawEntity : rel.getEntities())
                     {
+                        ColumnIdentifier entity = rawEntity.prepare(cfm);
                         boolean[] queriable = processRelationEntity(stmt, relation, entity, cfDef);
                         hasQueriableIndex |= queriable[0];
                         hasQueriableClusteringColumnIndex |= queriable[1];
@@ -1426,10 +1427,11 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 else
                 {
                     SingleColumnRelation rel = (SingleColumnRelation) relation;
-                    boolean[] queriable = processRelationEntity(stmt, relation, rel.getEntity(), cfDef);
+                    ColumnIdentifier entity = rel.getEntity().prepare(cfm);
+                    boolean[] queriable = processRelationEntity(stmt, relation, entity, cfDef);
                     hasQueriableIndex |= queriable[0];
                     hasQueriableClusteringColumnIndex |= queriable[1];
-                    Name name = cfDef.get(rel.getEntity());
+                    Name name = cfDef.get(entity);
                     hasSingleColumnRelations |= Kind.COLUMN_ALIAS.equals(name.kind);
                     updateRestrictionsForRelation(stmt, name, rel, boundNames);
                 }
@@ -1832,7 +1834,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             for (Relation relation : whereClause)
             {
                 SingleColumnRelation singleColumnRelation = (SingleColumnRelation) relation;
-                if (singleColumnRelation.onToken && !cfDef.get(singleColumnRelation.getEntity()).equals(iter.next()))
+                if (singleColumnRelation.onToken && !cfDef.get(singleColumnRelation.getEntity().prepare(cfDef.cfm)).equals(iter.next()))
                     throw new InvalidRequestException(String.format("The token function arguments must be in the partition key order: %s",
                                                                     Joiner.on(',').join(cfDef.partitionKeys())));
             }
@@ -1935,8 +1937,9 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             if (stmt.keyIsInRelation)
             {
                 stmt.orderingIndexes = new HashMap<CFDefinition.Name, Integer>();
-                for (ColumnIdentifier column : stmt.parameters.orderings.keySet())
+                for (ColumnIdentifier.Raw rawColumn : stmt.parameters.orderings.keySet())
                 {
+                    ColumnIdentifier column = rawColumn.prepare(cfDef.cfm);
                     final CFDefinition.Name name = cfDef.get(column);
                     if (name == null)
                         handleUnrecognizedOrderingColumn(column);
@@ -1977,9 +1980,9 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         {
             Boolean[] reversedMap = new Boolean[cfDef.clusteringColumnsCount()];
             int i = 0;
-            for (Map.Entry<ColumnIdentifier, Boolean> entry : stmt.parameters.orderings.entrySet())
+            for (Map.Entry<ColumnIdentifier.Raw, Boolean> entry : stmt.parameters.orderings.entrySet())
             {
-                ColumnIdentifier column = entry.getKey();
+                ColumnIdentifier column = entry.getKey().prepare(cfDef.cfm);
                 boolean reversed = entry.getValue();
 
                 CFDefinition.Name name = cfDef.get(column);
@@ -2056,7 +2059,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 // We only call this when sliceRestriction != null, i.e. for compact table with non composite comparator,
                 // so it can't be a MultiColumnRelation.
                 SingleColumnRelation rel = (SingleColumnRelation)r;
-                if (cfDef.get(rel.getEntity()).kind == CFDefinition.Name.Kind.COLUMN_ALIAS
+                if (cfDef.get(rel.getEntity().prepare(cfDef.cfm)).kind == CFDefinition.Name.Kind.COLUMN_ALIAS
                     && (rel.operator() == Relation.Type.GT || rel.operator() == Relation.Type.LT))
                     return rel;
             }
@@ -2096,13 +2099,13 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
 
     public static class Parameters
     {
-        private final Map<ColumnIdentifier, Boolean> orderings;
+        private final Map<ColumnIdentifier.Raw, Boolean> orderings;
         private final boolean isDistinct;
         private final boolean isCount;
         private final ColumnIdentifier countAlias;
         private final boolean allowFiltering;
 
-        public Parameters(Map<ColumnIdentifier, Boolean> orderings,
+        public Parameters(Map<ColumnIdentifier.Raw, Boolean> orderings,
                           boolean isDistinct,
                           boolean isCount,
                           ColumnIdentifier countAlias,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/11e65718/src/java/org/apache/cassandra/cql3/statements/Selectable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/Selectable.java b/src/java/org/apache/cassandra/cql3/statements/Selectable.java
index 9f25542..633bf71 100644
--- a/src/java/org/apache/cassandra/cql3/statements/Selectable.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Selectable.java
@@ -18,12 +18,20 @@
  */
 package org.apache.cassandra.cql3.statements;
 
+import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 
 public interface Selectable
 {
+
+    public static interface Raw
+    {
+        public Selectable prepare(CFMetaData cfm);
+    }
+
     public static class WritetimeOrTTL implements Selectable
     {
         public final ColumnIdentifier id;
@@ -40,6 +48,23 @@ public interface Selectable
         {
             return (isWritetime ? "writetime" : "ttl") + "(" + id + ")";
         }
+
+        public static class Raw implements Selectable.Raw
+        {
+            private final ColumnIdentifier.Raw id;
+            private final boolean isWritetime;
+
+            public Raw(ColumnIdentifier.Raw id, boolean isWritetime)
+            {
+                this.id = id;
+                this.isWritetime = isWritetime;
+            }
+
+            public WritetimeOrTTL prepare(CFMetaData cfm)
+            {
+                return new WritetimeOrTTL(id.prepare(cfm), isWritetime);
+            }
+        }
     }
 
     public static class WithFunction implements Selectable
@@ -65,5 +90,25 @@ public interface Selectable
             }
             return sb.append(")").toString();
         }
+
+        public static class Raw implements Selectable.Raw
+        {
+            private final String functionName;
+            private final List<Selectable.Raw> args;
+
+            public Raw(String functionName, List<Selectable.Raw> args)
+            {
+                this.functionName = functionName;
+                this.args = args;
+            }
+
+            public WithFunction prepare(CFMetaData cfm)
+            {
+                List<Selectable> preparedArgs = new ArrayList<>(args.size());
+                for (Selectable.Raw arg : args)
+                    preparedArgs.add(arg.prepare(cfm));
+                return new WithFunction(functionName, preparedArgs);
+            }
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/11e65718/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 18ca177..0135a76 100644
--- a/src/java/org/apache/cassandra/cql3/statements/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Selection.java
@@ -97,18 +97,24 @@ public abstract class Selection
 
     private static Selector makeSelector(CFDefinition cfDef, RawSelector raw, List<CFDefinition.Name> names, List<ColumnSpecification> metadata) throws InvalidRequestException
     {
-        if (raw.selectable instanceof ColumnIdentifier)
+        Selectable selectable = raw.selectable.prepare(cfDef.cfm);
+        return makeSelector(cfDef, selectable, raw.alias, names, metadata);
+    }
+
+    private static Selector makeSelector(CFDefinition cfDef, Selectable selectable, ColumnIdentifier alias, List<CFDefinition.Name> names, List<ColumnSpecification> metadata) throws InvalidRequestException
+    {
+        if (selectable instanceof ColumnIdentifier)
         {
-            CFDefinition.Name name = cfDef.get((ColumnIdentifier)raw.selectable);
+            CFDefinition.Name name = cfDef.get((ColumnIdentifier)selectable);
             if (name == null)
-                throw new InvalidRequestException(String.format("Undefined name %s in selection clause", raw.selectable));
+                throw new InvalidRequestException(String.format("Undefined name %s in selection clause", selectable));
             if (metadata != null)
-                metadata.add(raw.alias == null ? name : makeAliasSpec(cfDef, name.type, raw.alias));
+                metadata.add(alias == null ? name : makeAliasSpec(cfDef, name.type, alias));
             return new SimpleSelector(name.toString(), addAndGetIndex(name, names), name.type);
         }
-        else if (raw.selectable instanceof Selectable.WritetimeOrTTL)
+        else if (selectable instanceof Selectable.WritetimeOrTTL)
         {
-            Selectable.WritetimeOrTTL tot = (Selectable.WritetimeOrTTL)raw.selectable;
+            Selectable.WritetimeOrTTL tot = (Selectable.WritetimeOrTTL)selectable;
             CFDefinition.Name name = cfDef.get(tot.id);
             if (name == null)
                 throw new InvalidRequestException(String.format("Undefined name %s in selection clause", tot.id));
@@ -118,20 +124,20 @@ public abstract class Selection
                 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));
+                metadata.add(makeWritetimeOrTTLSpec(cfDef, tot, alias));
             return new WritetimeOrTTLSelector(name.toString(), addAndGetIndex(name, names), tot.isWritetime);
         }
         else
         {
-            Selectable.WithFunction withFun = (Selectable.WithFunction)raw.selectable;
+            Selectable.WithFunction withFun = (Selectable.WithFunction)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(cfDef, rawArg, null, names, null));
 
             AbstractType<?> returnType = Functions.getReturnType(withFun.functionName, cfDef.cfm.ksName, cfDef.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(cfDef, withFun, returnType, alias);
             Function fun = Functions.get(withFun.functionName, args, spec);
             if (metadata != null)
                 metadata.add(spec);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/11e65718/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 8453a76..022af26 100644
--- a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
@@ -23,6 +23,7 @@ import java.util.*;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
@@ -102,7 +103,7 @@ public class UpdateStatement extends ModificationStatement
 
     public static class ParsedInsert extends ModificationStatement.Parsed
     {
-        private final List<ColumnIdentifier> columnNames;
+        private final List<ColumnIdentifier.Raw> columnNames;
         private final List<Term.Raw> columnValues;
 
         /**
@@ -115,7 +116,7 @@ public class UpdateStatement extends ModificationStatement
          */
         public ParsedInsert(CFName name,
                             Attributes.Raw attrs,
-                            List<ColumnIdentifier> columnNames, List<Term.Raw> columnValues,
+                            List<ColumnIdentifier.Raw> columnNames, List<Term.Raw> columnValues,
                             boolean ifNotExists)
         {
             super(name, attrs, null, ifNotExists, false);
@@ -137,7 +138,7 @@ public class UpdateStatement extends ModificationStatement
 
             for (int i = 0; i < columnNames.size(); i++)
             {
-                CFDefinition.Name name = cfDef.get(columnNames.get(i));
+                CFDefinition.Name name = cfDef.get(columnNames.get(i).prepare(cfDef.cfm));
                 if (name == null)
                     throw new InvalidRequestException(String.format("Unknown identifier %s", columnNames.get(i)));
 
@@ -171,7 +172,7 @@ public class UpdateStatement extends ModificationStatement
     public static class ParsedUpdate extends ModificationStatement.Parsed
     {
         // Provided for an UPDATE
-        private final List<Pair<ColumnIdentifier, Operation.RawUpdate>> updates;
+        private final List<Pair<ColumnIdentifier.Raw, Operation.RawUpdate>> updates;
         private final List<Relation> whereClause;
 
         /**
@@ -185,9 +186,9 @@ public class UpdateStatement extends ModificationStatement
          */
         public ParsedUpdate(CFName name,
                             Attributes.Raw attrs,
-                            List<Pair<ColumnIdentifier, Operation.RawUpdate>> updates,
+                            List<Pair<ColumnIdentifier.Raw, Operation.RawUpdate>> updates,
                             List<Relation> whereClause,
-                            List<Pair<ColumnIdentifier, ColumnCondition.Raw>> conditions)
+                            List<Pair<ColumnIdentifier.Raw, ColumnCondition.Raw>> conditions)
         {
             super(name, attrs, conditions, false, false);
             this.updates = updates;
@@ -198,9 +199,9 @@ public class UpdateStatement extends ModificationStatement
         {
             UpdateStatement stmt = new UpdateStatement(ModificationStatement.StatementType.UPDATE, cfDef.cfm, attrs);
 
-            for (Pair<ColumnIdentifier, Operation.RawUpdate> entry : updates)
+            for (Pair<ColumnIdentifier.Raw, Operation.RawUpdate> entry : updates)
             {
-                CFDefinition.Name name = cfDef.get(entry.left);
+                CFDefinition.Name name = cfDef.get(entry.left.prepare(cfDef.cfm));
                 if (name == null)
                     throw new InvalidRequestException(String.format("Unknown identifier %s", entry.left));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/11e65718/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java b/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
new file mode 100644
index 0000000..86d2dac
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
@@ -0,0 +1,58 @@
+/*
+ * 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 org.apache.cassandra.utils.ByteBufferUtil;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+
+import static org.apache.cassandra.cql3.QueryProcessor.processInternal;
+import static org.junit.Assert.assertEquals;
+
+public class ThriftCompatibilityTest extends SchemaLoader
+{
+    private static UntypedResultSet execute(String query) throws Throwable
+    {
+        try
+        {
+            return processInternal(String.format(query));
+        }
+        catch (RuntimeException exc)
+        {
+            if (exc.getCause() != null)
+                throw exc.getCause();
+            throw exc;
+        }
+    }
+
+    /** Test For CASSANDRA-8178 */
+    @Test
+    public void testNonTextComparator() throws Throwable
+    {
+        // the comparator is IntegerType, and there is a column named 42 with a UTF8Type validation type
+        execute("INSERT INTO \"Keyspace1\".\"JdbcInteger\" (key, \"42\") VALUES (0x00000001, 'abc')");
+        execute("UPDATE \"Keyspace1\".\"JdbcInteger\" SET \"42\" = 'abc' WHERE key = 0x00000001");
+        execute("DELETE \"42\" FROM \"Keyspace1\".\"JdbcInteger\" WHERE key = 0x00000000");
+        UntypedResultSet results = execute("SELECT key, \"42\" FROM \"Keyspace1\".\"JdbcInteger\"");
+        assertEquals(1, results.size());
+        UntypedResultSet.Row row = results.iterator().next();
+        assertEquals(ByteBufferUtil.bytes(1), row.getBytes("key"));
+        assertEquals("abc", row.getString("42"));
+    }
+}


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

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

Conflicts:
	src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
	src/java/org/apache/cassandra/cql3/Cql.g
	src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
	src/java/org/apache/cassandra/cql3/statements/Selectable.java
	src/java/org/apache/cassandra/cql3/statements/Selection.java


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

Branch: refs/heads/trunk
Commit: 9bf17e15a8c98a457a4855e3bb5d61dfeebf986a
Parents: 254cd85 a446e80
Author: Tyler Hobbs <ty...@datastax.com>
Authored: Tue Nov 4 17:26:11 2014 -0600
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Tue Nov 4 17:26:11 2014 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/cql3/ColumnIdentifier.java |  61 ++++++++-
 src/java/org/apache/cassandra/cql3/Cql.g        | 127 ++++++++++---------
 .../cassandra/cql3/MultiColumnRelation.java     |  12 +-
 .../org/apache/cassandra/cql3/Operation.java    |  15 ++-
 .../cassandra/cql3/SingleColumnRelation.java    |  12 +-
 .../cassandra/cql3/selection/RawSelector.java   |   9 +-
 .../cassandra/cql3/selection/Selectable.java    |  60 +++++++++
 .../cassandra/cql3/selection/Selection.java     |   4 +-
 .../cql3/statements/AlterTableStatement.java    |  31 +++--
 .../cql3/statements/CreateIndexStatement.java   |  10 +-
 .../cql3/statements/DeleteStatement.java        |   7 +-
 .../cassandra/cql3/statements/IndexTarget.java  |  30 ++++-
 .../cql3/statements/ModificationStatement.java  |  21 +--
 .../cql3/statements/SelectStatement.java        |  26 ++--
 .../cql3/statements/UpdateStatement.java        |  17 +--
 .../unit/org/apache/cassandra/SchemaLoader.java |   6 +-
 .../cassandra/cql3/ThriftCompatibilityTest.java |  72 +++++++++++
 18 files changed, 382 insertions(+), 139 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9bf17e15/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
index 9b4f29f,c1dcd87..e1454b1
--- a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
@@@ -24,11 -23,10 +24,13 @@@ import java.nio.ByteBuffer
  import org.apache.cassandra.cache.IMeasurableMemory;
  import org.apache.cassandra.config.CFMetaData;
  import org.apache.cassandra.config.ColumnDefinition;
 -import org.apache.cassandra.cql3.statements.Selectable;
 +import org.apache.cassandra.cql3.selection.Selectable;
 +import org.apache.cassandra.cql3.selection.Selector;
 +import org.apache.cassandra.cql3.selection.SimpleSelector;
  import org.apache.cassandra.db.marshal.AbstractType;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
+ import org.apache.cassandra.db.marshal.CompositeType;
+ import org.apache.cassandra.db.marshal.UTF8Type;
  import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.utils.ObjectSizes;
  import org.apache.cassandra.utils.memory.AbstractAllocator;
@@@ -37,7 -35,7 +39,7 @@@
   * Represents an identifer for a CQL column definition.
   * TODO : should support light-weight mode without text representation for when not interned
   */
- public class ColumnIdentifier extends Selectable implements IMeasurableMemory
 -public class ColumnIdentifier implements Selectable, IMeasurableMemory
++public class ColumnIdentifier extends org.apache.cassandra.cql3.selection.Selectable implements IMeasurableMemory
  {
      public final ByteBuffer bytes;
      private final String text;
@@@ -107,12 -105,61 +109,69 @@@
          return new ColumnIdentifier(allocator.clone(bytes), text);
      }
  
 +    public Selector.Factory newSelectorFactory(CFMetaData cfm, List<ColumnDefinition> defs) throws InvalidRequestException
 +    {
 +        ColumnDefinition def = cfm.getColumnDefinition(this);
 +        if (def == null)
 +            throw new InvalidRequestException(String.format("Undefined name %s in selection clause", this));
 +
 +        return SimpleSelector.newFactory(def.name.toString(), addAndGetIndex(def, defs), def.type);
 +    }
+ 
+     /**
+      * Because Thrift-created tables may have a non-text comparator, we cannot determine the proper 'key' until
+      * we know the comparator. ColumnIdentifier.Raw is a placeholder that can be converted to a real ColumnIdentifier
+      * once the comparator is known with prepare(). This should only be used with identifiers that are actual
+      * column names. See CASSANDRA-8178 for more background.
+      */
+     public static class Raw implements Selectable.Raw
+     {
+         private final String rawText;
+         private final String text;
+ 
+         public Raw(String rawText, boolean keepCase)
+         {
+             this.rawText = rawText;
+             this.text =  keepCase ? rawText : rawText.toLowerCase(Locale.US);
+         }
+ 
+         public ColumnIdentifier prepare(CFMetaData cfm)
+         {
+             AbstractType<?> comparator = cfm.comparator.asAbstractType();
+             if (cfm.getIsDense() || comparator instanceof CompositeType || comparator instanceof UTF8Type)
+                 return new ColumnIdentifier(text, true);
+ 
+             // We have a Thrift-created table with a non-text comparator.  We need to parse column names with the comparator
+             // to get the correct ByteBuffer representation.  However, this doesn't apply to key aliases, so we need to
+             // make a special check for those and treat them normally.  See CASSANDRA-8178.
+             ByteBuffer bufferName = ByteBufferUtil.bytes(text);
+             for (ColumnDefinition def : cfm.partitionKeyColumns())
+             {
+                 if (def.name.bytes.equals(bufferName))
+                     return new ColumnIdentifier(text, true);
+             }
+             return new ColumnIdentifier(comparator.fromString(rawText), text);
+         }
+ 
+         @Override
+         public final int hashCode()
+         {
+             return text.hashCode();
+         }
+ 
+         @Override
+         public final boolean equals(Object o)
+         {
+             if(!(o instanceof ColumnIdentifier.Raw))
+                 return false;
+             ColumnIdentifier.Raw that = (ColumnIdentifier.Raw)o;
+             return text.equals(that.text);
+         }
+ 
+         @Override
+         public String toString()
+         {
+             return text;
+         }
+     }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9bf17e15/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Cql.g
index 89dde62,b1c598b..adfe7a3
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@@ -263,8 -260,10 +263,8 @@@ useStatement returns [UseStatement stmt
  selectStatement returns [SelectStatement.RawStatement expr]
      @init {
          boolean isDistinct = false;
 -        boolean isCount = false;
 -        ColumnIdentifier countAlias = null;
          Term.Raw limit = null;
-         Map<ColumnIdentifier, Boolean> orderings = new LinkedHashMap<ColumnIdentifier, Boolean>();
+         Map<ColumnIdentifier.Raw, Boolean> orderings = new LinkedHashMap<ColumnIdentifier.Raw, Boolean>();
          boolean allowFiltering = false;
      }
      : K_SELECT ( ( K_DISTINCT { isDistinct = true; } )? sclause=selectClause
@@@ -309,13 -310,8 +309,13 @@@ selectionFunctionArgs returns [List<Sel
      ;
  
  selectCountClause returns [List<RawSelector> expr]
 -    : '\*'           { $expr = Collections.<RawSelector>emptyList();}
 -    | i=INTEGER      { if (!i.getText().equals("1")) addRecognitionError("Only COUNT(1) is supported, got COUNT(" + i.getText() + ")"); $expr = Collections.<RawSelector>emptyList();}
 +    @init{ ColumnIdentifier alias = new ColumnIdentifier("count", false); }
-     : K_COUNT '(' countArgument ')' (K_AS c=cident { alias = c; })? { $expr = new ArrayList<RawSelector>(); $expr.add( new RawSelector(new Selectable.WithFunction(new FunctionName("countRows"), Collections.<Selectable>emptyList()), alias));}
++    : K_COUNT '(' countArgument ')' (K_AS c=ident { alias = c; })? { $expr = new ArrayList<RawSelector>(); $expr.add( new RawSelector(new Selectable.WithFunction.Raw(new FunctionName("countRows"), Collections.<Selectable.Raw>emptyList()), alias));}
 +    ;
 +
 +countArgument
 +    : '\*'
 +    | i=INTEGER { if (!i.getText().equals("1")) addRecognitionError("Only COUNT(1) is supported, got COUNT(" + i.getText() + ")");}
      ;
  
  whereClause returns [List<Relation> clause]
@@@ -323,11 -319,12 +323,11 @@@
      : relation[$clause] (K_AND relation[$clause])*
      ;
  
- orderByClause[Map<ColumnIdentifier, Boolean> orderings]
+ orderByClause[Map<ColumnIdentifier.Raw, Boolean> orderings]
      @init{
 -        ColumnIdentifier.Raw orderBy = null;
          boolean reversed = false;
      }
 -    : c=cident { orderBy = c; } (K_ASC | K_DESC { reversed = true; })? { orderings.put(c, reversed); }
 +    : c=cident (K_ASC | K_DESC { reversed = true; })? { orderings.put(c, reversed); }
      ;
  
  /**
@@@ -488,53 -485,6 +488,53 @@@ batchStatementObjective returns [Modifi
      | d=deleteStatement  { $statement = d; }
      ;
  
 +createFunctionStatement returns [CreateFunctionStatement expr]
 +    @init {
 +        boolean orReplace = false;
 +        boolean ifNotExists = false;
 +
 +        boolean deterministic = true;
 +        List<ColumnIdentifier> argsNames = new ArrayList<>();
 +        List<CQL3Type.Raw> argsTypes = new ArrayList<>();
 +    }
 +    : K_CREATE (K_OR K_REPLACE { orReplace = true; })?
 +      ((K_NON { deterministic = false; })? K_DETERMINISTIC)?
 +      K_FUNCTION
 +      (K_IF K_NOT K_EXISTS { ifNotExists = true; })?
 +      fn=functionName
 +      '('
 +        (
-           k=cident v=comparatorType { argsNames.add(k); argsTypes.add(v); }
-           ( ',' k=cident v=comparatorType { argsNames.add(k); argsTypes.add(v); } )*
++          k=ident v=comparatorType { argsNames.add(k); argsTypes.add(v); }
++          ( ',' k=ident v=comparatorType { argsNames.add(k); argsTypes.add(v); } )*
 +        )?
 +      ')'
 +      K_RETURNS rt = comparatorType
 +      K_LANGUAGE language = IDENT
 +      K_AS body = STRING_LITERAL
 +      { $expr = new CreateFunctionStatement(fn, $language.text.toLowerCase(), $body.text, deterministic, argsNames, argsTypes, rt, orReplace, ifNotExists); }
 +    ;
 +
 +dropFunctionStatement returns [DropFunctionStatement expr]
 +    @init {
 +        boolean ifExists = false;
 +        List<CQL3Type.Raw> argsTypes = new ArrayList<>();
 +        boolean argsPresent = false;
 +    }
 +    : K_DROP K_FUNCTION
 +      (K_IF K_EXISTS { ifExists = true; } )?
 +      fn=functionName
 +      (
 +        '('
 +          (
 +            v=comparatorType { argsTypes.add(v); }
 +            ( ',' v=comparatorType { argsTypes.add(v); } )*
 +          )?
 +        ')'
 +        { argsPresent = true; }
 +      )?
 +      { $expr = new DropFunctionStatement(fn, argsTypes, argsPresent, ifExists); }
 +    ;
 +
  /**
   * CREATE KEYSPACE [IF NOT EXISTS] <KEYSPACE> WITH attr1 = value1 AND attr2 = value2;
   */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9bf17e15/src/java/org/apache/cassandra/cql3/selection/RawSelector.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/RawSelector.java
index 372c4b3,0000000..c7e2658
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/RawSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/RawSelector.java
@@@ -1,55 -1,0 +1,56 @@@
 +/*
 + * 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.selection;
 +
 +import java.util.List;
 +
++import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.cql3.ColumnIdentifier;
 +
 +import com.google.common.base.Function;
 +import com.google.common.collect.Lists;
 +
 +public class RawSelector
 +{
-     public final Selectable selectable;
++    public final Selectable.Raw selectable;
 +    public final ColumnIdentifier alias;
 +
-     public RawSelector(Selectable selectable, ColumnIdentifier alias)
++    public RawSelector(Selectable.Raw selectable, ColumnIdentifier alias)
 +    {
 +        this.selectable = selectable;
 +        this.alias = alias;
 +    }
 +
 +    /**
 +     * Converts the specified list of <code>RawSelector</code>s into a list of <code>Selectable</code>s.
 +     *
 +     * @param raws the <code>RawSelector</code>s to converts.
 +     * @return a list of <code>Selectable</code>s
 +     */
-     public static List<Selectable> toSelectables(List<RawSelector> raws)
++    public static List<Selectable> toSelectables(List<RawSelector> raws, final CFMetaData cfm)
 +    {
 +        return Lists.transform(raws, new Function<RawSelector, Selectable>()
 +        {
 +            public Selectable apply(RawSelector raw)
 +            {
-                 return raw.selectable;
++                return raw.selectable.prepare(cfm);
 +            }
 +        });
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9bf17e15/src/java/org/apache/cassandra/cql3/selection/Selectable.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/Selectable.java
index 9324647,0000000..48ce11a
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selectable.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selectable.java
@@@ -1,166 -1,0 +1,226 @@@
 +/*
 + * 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.selection;
 +
++import java.util.ArrayList;
 +import java.util.List;
 +
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.ColumnIdentifier;
 +import org.apache.cassandra.cql3.functions.Function;
 +import org.apache.cassandra.cql3.functions.FunctionName;
 +import org.apache.cassandra.cql3.functions.Functions;
 +import org.apache.cassandra.db.marshal.AbstractType;
 +import org.apache.cassandra.db.marshal.UserType;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.commons.lang3.text.StrBuilder;
 +
 +public abstract class Selectable
 +{
 +    public abstract Selector.Factory newSelectorFactory(CFMetaData cfm, List<ColumnDefinition> defs)
 +            throws InvalidRequestException;
 +
 +    protected static int addAndGetIndex(ColumnDefinition def, List<ColumnDefinition> l)
 +    {
 +        int idx = l.indexOf(def);
 +        if (idx < 0)
 +        {
 +            idx = l.size();
 +            l.add(def);
 +        }
 +        return idx;
 +    }
 +
++    public static interface Raw
++    {
++        public Selectable prepare(CFMetaData cfm);
++    }
++
 +    public static class WritetimeOrTTL extends Selectable
 +    {
 +        public final ColumnIdentifier id;
 +        public final boolean isWritetime;
 +
 +        public WritetimeOrTTL(ColumnIdentifier id, boolean isWritetime)
 +        {
 +            this.id = id;
 +            this.isWritetime = isWritetime;
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return (isWritetime ? "writetime" : "ttl") + "(" + id + ")";
 +        }
 +
 +        public Selector.Factory newSelectorFactory(CFMetaData cfm,
 +                                                   List<ColumnDefinition> defs) throws InvalidRequestException
 +        {
 +            ColumnDefinition def = cfm.getColumnDefinition(id);
 +            if (def == null)
 +                throw new InvalidRequestException(String.format("Undefined name %s in selection clause", id));
 +            if (def.isPrimaryKeyColumn())
 +                throw new InvalidRequestException(
 +                        String.format("Cannot use selection function %s on PRIMARY KEY part %s",
 +                                      isWritetime ? "writeTime" : "ttl",
 +                                      def.name));
 +            if (def.type.isCollection())
 +                throw new InvalidRequestException(String.format("Cannot use selection function %s on collections",
 +                                                                isWritetime ? "writeTime" : "ttl"));
 +
 +            return WritetimeOrTTLSelector.newFactory(def.name.toString(), addAndGetIndex(def, defs), isWritetime);
 +        }
++
++        public static class Raw implements Selectable.Raw
++        {
++            private final ColumnIdentifier.Raw id;
++            private final boolean isWritetime;
++
++            public Raw(ColumnIdentifier.Raw id, boolean isWritetime)
++            {
++                this.id = id;
++                this.isWritetime = isWritetime;
++            }
++
++            public WritetimeOrTTL prepare(CFMetaData cfm)
++            {
++                return new WritetimeOrTTL(id.prepare(cfm), isWritetime);
++            }
++        }
 +    }
 +
 +    public static class WithFunction extends Selectable
 +    {
 +        public final FunctionName functionName;
 +        public final List<Selectable> args;
 +
 +        public WithFunction(FunctionName functionName, List<Selectable> args)
 +        {
 +            this.functionName = functionName;
 +            this.args = args;
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return new StrBuilder().append(functionName)
 +                                   .append("(")
 +                                   .appendWithSeparators(args, ", ")
 +                                   .append(")")
 +                                   .toString();
 +        }
 +
 +        public Selector.Factory newSelectorFactory(CFMetaData cfm,
 +                                                   List<ColumnDefinition> defs) throws InvalidRequestException
 +        {
 +            SelectorFactories factories  =
 +                    SelectorFactories.createFactoriesAndCollectColumnDefinitions(args, cfm, defs);
 +
 +            // resolve built-in functions before user defined functions
 +            Function fun = Functions.get(cfm.ksName, functionName, factories.newInstances(), cfm.ksName, cfm.cfName);
 +            if (fun == null)
 +                throw new InvalidRequestException(String.format("Unknown function '%s'", functionName));
 +            if (fun.returnType() == null)
 +                throw new InvalidRequestException(String.format("Unknown function %s called in selection clause",
 +                                                                functionName));
 +
 +            return AbstractFunctionSelector.newFactory(fun, factories);
 +        }
++
++        public static class Raw implements Selectable.Raw
++        {
++            private final FunctionName functionName;
++            private final List<Selectable.Raw> args;
++
++            public Raw(FunctionName functionName, List<Selectable.Raw> args)
++            {
++                this.functionName = functionName;
++                this.args = args;
++            }
++
++            public WithFunction prepare(CFMetaData cfm)
++            {
++                List<Selectable> preparedArgs = new ArrayList<>(args.size());
++                for (Selectable.Raw arg : args)
++                    preparedArgs.add(arg.prepare(cfm));
++                return new WithFunction(functionName, preparedArgs);
++            }
++        }
 +    }
 +
 +    public static class WithFieldSelection extends Selectable
 +    {
 +        public final Selectable selected;
 +        public final ColumnIdentifier field;
 +
 +        public WithFieldSelection(Selectable selected, ColumnIdentifier field)
 +        {
 +            this.selected = selected;
 +            this.field = field;
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("%s.%s", selected, field);
 +        }
 +
 +        public Selector.Factory newSelectorFactory(CFMetaData cfm,
 +                                                   List<ColumnDefinition> defs) throws InvalidRequestException
 +        {
 +            Selector.Factory factory = selected.newSelectorFactory(cfm, defs);
 +            AbstractType<?> type = factory.newInstance().getType();
 +            if (!(type instanceof UserType))
 +                throw new InvalidRequestException(
 +                        String.format("Invalid field selection: %s of type %s is not a user type",
 +                                      selected,
 +                                      type.asCQL3Type()));
 +
 +            UserType ut = (UserType) type;
 +            for (int i = 0; i < ut.size(); i++)
 +            {
 +                if (!ut.fieldName(i).equals(field.bytes))
 +                    continue;
 +                return FieldSelector.newFactory(ut, i, factory);
 +            }
 +            throw new InvalidRequestException(String.format("%s of type %s has no field %s",
 +                                                            selected,
 +                                                            type.asCQL3Type(),
 +                                                            field));
 +        }
++
++        public static class Raw implements Selectable.Raw
++        {
++            private final Selectable.Raw selected;
++            private final ColumnIdentifier.Raw field;
++
++            public Raw(Selectable.Raw selected, ColumnIdentifier.Raw field)
++            {
++                this.selected = selected;
++                this.field = field;
++            }
++
++            public WithFieldSelection prepare(CFMetaData cfm)
++            {
++                return new WithFieldSelection(selected.prepare(cfm), field.prepare(cfm));
++            }
++        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9bf17e15/src/java/org/apache/cassandra/cql3/selection/Selection.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/Selection.java
index cd5e2a8,0000000..17e2e92
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selection.java
@@@ -1,390 -1,0 +1,390 @@@
 +/*
 + * 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.selection;
 +
 +import java.nio.ByteBuffer;
 +import java.util.ArrayList;
 +import java.util.Collection;
 +import java.util.Iterator;
 +import java.util.List;
 +
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.ColumnIdentifier;
 +import org.apache.cassandra.cql3.ColumnSpecification;
 +import org.apache.cassandra.cql3.ResultSet;
 +import org.apache.cassandra.db.Cell;
 +import org.apache.cassandra.db.CounterCell;
 +import org.apache.cassandra.db.ExpiringCell;
 +import org.apache.cassandra.db.context.CounterContext;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +
 +import com.google.common.collect.Iterators;
 +
 +public abstract class Selection
 +{
 +    private final Collection<ColumnDefinition> columns;
 +    private final ResultSet.Metadata metadata;
 +    private final boolean collectTimestamps;
 +    private final boolean collectTTLs;
 +
 +    protected Selection(Collection<ColumnDefinition> columns, List<ColumnSpecification> metadata, boolean collectTimestamps, boolean collectTTLs)
 +    {
 +        this.columns = columns;
 +        this.metadata = new ResultSet.Metadata(metadata);
 +        this.collectTimestamps = collectTimestamps;
 +        this.collectTTLs = collectTTLs;
 +    }
 +
 +    // Overriden by SimpleSelection when appropriate.
 +    public boolean isWildcard()
 +    {
 +        return false;
 +    }
 +
 +    public ResultSet.Metadata getResultMetadata()
 +    {
 +        return metadata;
 +    }
 +
 +    public static Selection wildcard(CFMetaData cfm)
 +    {
 +        List<ColumnDefinition> all = new ArrayList<ColumnDefinition>(cfm.allColumns().size());
 +        Iterators.addAll(all, cfm.allColumnsInSelectOrder());
 +        return new SimpleSelection(all, true);
 +    }
 +
 +    public static Selection forColumns(Collection<ColumnDefinition> columns)
 +    {
 +        return new SimpleSelection(columns, false);
 +    }
 +
 +    public int addColumnForOrdering(ColumnDefinition c)
 +    {
 +        columns.add(c);
 +        metadata.addNonSerializedColumn(c);
 +        return columns.size() - 1;
 +    }
 +
 +    private static boolean isUsingFunction(List<RawSelector> rawSelectors)
 +    {
 +        for (RawSelector rawSelector : rawSelectors)
 +        {
 +            if (!(rawSelector.selectable instanceof ColumnIdentifier))
 +                return true;
 +        }
 +        return false;
 +    }
 +
 +    public static Selection fromSelectors(CFMetaData cfm, List<RawSelector> rawSelectors) throws InvalidRequestException
 +    {
 +        List<ColumnDefinition> defs = new ArrayList<ColumnDefinition>();
 +
 +        SelectorFactories factories =
-                 SelectorFactories.createFactoriesAndCollectColumnDefinitions(RawSelector.toSelectables(rawSelectors), cfm, defs);
++                SelectorFactories.createFactoriesAndCollectColumnDefinitions(RawSelector.toSelectables(rawSelectors, cfm), cfm, defs);
 +        List<ColumnSpecification> metadata = collectMetadata(cfm, rawSelectors, factories);
 +
 +        return isUsingFunction(rawSelectors) ? new SelectionWithFunctions(defs, metadata, factories)
 +                                             : new SimpleSelection(defs, metadata, false);
 +    }
 +
 +    private static List<ColumnSpecification> collectMetadata(CFMetaData cfm,
 +                                                             List<RawSelector> rawSelectors,
 +                                                             SelectorFactories factories)
 +    {
 +        List<ColumnSpecification> metadata = new ArrayList<ColumnSpecification>(rawSelectors.size());
 +        Iterator<RawSelector> iter = rawSelectors.iterator();
 +        for (Selector.Factory factory : factories)
 +        {
 +            ColumnSpecification colSpec = factory.getColumnSpecification(cfm);
 +            ColumnIdentifier alias = iter.next().alias;
 +            metadata.add(alias == null ? colSpec : colSpec.withAlias(alias));
 +        }
 +        return metadata;
 +    }
 +
 +    protected abstract Selectors newSelectors();
 +
 +    /**
 +     * @return the list of CQL3 columns value this SelectionClause needs.
 +     */
 +    public Collection<ColumnDefinition> getColumns()
 +    {
 +        return columns;
 +    }
 +
 +    public ResultSetBuilder resultSetBuilder(long now)
 +    {
 +        return new ResultSetBuilder(now);
 +    }
 +
 +    public abstract boolean isAggregate();
 +
 +    /**
 +     * Checks that selectors are either all aggregates or that none of them is.
 +     *
 +     * @param selectors the selectors to test.
-      * @param msgTemplate the error message template
++     * @param messageTemplate the error message template
 +     * @param messageArgs the error message arguments
 +     * @throws InvalidRequestException if some of the selectors are aggregate but not all of them
 +     */
 +    static void validateSelectors(List<Selector> selectors, String messageTemplate, Object... messageArgs)
 +            throws InvalidRequestException
 +    {
 +        int aggregates = 0;
 +        for (Selector s : selectors)
 +            if (s.isAggregate())
 +                ++aggregates;
 +
 +        if (aggregates != 0 && aggregates != selectors.size())
 +            throw new InvalidRequestException(String.format(messageTemplate, messageArgs));
 +    }
 +
 +    public class ResultSetBuilder
 +    {
 +        private final ResultSet resultSet;
 +
 +        /**
 +         * As multiple thread can access a <code>Selection</code> instance each <code>ResultSetBuilder</code> will use
 +         * its own <code>Selectors</code> instance.
 +         */
 +        private final Selectors selectors;
 +
 +        /*
 +         * We'll build CQL3 row one by one.
 +         * The currentRow is the values for the (CQL3) columns we've fetched.
 +         * We also collect timestamps and ttls for the case where the writetime and
 +         * ttl functions are used. Note that we might collect timestamp and/or ttls
 +         * we don't care about, but since the array below are allocated just once,
 +         * it doesn't matter performance wise.
 +         */
 +        List<ByteBuffer> current;
 +        final long[] timestamps;
 +        final int[] ttls;
 +        final long now;
 +
 +        private ResultSetBuilder(long now)
 +        {
 +            this.resultSet = new ResultSet(getResultMetadata().copy(), new ArrayList<List<ByteBuffer>>());
 +            this.selectors = newSelectors();
 +            this.timestamps = collectTimestamps ? new long[columns.size()] : null;
 +            this.ttls = collectTTLs ? new int[columns.size()] : null;
 +            this.now = now;
 +        }
 +
 +        public void add(ByteBuffer v)
 +        {
 +            current.add(v);
 +        }
 +
 +        public void add(Cell c)
 +        {
 +            current.add(isDead(c) ? null : value(c));
 +            if (timestamps != null)
 +            {
 +                timestamps[current.size() - 1] = isDead(c) ? Long.MIN_VALUE : c.timestamp();
 +            }
 +            if (ttls != null)
 +            {
 +                int ttl = -1;
 +                if (!isDead(c) && c instanceof ExpiringCell)
 +                    ttl = c.getLocalDeletionTime() - (int) (now / 1000);
 +                ttls[current.size() - 1] = ttl;
 +            }
 +        }
 +
 +        private boolean isDead(Cell c)
 +        {
 +            return c == null || !c.isLive(now);
 +        }
 +
 +        public void newRow() throws InvalidRequestException
 +        {
 +            if (current != null)
 +            {
 +                selectors.addInputRow(this);
 +                if (!selectors.isAggregate())
 +                {
 +                    resultSet.addRow(selectors.getOutputRow());
 +                    selectors.reset();
 +                }
 +            }
 +            current = new ArrayList<ByteBuffer>(columns.size());
 +        }
 +
 +        public ResultSet build() throws InvalidRequestException
 +        {
 +            if (current != null)
 +            {
 +                selectors.addInputRow(this);
 +                resultSet.addRow(selectors.getOutputRow());
 +                selectors.reset();
 +                current = null;
 +            }
 +            return resultSet;
 +        }
 +
 +        private ByteBuffer value(Cell c)
 +        {
 +            return (c instanceof CounterCell)
 +                ? ByteBufferUtil.bytes(CounterContext.instance().total(c.value()))
 +                : c.value();
 +        }
 +    }
 +
 +    private static interface Selectors
 +    {
 +        public boolean isAggregate();
 +
 +        /**
 +         * Adds the current row of the specified <code>ResultSetBuilder</code>.
 +         *
 +         * @param rs the <code>ResultSetBuilder</code>
 +         * @throws InvalidRequestException
 +         */
 +        public void addInputRow(ResultSetBuilder rs) throws InvalidRequestException;
 +
 +        public List<ByteBuffer> getOutputRow() throws InvalidRequestException;
 +
 +        public void reset();
 +    }
 +
 +    // Special cased selection for when no function is used (this save some allocations).
 +    private static class SimpleSelection extends Selection
 +    {
 +        private final boolean isWildcard;
 +
 +        public SimpleSelection(Collection<ColumnDefinition> columns, boolean isWildcard)
 +        {
 +            this(columns, new ArrayList<ColumnSpecification>(columns), isWildcard);
 +        }
 +
 +        public SimpleSelection(Collection<ColumnDefinition> columns, List<ColumnSpecification> metadata, boolean isWildcard)
 +        {
 +            /*
 +             * In theory, even a simple selection could have multiple time the same column, so we
 +             * could filter those duplicate out of columns. But since we're very unlikely to
 +             * get much duplicate in practice, it's more efficient not to bother.
 +             */
 +            super(columns, metadata, false, false);
 +            this.isWildcard = isWildcard;
 +        }
 +
 +        @Override
 +        public boolean isWildcard()
 +        {
 +            return isWildcard;
 +        }
 +
 +        public boolean isAggregate()
 +        {
 +            return false;
 +        }
 +
 +        protected Selectors newSelectors()
 +        {
 +            return new Selectors()
 +            {
 +                private List<ByteBuffer> current;
 +
 +                public void reset()
 +                {
 +                    current = null;
 +                }
 +
 +                public List<ByteBuffer> getOutputRow()
 +                {
 +                    return current;
 +                }
 +
 +                public void addInputRow(ResultSetBuilder rs) throws InvalidRequestException
 +                {
 +                    current = rs.current;
 +                }
 +
 +                public boolean isAggregate()
 +                {
 +                    return false;
 +                }
 +            };
 +        }
 +    }
 +
 +    private static class SelectionWithFunctions extends Selection
 +    {
 +        private final SelectorFactories factories;
 +
 +        public SelectionWithFunctions(Collection<ColumnDefinition> columns,
 +                                      List<ColumnSpecification> metadata,
 +                                      SelectorFactories factories) throws InvalidRequestException
 +        {
 +            super(columns, metadata, factories.containsWritetimeSelectorFactory(), factories.containsTTLSelectorFactory());
 +            this.factories = factories;
 +
 +            if (factories.doesAggregation() && !factories.containsOnlyAggregateFunctions())
 +                throw new InvalidRequestException("the select clause must either contains only aggregates or none");
 +        }
 +
 +        public boolean isAggregate()
 +        {
 +            return factories.containsOnlyAggregateFunctions();
 +        }
 +
 +        protected Selectors newSelectors()
 +        {
 +            return new Selectors()
 +            {
 +                private final List<Selector> selectors = factories.newInstances();
 +
 +                public void reset()
 +                {
 +                    for (int i = 0, m = selectors.size(); i < m; i++)
 +                    {
 +                        selectors.get(i).reset();
 +                    }
 +                }
 +
 +                public boolean isAggregate()
 +                {
 +                    return factories.containsOnlyAggregateFunctions();
 +                }
 +
 +                public List<ByteBuffer> getOutputRow() throws InvalidRequestException
 +                {
 +                    List<ByteBuffer> outputRow = new ArrayList<>(selectors.size());
 +
 +                    for (int i = 0, m = selectors.size(); i < m; i++)
 +                    {
 +                        outputRow.add(selectors.get(i).getOutput());
 +                    }
 +                    return outputRow;
 +                }
 +
 +                public void addInputRow(ResultSetBuilder rs) throws InvalidRequestException
 +                {
 +                    for (int i = 0, m = selectors.size(); i < m; i++)
 +                    {
 +                        selectors.get(i).addInput(rs);
 +                    }
 +                }
 +            };
 +        }
 +
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9bf17e15/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9bf17e15/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9bf17e15/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index de9da86,2632ee2..686f50f
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -101,7 -98,7 +101,7 @@@ public class SelectStatement implement
      private boolean selectsOnlyStaticColumns;
  
      // Used by forSelection below
-     private static final Parameters defaultParameters = new Parameters(Collections.<ColumnIdentifier, Boolean>emptyMap(), false, false);
 -    private static final Parameters defaultParameters = new Parameters(Collections.<ColumnIdentifier.Raw, Boolean>emptyMap(), false, false, null, false);
++    private static final Parameters defaultParameters = new Parameters(Collections.<ColumnIdentifier.Raw, Boolean>emptyMap(), false, false);
  
      private static final Predicate<ColumnDefinition> isStaticFilter = new Predicate<ColumnDefinition>()
      {
@@@ -2120,12 -2166,16 +2122,12 @@@
  
      public static class Parameters
      {
-         private final Map<ColumnIdentifier, Boolean> orderings;
+         private final Map<ColumnIdentifier.Raw, Boolean> orderings;
          private final boolean isDistinct;
 -        private final boolean isCount;
 -        private final ColumnIdentifier countAlias;
          private final boolean allowFiltering;
  
-         public Parameters(Map<ColumnIdentifier, Boolean> orderings,
+         public Parameters(Map<ColumnIdentifier.Raw, Boolean> orderings,
                            boolean isDistinct,
 -                          boolean isCount,
 -                          ColumnIdentifier countAlias,
                            boolean allowFiltering)
          {
              this.orderings = orderings;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9bf17e15/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9bf17e15/test/unit/org/apache/cassandra/SchemaLoader.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/SchemaLoader.java
index bb862e3,4a1c104..2fcfd55
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@@ -299,33 -309,10 +299,33 @@@ public class SchemaLoade
          if (Boolean.parseBoolean(System.getProperty("cassandra.test.compression", "false")))
              useCompression(schema);
  
 -        return schema;
 +        // if you're messing with low-level sstable stuff, it can be useful to inject the schema directly
 +        // Schema.instance.load(schemaDefinition());
 +        for (KSMetaData ksm : schema)
 +            MigrationManager.announceNewKeyspace(ksm, false);
 +    }
 +
 +    public static void createKeyspace(String keyspaceName,
 +                                      Class<? extends AbstractReplicationStrategy> strategy,
 +                                      Map<String, String> options,
 +                                      CFMetaData... cfmetas) throws ConfigurationException
 +    {
 +        createKeyspace(keyspaceName, true, true, strategy, options, cfmetas);
 +    }
 +
 +    public static void createKeyspace(String keyspaceName,
 +                                      boolean durable,
 +                                      boolean announceLocally,
 +                                      Class<? extends AbstractReplicationStrategy> strategy,
 +                                      Map<String, String> options,
 +                                      CFMetaData... cfmetas) throws ConfigurationException
 +    {
 +        KSMetaData ksm = durable ? KSMetaData.testMetadata(keyspaceName, strategy, options, cfmetas)
 +                                 : KSMetaData.testMetadataNotDurable(keyspaceName, strategy, options, cfmetas);
 +        MigrationManager.announceNewKeyspace(ksm, announceLocally);
      }
  
--    private static ColumnDefinition integerColumn(String ksName, String cfName)
++    public static ColumnDefinition integerColumn(String ksName, String cfName)
      {
          return new ColumnDefinition(ksName,
                                      cfName,
@@@ -412,7 -399,7 +412,7 @@@
          return CFMetaData.denseCFMetaData(ksName, cfName, comp).defaultValidator(comp);
      }
  
--    private static CFMetaData jdbcSparseCFMD(String ksName, String cfName, AbstractType comp)
++    public static CFMetaData jdbcSparseCFMD(String ksName, String cfName, AbstractType comp)
      {
          return CFMetaData.sparseCFMetaData(ksName, cfName, comp).defaultValidator(comp);
      }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9bf17e15/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
index 0000000,662800b..deb3082
mode 000000,100644..100644
--- a/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
@@@ -1,0 -1,57 +1,72 @@@
+ /*
+  * 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 org.apache.cassandra.config.KSMetaData;
++import org.apache.cassandra.db.marshal.Int32Type;
++import org.apache.cassandra.locator.SimpleStrategy;
+ import org.apache.cassandra.utils.ByteBufferUtil;
++import org.junit.BeforeClass;
+ import org.junit.Test;
+ 
+ import org.apache.cassandra.SchemaLoader;
+ 
+ import static org.junit.Assert.assertEquals;
+ 
+ public class ThriftCompatibilityTest extends SchemaLoader
+ {
++    @BeforeClass
++    public static void defineSchema() throws Exception
++    {
++        SchemaLoader.prepareServer();
++        SchemaLoader.createKeyspace("thriftcompat",
++                                    SimpleStrategy.class,
++                                    KSMetaData.optsWithRF(1),
++                                    jdbcSparseCFMD("thriftcompat", "JdbcInteger", Int32Type.instance)
++                                            .addColumnDefinition(integerColumn("thriftcompat", "JdbcInteger")));
++    }
++
+     private static UntypedResultSet execute(String query) throws Throwable
+     {
+         try
+         {
+             return QueryProcessor.executeInternal(String.format(query));
+         }
+         catch (RuntimeException exc)
+         {
+             if (exc.getCause() != null)
+                 throw exc.getCause();
+             throw exc;
+         }
+     }
+ 
+     /** Test For CASSANDRA-8178 */
+     @Test
+     public void testNonTextComparator() throws Throwable
+     {
+         // the comparator is IntegerType, and there is a column named 42 with a UTF8Type validation type
 -        execute("INSERT INTO \"Keyspace1\".\"JdbcInteger\" (key, \"42\") VALUES (0x00000001, 'abc')");
 -        execute("UPDATE \"Keyspace1\".\"JdbcInteger\" SET \"42\" = 'abc' WHERE key = 0x00000001");
 -        execute("DELETE \"42\" FROM \"Keyspace1\".\"JdbcInteger\" WHERE key = 0x00000000");
 -        UntypedResultSet results = execute("SELECT key, \"42\" FROM \"Keyspace1\".\"JdbcInteger\"");
++        execute("INSERT INTO \"thriftcompat\".\"JdbcInteger\" (key, \"42\") VALUES (0x00000001, 'abc')");
++        execute("UPDATE \"thriftcompat\".\"JdbcInteger\" SET \"42\" = 'abc' WHERE key = 0x00000001");
++        execute("DELETE \"42\" FROM \"thriftcompat\".\"JdbcInteger\" WHERE key = 0x00000000");
++        UntypedResultSet results = execute("SELECT key, \"42\" FROM \"thriftcompat\".\"JdbcInteger\"");
+         assertEquals(1, results.size());
+         UntypedResultSet.Row row = results.iterator().next();
+         assertEquals(ByteBufferUtil.bytes(1), row.getBytes("key"));
+         assertEquals("abc", row.getString("42"));
+     }
+ }


[2/3] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1

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


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

Branch: refs/heads/trunk
Commit: a446e80e08e20196c08aa488508c8440472c0623
Parents: 5acc7a9 11e6571
Author: Tyler Hobbs <ty...@datastax.com>
Authored: Tue Nov 4 16:43:17 2014 -0600
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Tue Nov 4 16:43:17 2014 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/cql3/ColumnIdentifier.java |  61 +++++++++
 src/java/org/apache/cassandra/cql3/Cql.g        | 125 ++++++++++---------
 .../cassandra/cql3/MultiColumnRelation.java     |  12 +-
 .../org/apache/cassandra/cql3/Operation.java    |  15 +--
 .../cassandra/cql3/SingleColumnRelation.java    |  12 +-
 .../cql3/statements/AlterTableStatement.java    |  31 +++--
 .../cql3/statements/CreateIndexStatement.java   |  10 +-
 .../cql3/statements/DeleteStatement.java        |   7 +-
 .../cassandra/cql3/statements/IndexTarget.java  |  30 ++++-
 .../cql3/statements/ModificationStatement.java  |  21 ++--
 .../cassandra/cql3/statements/RawSelector.java  |   4 +-
 .../cql3/statements/SelectStatement.java        |  26 ++--
 .../cassandra/cql3/statements/Selectable.java   |  62 +++++++++
 .../cassandra/cql3/statements/Selection.java    |  38 +++---
 .../cql3/statements/UpdateStatement.java        |  17 +--
 .../unit/org/apache/cassandra/SchemaLoader.java |   2 +-
 .../cassandra/cql3/ThriftCompatibilityTest.java |  57 +++++++++
 18 files changed, 385 insertions(+), 146 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a446e80e/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 9dbb66b,d5f607f..ddbc810
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,22 -1,5 +1,23 @@@
 -2.0.12:
 +2.1.2
 + * Fix issues with CONTAINS (KEY) queries on secondary indexes
 +   (CASSANDRA-8147)
 + * Fix read-rate tracking of sstables for some queries (CASSANDRA-8239)
 + * Fix default timestamp in QueryOptions (CASSANDRA-8246)
 + * Set socket timeout when reading remote version (CASSANDRA-8188)
 + * Refactor how we track live size (CASSANDRA-7852)
 + * Make sure unfinished compaction files are removed (CASSANDRA-8124)
 + * Fix shutdown when run as Windows service (CASSANDRA-8136)
 + * Fix DESCRIBE TABLE with custom indexes (CASSANDRA-8031)
 + * Fix race in RecoveryManagerTest (CASSANDRA-8176)
 + * Avoid IllegalArgumentException while sorting sstables in
 +   IndexSummaryManager (CASSANDRA-8182)
 + * Shutdown JVM on file descriptor exhaustion (CASSANDRA-7579)
 + * Add 'die' policy for commit log and disk failure (CASSANDRA-7927)
 + * Fix installing as service on Windows (CASSANDRA-8115)
 + * Fix CREATE TABLE for CQL2 (CASSANDRA-8144)
 + * Avoid boxing in ColumnStats min/max trackers (CASSANDRA-8109)
 +Merged from 2.0:
+  * Correctly handle non-text column names in cql3 (CASSANDRA-8178)
   * Fix deletion for indexes on primary key columns (CASSANDRA-8206)
   * Add 'nodetool statusgossip' (CASSANDRA-8125)
   * Improve client notification that nodes are ready for requests (CASSANDRA-7510)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a446e80e/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
index fa151d2,f284436..c1dcd87
--- a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
@@@ -20,12 -20,13 +20,16 @@@ package org.apache.cassandra.cql3
  import java.util.Locale;
  import java.nio.ByteBuffer;
  
 +import org.apache.cassandra.cache.IMeasurableMemory;
+ import org.apache.cassandra.config.CFMetaData;
+ import org.apache.cassandra.config.ColumnDefinition;
  import org.apache.cassandra.cql3.statements.Selectable;
  import org.apache.cassandra.db.marshal.AbstractType;
+ import org.apache.cassandra.db.marshal.CompositeType;
+ import org.apache.cassandra.db.marshal.UTF8Type;
  import org.apache.cassandra.utils.ByteBufferUtil;
 +import org.apache.cassandra.utils.ObjectSizes;
 +import org.apache.cassandra.utils.memory.AbstractAllocator;
  
  /**
   * Represents an identifer for a CQL column definition.
@@@ -82,23 -75,59 +86,80 @@@ public class ColumnIdentifier implement
          return text;
      }
  
 +    public long unsharedHeapSize()
 +    {
 +        return EMPTY_SIZE
 +             + ObjectSizes.sizeOnHeapOf(bytes)
 +             + ObjectSizes.sizeOf(text);
 +    }
 +
 +    public long unsharedHeapSizeExcludingData()
 +    {
 +        return EMPTY_SIZE
 +             + ObjectSizes.sizeOnHeapExcludingData(bytes)
 +             + ObjectSizes.sizeOf(text);
 +    }
 +
 +    public ColumnIdentifier clone(AbstractAllocator allocator)
 +    {
 +        return new ColumnIdentifier(allocator.clone(bytes), text);
 +    }
 +
++
+     /**
+      * Because Thrift-created tables may have a non-text comparator, we cannot determine the proper 'key' until
+      * we know the comparator. ColumnIdentifier.Raw is a placeholder that can be converted to a real ColumnIdentifier
+      * once the comparator is known with prepare(). This should only be used with identifiers that are actual
+      * column names. See CASSANDRA-8178 for more background.
+      */
+     public static class Raw implements Selectable.Raw
+     {
+         private final String rawText;
+         private final String text;
+ 
+         public Raw(String rawText, boolean keepCase)
+         {
+             this.rawText = rawText;
 -            this.text =  keepCase ? rawText : rawText.toLowerCase();
++            this.text =  keepCase ? rawText : rawText.toLowerCase(Locale.US);
+         }
+ 
+         public ColumnIdentifier prepare(CFMetaData cfm)
+         {
 -            if (cfm.getIsDense() || cfm.comparator instanceof CompositeType || cfm.comparator instanceof UTF8Type)
++            AbstractType<?> comparator = cfm.comparator.asAbstractType();
++            if (cfm.getIsDense() || comparator instanceof CompositeType || comparator instanceof UTF8Type)
+                 return new ColumnIdentifier(text, true);
+ 
+             // We have a Thrift-created table with a non-text comparator.  We need to parse column names with the comparator
+             // to get the correct ByteBuffer representation.  However, this doesn't apply to key aliases, so we need to
+             // make a special check for those and treat them normally.  See CASSANDRA-8178.
+             ByteBuffer bufferName = ByteBufferUtil.bytes(text);
+             for (ColumnDefinition def : cfm.partitionKeyColumns())
+             {
 -                if (def.name.equals(bufferName))
++                if (def.name.bytes.equals(bufferName))
+                     return new ColumnIdentifier(text, true);
+             }
 -            return new ColumnIdentifier(cfm.comparator.fromString(rawText), text);
++            return new ColumnIdentifier(comparator.fromString(rawText), text);
+         }
+ 
+         @Override
+         public final int hashCode()
+         {
+             return text.hashCode();
+         }
+ 
+         @Override
+         public final boolean equals(Object o)
+         {
+             if(!(o instanceof ColumnIdentifier.Raw))
+                 return false;
+             ColumnIdentifier.Raw that = (ColumnIdentifier.Raw)o;
+             return text.equals(that.text);
+         }
+ 
+         @Override
+         public String toString()
+         {
+             return text;
+         }
+     }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a446e80e/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Cql.g
index 189ae7d,43b1c01..b1c598b
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@@ -290,23 -277,21 +290,23 @@@ selectClause returns [List<RawSelector
  
  selector returns [RawSelector s]
      @init{ ColumnIdentifier alias = null; }
-     : us=unaliasedSelector (K_AS c=cident { alias = c; })? { $s = new RawSelector(us, alias); }
+     : us=unaliasedSelector (K_AS c=ident { alias = c; })? { $s = new RawSelector(us, alias); }
      ;
  
- unaliasedSelector returns [Selectable s]
-     @init { Selectable tmp = null; }
+ unaliasedSelector returns [Selectable.Raw s]
 -    : c=cident                                  { $s = c; }
 -    | K_WRITETIME '(' c=cident ')'              { $s = new Selectable.WritetimeOrTTL.Raw(c, true); }
 -    | K_TTL       '(' c=cident ')'              { $s = new Selectable.WritetimeOrTTL.Raw(c, false); }
 -    | f=functionName args=selectionFunctionArgs { $s = new Selectable.WithFunction.Raw(f, args); }
++    @init { Selectable.Raw tmp = null; }
 +    :  ( c=cident                                  { tmp = c; }
-        | K_WRITETIME '(' c=cident ')'              { tmp = new Selectable.WritetimeOrTTL(c, true); }
-        | K_TTL       '(' c=cident ')'              { tmp = new Selectable.WritetimeOrTTL(c, false); }
-        | f=functionName args=selectionFunctionArgs { tmp = new Selectable.WithFunction(f, args); }
-        ) ( '.' fi=cident { tmp = new Selectable.WithFieldSelection(tmp, fi); } )* { $s = tmp; }
++       | K_WRITETIME '(' c=cident ')'              { tmp = new Selectable.WritetimeOrTTL.Raw(c, true); }
++       | K_TTL       '(' c=cident ')'              { tmp = new Selectable.WritetimeOrTTL.Raw(c, false); }
++       | f=functionName args=selectionFunctionArgs { tmp = new Selectable.WithFunction.Raw(f, args); }
++       ) ( '.' fi=cident { tmp = new Selectable.WithFieldSelection.Raw(tmp, fi); } )* { $s = tmp; }
      ;
  
- selectionFunctionArgs returns [List<Selectable> a]
+ selectionFunctionArgs returns [List<Selectable.Raw> a]
      : '(' ')' { $a = Collections.emptyList(); }
-     | '(' s1=unaliasedSelector { List<Selectable> args = new ArrayList<Selectable>(); args.add(s1); }
+     | '(' s1=unaliasedSelector { List<Selectable.Raw> args = new ArrayList<Selectable.Raw>(); args.add(s1); }
            ( ',' sn=unaliasedSelector { args.add(sn); } )*
 -       ')' { $a = args; }
 +      ')' { $a = args; }
      ;
  
  selectCountClause returns [List<RawSelector> expr]
@@@ -535,29 -520,9 +535,29 @@@ cfamProperty[CreateTableStatement.RawSt
  
  cfamOrdering[CreateTableStatement.RawStatement expr]
      @init{ boolean reversed=false; }
-     : k=cident (K_ASC | K_DESC { reversed=true;} ) { $expr.setOrdering(k, reversed); }
+     : k=ident (K_ASC | K_DESC { reversed=true;} ) { $expr.setOrdering(k, reversed); }
      ;
  
 +
 +/**
 + * CREATE TYPE foo (
 + *    <name1> <type1>,
 + *    <name2> <type2>,
 + *    ....
 + * )
 + */
 +createTypeStatement returns [CreateTypeStatement expr]
 +    @init { boolean ifNotExists = false; }
 +    : K_CREATE K_TYPE (K_IF K_NOT K_EXISTS { ifNotExists = true; } )?
 +         tn=userTypeName { $expr = new CreateTypeStatement(tn, ifNotExists); }
 +         '(' typeColumns[expr] ( ',' typeColumns[expr]? )* ')'
 +    ;
 +
 +typeColumns[CreateTypeStatement expr]
-     : k=cident v=comparatorType { $expr.addDefinition(k, v); }
++    : k=ident v=comparatorType { $expr.addDefinition(k, v); }
 +    ;
 +
 +
  /**
   * CREATE INDEX [IF NOT EXISTS] [indexName] ON <columnFamily> (<columnName>);
   * CREATE CUSTOM INDEX [IF NOT EXISTS] [indexName] ON <columnFamily> (<columnName>) USING <indexClass>;
@@@ -574,12 -539,6 +574,12 @@@ createIndexStatement returns [CreateInd
        { $expr = new CreateIndexStatement(cf, $idxName.text, id, props, ifNotExists); }
      ;
  
- indexIdent returns [IndexTarget id]
-     : c=cident                { $id = IndexTarget.of(c); }
-     | K_KEYS '(' c=cident ')' { $id = IndexTarget.keysOf(c); }
++indexIdent returns [IndexTarget.Raw id]
++    : c=cident                { $id = IndexTarget.Raw.of(c); }
++    | K_KEYS '(' c=cident ')' { $id = IndexTarget.Raw.keysOf(c); }
 +    ;
 +
 +
  /**
   * CREATE TRIGGER triggerName ON columnFamily USING 'triggerClass';
   */
@@@ -640,24 -594,6 +640,24 @@@ alterTableStatement returns [AlterTable
      ;
  
  /**
 + * ALTER TYPE <name> ALTER <field> TYPE <newtype>;
 + * ALTER TYPE <name> ADD <field> <newtype>;
 + * ALTER TYPE <name> RENAME <field> TO <newtype> AND ...;
 + */
 +alterTypeStatement returns [AlterTypeStatement expr]
 +    : K_ALTER K_TYPE name=userTypeName
-           ( K_ALTER f=cident K_TYPE v=comparatorType { $expr = AlterTypeStatement.alter(name, f, v); }
-           | K_ADD   f=cident v=comparatorType        { $expr = AlterTypeStatement.addition(name, f, v); }
++          ( K_ALTER f=ident K_TYPE v=comparatorType { $expr = AlterTypeStatement.alter(name, f, v); }
++          | K_ADD   f=ident v=comparatorType        { $expr = AlterTypeStatement.addition(name, f, v); }
 +          | K_RENAME
 +               { Map<ColumnIdentifier, ColumnIdentifier> renames = new HashMap<ColumnIdentifier, ColumnIdentifier>(); }
-                  id1=cident K_TO toId1=cident { renames.put(id1, toId1); }
-                  ( K_AND idn=cident K_TO toIdn=cident { renames.put(idn, toIdn); } )*
++                 id1=ident K_TO toId1=ident { renames.put(id1, toId1); }
++                 ( K_AND idn=ident K_TO toIdn=ident { renames.put(idn, toIdn); } )*
 +               { $expr = AlterTypeStatement.renames(name, renames); }
 +          )
 +    ;
 +
 +
 +/**
   * DROP KEYSPACE [IF EXISTS] <KSP>;
   */
  dropKeyspaceStatement returns [DropKeyspaceStatement ksp]
@@@ -812,8 -740,17 +812,17 @@@ userOption[UserOptions opts
  
  /** DEFINITIONS **/
  
- // Column Identifiers
- cident returns [ColumnIdentifier id]
+ // Column Identifiers.  These need to be treated differently from other
+ // identifiers because the underlying comparator is not necessarily text. See
+ // CASSANDRA-8178 for details.
+ cident returns [ColumnIdentifier.Raw id]
+     : t=IDENT              { $id = new ColumnIdentifier.Raw($t.text, false); }
+     | t=QUOTED_NAME        { $id = new ColumnIdentifier.Raw($t.text, true); }
+     | k=unreserved_keyword { $id = new ColumnIdentifier.Raw(k, false); }
+     ;
+ 
 -// Identifiers that do not refer to columns.
++// Identifiers that do not refer to columns or where the comparator is known to be text
+ ident returns [ColumnIdentifier id]
      : t=IDENT              { $id = new ColumnIdentifier($t.text, false); }
      | t=QUOTED_NAME        { $id = new ColumnIdentifier($t.text, true); }
      | k=unreserved_keyword { $id = new ColumnIdentifier(k, false); }
@@@ -841,10 -767,6 +850,10 @@@ columnFamilyName returns [CFName name
      : (cfOrKsName[name, true] '.')? cfOrKsName[name, false]
      ;
  
 +userTypeName returns [UTName name]
-     : (ks=cident '.')? ut=non_type_ident { return new UTName(ks, ut); }
++    : (ks=ident '.')? ut=non_type_ident { return new UTName(ks, ut); }
 +    ;
 +
  cfOrKsName[CFName name, boolean isKs]
      : t=IDENT              { if (isKs) $name.setKeyspace($t.text, false); else $name.setColumnFamily($t.text, false); }
      | t=QUOTED_NAME        { if (isKs) $name.setKeyspace($t.text, true); else $name.setColumnFamily($t.text, true); }
@@@ -886,26 -808,11 +895,26 @@@ collectionLiteral returns [Term.Raw val
      | '{' '}' { $value = new Sets.Literal(Collections.<Term.Raw>emptyList()); }
      ;
  
 +usertypeLiteral returns [UserTypes.Literal ut]
 +    @init{ Map<ColumnIdentifier, Term.Raw> m = new HashMap<ColumnIdentifier, Term.Raw>(); }
 +    @after{ $ut = new UserTypes.Literal(m); }
 +    // We don't allow empty literals because that conflicts with sets/maps and is currently useless since we don't allow empty user types
-     : '{' k1=cident ':' v1=term { m.put(k1, v1); } ( ',' kn=cident ':' vn=term { m.put(kn, vn); } )* '}'
++    : '{' k1=ident ':' v1=term { m.put(k1, v1); } ( ',' kn=ident ':' vn=term { m.put(kn, vn); } )* '}'
 +    ;
 +
 +tupleLiteral returns [Tuples.Literal tt]
 +    @init{ List<Term.Raw> l = new ArrayList<Term.Raw>(); }
 +    @after{ $tt = new Tuples.Literal(l); }
 +    : '(' t1=term { l.add(t1); } ( ',' tn=term { l.add(tn); } )* ')'
 +    ;
 +
  value returns [Term.Raw value]
      : c=constant           { $value = c; }
 -    | l=collection_literal { $value = l; }
 +    | l=collectionLiteral  { $value = l; }
 +    | u=usertypeLiteral    { $value = u; }
 +    | t=tupleLiteral       { $value = t; }
      | K_NULL               { $value = Constants.NULL_LITERAL; }
-     | ':' id=cident        { $value = newBindVariables(id); }
+     | ':' id=ident         { $value = newBindVariables(id); }
      | QMARK                { $value = newBindVariables(null); }
      ;
  
@@@ -935,17 -842,8 +944,17 @@@ term returns [Term.Raw term
      | '(' c=comparatorType ')' t=term  { $term = new TypeCast(c, t); }
      ;
  
- columnOperation[List<Pair<ColumnIdentifier, Operation.RawUpdate>> operations]
+ columnOperation[List<Pair<ColumnIdentifier.Raw, Operation.RawUpdate>> operations]
 -    : key=cident '=' t=term ('+' c=cident )?
 +    : key=cident columnOperationDifferentiator[operations, key]
 +    ;
-     
- columnOperationDifferentiator[List<Pair<ColumnIdentifier, Operation.RawUpdate>> operations, ColumnIdentifier key]
++
++columnOperationDifferentiator[List<Pair<ColumnIdentifier.Raw, Operation.RawUpdate>> operations, ColumnIdentifier.Raw key]
 +    : '=' normalColumnOperation[operations, key]
 +    | '[' k=term ']' specializedColumnOperation[operations, key, k]
 +    ;
-     
- normalColumnOperation[List<Pair<ColumnIdentifier, Operation.RawUpdate>> operations, ColumnIdentifier key]
++
++normalColumnOperation[List<Pair<ColumnIdentifier.Raw, Operation.RawUpdate>> operations, ColumnIdentifier.Raw key]
 +    : t=term ('+' c=cident )?
        {
            if (c == null)
            {
@@@ -972,31 -870,16 +981,31 @@@
                addRecognitionError("Only expressions of the form X = X " + ($i.text.charAt(0) == '-' ? '-' : '+') + " <value> are supported.");
            addRawUpdate(operations, key, new Operation.Addition(Constants.Literal.integer($i.text)));
        }
 -    | key=cident '[' k=term ']' '=' t=term
 +    ;
-       
- specializedColumnOperation[List<Pair<ColumnIdentifier, Operation.RawUpdate>> operations, ColumnIdentifier key, Term.Raw k]
++
++specializedColumnOperation[List<Pair<ColumnIdentifier.Raw, Operation.RawUpdate>> operations, ColumnIdentifier.Raw key, Term.Raw k]
 +    : '=' t=term
        {
            addRawUpdate(operations, key, new Operation.SetElement(k, t));
        }
      ;
  
- columnCondition[List<Pair<ColumnIdentifier, ColumnCondition.Raw>> conditions]
+ columnCondition[List<Pair<ColumnIdentifier.Raw, ColumnCondition.Raw>> conditions]
      // Note: we'll reject duplicates later
 -    : key=cident '=' t=term { conditions.add(Pair.create(key, ColumnCondition.Raw.simpleEqual(t))); }
 -    | key=cident '[' element=term ']' '=' t=term { conditions.add(Pair.create(key, ColumnCondition.Raw.collectionEqual(t, element))); } 
 +    : key=cident
 +        ( op=relationType t=term { conditions.add(Pair.create(key, ColumnCondition.Raw.simpleCondition(t, op))); }
 +        | K_IN
 +            ( values=singleColumnInValues { conditions.add(Pair.create(key, ColumnCondition.Raw.simpleInCondition(values))); }
 +            | marker=inMarker { conditions.add(Pair.create(key, ColumnCondition.Raw.simpleInCondition(marker))); }
 +            )
 +        | '[' element=term ']'
 +            ( op=relationType t=term { conditions.add(Pair.create(key, ColumnCondition.Raw.collectionCondition(t, element, op))); }
 +            | K_IN
 +                ( values=singleColumnInValues { conditions.add(Pair.create(key, ColumnCondition.Raw.collectionInCondition(element, values))); }
 +                | marker=inMarker { conditions.add(Pair.create(key, ColumnCondition.Raw.collectionInCondition(element, marker))); }
 +                )
 +            )
 +        )
      ;
  
  properties[PropertyDefinitions props]
@@@ -1004,8 -887,8 +1013,8 @@@
      ;
  
  property[PropertyDefinitions props]
-     : k=cident '=' (simple=propertyValue { try { $props.addProperty(k.toString(), simple); } catch (SyntaxException e) { addRecognitionError(e.getMessage()); } }
-                    |   map=mapLiteral    { try { $props.addProperty(k.toString(), convertPropertyMap(map)); } catch (SyntaxException e) { addRecognitionError(e.getMessage()); } })
+     : k=ident '=' (simple=propertyValue { try { $props.addProperty(k.toString(), simple); } catch (SyntaxException e) { addRecognitionError(e.getMessage()); } }
 -                   |   map=map_literal   { try { $props.addProperty(k.toString(), convertPropertyMap(map)); } catch (SyntaxException e) { addRecognitionError(e.getMessage()); } })
++                  |   map=mapLiteral    { try { $props.addProperty(k.toString(), convertPropertyMap(map)); } catch (SyntaxException e) { addRecognitionError(e.getMessage()); } })
      ;
  
  propertyValue returns [String str]
@@@ -1090,22 -975,12 +1099,22 @@@ tupleOfMarkersForTuples returns [List<T
  
  inMarkerForTuple returns [Tuples.INRaw marker]
      : QMARK { $marker = newTupleINBindVariables(null); }
-     | ':' name=cident { $marker = newTupleINBindVariables(name); }
+     | ':' name=ident { $marker = newTupleINBindVariables(name); }
      ;
  
 -comparatorType returns [CQL3Type t]
 -    : c=native_type     { $t = c; }
 +comparatorType returns [CQL3Type.Raw t]
 +    : n=native_type     { $t = CQL3Type.Raw.from(n); }
      | c=collection_type { $t = c; }
 +    | tt=tuple_type     { $t = tt; }
 +    | id=userTypeName   { $t = CQL3Type.Raw.userType(id); }
 +    | K_FROZEN '<' f=comparatorType '>'
 +      {
 +        try {
 +            $t = CQL3Type.Raw.frozen(f);
 +        } catch (InvalidRequestException e) {
 +            addRecognitionError(e.getMessage());
 +        }
 +      }
      | s=STRING_LITERAL
        {
          try {
@@@ -1161,18 -1030,9 +1170,18 @@@ usernam
      | STRING_LITERAL
      ;
  
- // Basically the same than cident, but we need to exlude existing CQL3 types
++// Basically the same as cident, but we need to exlude existing CQL3 types
 +// (which for some reason are not reserved otherwise)
 +non_type_ident returns [ColumnIdentifier id]
 +    : t=IDENT                    { if (reservedTypeNames.contains($t.text)) addRecognitionError("Invalid (reserved) user type name " + $t.text); $id = new ColumnIdentifier($t.text, false); }
 +    | t=QUOTED_NAME              { $id = new ColumnIdentifier($t.text, true); }
 +    | k=basic_unreserved_keyword { $id = new ColumnIdentifier(k, false); }
 +    | kk=K_KEY                   { $id = new ColumnIdentifier($kk.text, false); }
 +    ;
 +
  unreserved_keyword returns [String str]
      : u=unreserved_function_keyword     { $str = u; }
 -    | k=(K_TTL | K_COUNT | K_WRITETIME) { $str = $k.text; }
 +    | k=(K_TTL | K_COUNT | K_WRITETIME | K_KEY) { $str = $k.text; }
      ;
  
  unreserved_function_keyword returns [String str]

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a446e80e/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
index 9c51d89,d79bba5..37eb69e
--- a/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
+++ b/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
@@@ -39,7 -39,7 +39,7 @@@ public class MultiColumnRelation extend
  
      private final Tuples.INRaw inMarker;
  
-     private MultiColumnRelation(List<ColumnIdentifier> entities, Operator relationType, Term.MultiColumnRaw valuesOrMarker, List<? extends Term.MultiColumnRaw> inValues, Tuples.INRaw inMarker)
 -    private MultiColumnRelation(List<ColumnIdentifier.Raw> entities, Type relationType, Term.MultiColumnRaw valuesOrMarker, List<? extends Term.MultiColumnRaw> inValues, Tuples.INRaw inMarker)
++    private MultiColumnRelation(List<ColumnIdentifier.Raw> entities, Operator relationType, Term.MultiColumnRaw valuesOrMarker, List<? extends Term.MultiColumnRaw> inValues, Tuples.INRaw inMarker)
      {
          this.entities = entities;
          this.relationType = relationType;
@@@ -56,9 -56,9 +56,9 @@@
       * @param relationType the relation operator
       * @param valuesOrMarker a Tuples.Literal instance or a Tuples.Raw marker
       */
-     public static MultiColumnRelation createNonInRelation(List<ColumnIdentifier> entities, Operator relationType, Term.MultiColumnRaw valuesOrMarker)
 -    public static MultiColumnRelation createNonInRelation(List<ColumnIdentifier.Raw> entities, Type relationType, Term.MultiColumnRaw valuesOrMarker)
++    public static MultiColumnRelation createNonInRelation(List<ColumnIdentifier.Raw> entities, Operator relationType, Term.MultiColumnRaw valuesOrMarker)
      {
 -        assert relationType != Relation.Type.IN;
 +        assert relationType != Operator.IN;
          return new MultiColumnRelation(entities, relationType, valuesOrMarker, null, null);
      }
  
@@@ -68,9 -68,9 +68,9 @@@
       * @param entities the columns on the LHS of the relation
       * @param inValues a list of Tuples.Literal instances or a Tuples.Raw markers
       */
-     public static MultiColumnRelation createInRelation(List<ColumnIdentifier> entities, List<? extends Term.MultiColumnRaw> inValues)
+     public static MultiColumnRelation createInRelation(List<ColumnIdentifier.Raw> entities, List<? extends Term.MultiColumnRaw> inValues)
      {
 -        return new MultiColumnRelation(entities, Relation.Type.IN, null, inValues, null);
 +        return new MultiColumnRelation(entities, Operator.IN, null, inValues, null);
      }
  
      /**
@@@ -79,12 -79,12 +79,12 @@@
       * @param entities the columns on the LHS of the relation
       * @param inMarker a single IN marker
       */
-     public static MultiColumnRelation createSingleMarkerInRelation(List<ColumnIdentifier> entities, Tuples.INRaw inMarker)
+     public static MultiColumnRelation createSingleMarkerInRelation(List<ColumnIdentifier.Raw> entities, Tuples.INRaw inMarker)
      {
 -        return new MultiColumnRelation(entities, Relation.Type.IN, null, null, inMarker);
 +        return new MultiColumnRelation(entities, Operator.IN, null, null, inMarker);
      }
  
-     public List<ColumnIdentifier> getEntities()
+     public List<ColumnIdentifier.Raw> getEntities()
      {
          return entities;
      }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a446e80e/src/java/org/apache/cassandra/cql3/Operation.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Operation.java
index ebcb30b,1792a2d..32b6a12
--- a/src/java/org/apache/cassandra/cql3/Operation.java
+++ b/src/java/org/apache/cassandra/cql3/Operation.java
@@@ -19,10 -19,12 +19,11 @@@ 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;
 -import org.apache.cassandra.db.marshal.ListType;
 +import org.apache.cassandra.db.composites.Composite;
 +import org.apache.cassandra.db.marshal.*;
  import org.apache.cassandra.exceptions.InvalidRequestException;
  
  /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a446e80e/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
index fd427fd,d63493e..d5109f5
--- a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
+++ b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
@@@ -31,7 -31,7 +31,7 @@@ public class SingleColumnRelation exten
      private final List<Term.Raw> inValues;
      public final boolean onToken;
  
-     private SingleColumnRelation(ColumnIdentifier entity, Operator type, Term.Raw value, List<Term.Raw> inValues, boolean onToken)
 -    private SingleColumnRelation(ColumnIdentifier.Raw entity, Type type, Term.Raw value, List<Term.Raw> inValues, boolean onToken)
++    private SingleColumnRelation(ColumnIdentifier.Raw entity, Operator type, Term.Raw value, List<Term.Raw> inValues, boolean onToken)
      {
          this.entity = entity;
          this.relationType = type;
@@@ -47,22 -47,22 +47,22 @@@
       * @param type the type that describes how this entity relates to the value.
       * @param value the value being compared.
       */
-     public SingleColumnRelation(ColumnIdentifier entity, Operator type, Term.Raw value)
 -    public SingleColumnRelation(ColumnIdentifier.Raw entity, Type type, Term.Raw value)
++    public SingleColumnRelation(ColumnIdentifier.Raw entity, Operator type, Term.Raw value)
      {
          this(entity, type, value, null, false);
      }
  
-     public SingleColumnRelation(ColumnIdentifier entity, Operator type, Term.Raw value, boolean onToken)
 -    public SingleColumnRelation(ColumnIdentifier.Raw entity, Type type, Term.Raw value, boolean onToken)
++    public SingleColumnRelation(ColumnIdentifier.Raw entity, Operator type, Term.Raw value, boolean onToken)
      {
          this(entity, type, value, null, onToken);
      }
  
-     public static SingleColumnRelation createInRelation(ColumnIdentifier entity, List<Term.Raw> inValues)
+     public static SingleColumnRelation createInRelation(ColumnIdentifier.Raw entity, List<Term.Raw> inValues)
      {
 -        return new SingleColumnRelation(entity, Type.IN, null, inValues, false);
 +        return new SingleColumnRelation(entity, Operator.IN, null, inValues, false);
      }
  
-     public ColumnIdentifier getEntity()
+     public ColumnIdentifier.Raw getEntity()
      {
          return entity;
      }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a446e80e/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
index 33f4fff,32f949f..7be635f
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
@@@ -41,18 -43,18 +41,18 @@@ public class AlterTableStatement extend
      }
  
      public final Type oType;
 -    public final CQL3Type validator;
 +    public final CQL3Type.Raw validator;
-     public final ColumnIdentifier columnName;
+     public final ColumnIdentifier.Raw rawColumnName;
      private final CFPropDefs cfProps;
-     private final Map<ColumnIdentifier, ColumnIdentifier> renames;
+     private final Map<ColumnIdentifier.Raw, ColumnIdentifier.Raw> renames;
      private final boolean isStatic; // Only for ALTER ADD
  
      public AlterTableStatement(CFName name,
                                 Type type,
-                                ColumnIdentifier columnName,
+                                ColumnIdentifier.Raw columnName,
 -                               CQL3Type validator,
 +                               CQL3Type.Raw validator,
                                 CFPropDefs cfProps,
-                                Map<ColumnIdentifier, ColumnIdentifier> renames,
+                                Map<ColumnIdentifier.Raw, ColumnIdentifier.Raw> renames,
                                 boolean isStatic)
      {
          super(name);
@@@ -74,18 -76,26 +74,25 @@@
          // validated in announceMigration()
      }
  
 -    public boolean announceMigration() throws RequestValidationException
 +    public boolean announceMigration(boolean isLocalOnly) throws RequestValidationException
      {
          CFMetaData meta = validateColumnFamily(keyspace(), columnFamily());
 -        CFMetaData cfm = meta.clone();
 -
 -        CFDefinition cfDef = meta.getCfDef();
 +        CFMetaData cfm = meta.copy();
  
 +        CQL3Type validator = this.validator == null ? null : this.validator.prepare(keyspace());
+         ColumnIdentifier columnName = null;
 -        CFDefinition.Name name = null;
++        ColumnDefinition def = null;
+         if (rawColumnName != null)
+         {
+             columnName = rawColumnName.prepare(cfm);
 -            name = cfDef.get(columnName);
++            def = cfm.getColumnDefinition(columnName);
+         }
  
          switch (oType)
          {
              case ADD:
+                 assert columnName != null;
 -                if (cfDef.isCompact)
 +                if (cfm.comparator.isDense())
                      throw new InvalidRequestException("Cannot add new column to a COMPACT STORAGE table");
  
                  if (isStatic)
@@@ -142,12 -161,13 +149,13 @@@
                  break;
  
              case ALTER:
+                 assert columnName != null;
 -                if (name == null)
 -                    throw new InvalidRequestException(String.format("Column %s was not found in table %s", columnName, columnFamily()));
 +                if (def == null)
 +                    throw new InvalidRequestException(String.format("Cell %s was not found in table %s", columnName, columnFamily()));
  
 -                switch (name.kind)
 +                switch (def.kind)
                  {
 -                    case KEY_ALIAS:
 +                    case PARTITION_KEY:
                          AbstractType<?> newType = validator.getType();
                          if (newType instanceof CounterColumnType)
                              throw new InvalidRequestException(String.format("counter type is not supported for PRIMARY KEY part %s", columnName));
@@@ -223,17 -238,18 +231,18 @@@
                  break;
  
              case DROP:
+                 assert columnName != null;
 -                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 table");
 +                if (def == null)
                      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.regularAndStaticColumns())
@@@ -255,8 -271,12 +264,12 @@@
                  cfProps.applyToCFMetadata(cfm);
                  break;
              case RENAME:
-                 for (Map.Entry<ColumnIdentifier, ColumnIdentifier> entry : renames.entrySet())
-                     cfm.renameColumn(entry.getKey(), entry.getValue());
+                 for (Map.Entry<ColumnIdentifier.Raw, ColumnIdentifier.Raw> entry : renames.entrySet())
+                 {
+                     ColumnIdentifier from = entry.getKey().prepare(cfm);
+                     ColumnIdentifier to = entry.getValue().prepare(cfm);
 -                    cfm.renameColumn(from.key, from.toString(), to.key, to.toString());
++                    cfm.renameColumn(from, to);
+                 }
                  break;
          }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a446e80e/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
index fc5c4d1,e173e8c..c93adc2
--- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
@@@ -44,19 -40,19 +44,19 @@@ public class CreateIndexStatement exten
      private static final Logger logger = LoggerFactory.getLogger(CreateIndexStatement.class);
  
      private final String indexName;
-     private final IndexTarget target;
 -    private final ColumnIdentifier.Raw rawColumnName;
++    private final IndexTarget.Raw rawTarget;
      private final IndexPropDefs properties;
      private final boolean ifNotExists;
  
      public CreateIndexStatement(CFName name,
                                  String indexName,
-                                 IndexTarget target,
 -                                ColumnIdentifier.Raw rawColumnName,
++                                IndexTarget.Raw target,
                                  IndexPropDefs properties,
                                  boolean ifNotExists)
      {
          super(name);
          this.indexName = indexName;
-         this.target = target;
 -        this.rawColumnName = rawColumnName;
++        this.rawTarget = target;
          this.properties = properties;
          this.ifNotExists = ifNotExists;
      }
@@@ -69,17 -65,14 +69,18 @@@
      public void validate(ClientState state) throws RequestValidationException
      {
          CFMetaData cfm = ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
 -        if (cfm.getDefaultValidator().isCommutative())
 +        if (cfm.isCounter())
              throw new InvalidRequestException("Secondary indexes are not supported on counter tables");
  
 -        ColumnIdentifier columnName = rawColumnName.prepare(cfm);
 -        ColumnDefinition cd = cfm.getColumnDefinition(columnName.key);
++        IndexTarget target = rawTarget.prepare(cfm);
 +        ColumnDefinition cd = cfm.getColumnDefinition(target.column);
  
          if (cd == null)
 -            throw new InvalidRequestException("No column definition found for column " + columnName);
 +            throw new InvalidRequestException("No column definition found for column " + target.column);
 +
 +        boolean isMap = cd.type instanceof MapType;
 +        if (target.isCollectionKeys && !isMap)
 +            throw new InvalidRequestException("Cannot create index on keys of column " + target + " with non map type");
  
          if (cd.getIndexType() != null)
          {
@@@ -111,18 -94,22 +112,19 @@@
          // 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 boolean announceMigration() throws RequestValidationException
 +    public boolean announceMigration(boolean isLocalOnly) throws RequestValidationException
      {
-         logger.debug("Updating column {} definition for index {}", target.column, indexName);
 -        CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), columnFamily()).clone();
 -        ColumnIdentifier columnName = rawColumnName.prepare(cfm);
 -        logger.debug("Updating column {} definition for index {}", columnName, indexName);
 -        ColumnDefinition cd = cfm.getColumnDefinition(columnName.key);
 +        CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), columnFamily()).copy();
++        IndexTarget target = rawTarget.prepare(cfm);
++        logger.debug("Updating column {} definition for index {}", target.column, indexName);
 +        ColumnDefinition cd = cfm.getColumnDefinition(target.column);
  
          if (cd.getIndexType() != null && ifNotExists)
              return false;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a446e80e/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
index 93dd8c5,e3aa8ce..b49f60b
--- a/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
@@@ -126,9 -134,10 +126,10 @@@ public class DeleteStatement extends Mo
  
              for (Operation.RawDeletion deletion : deletions)
              {
-                 ColumnDefinition def = cfm.getColumnDefinition(deletion.affectedColumn());
 -                ColumnIdentifier id = deletion.affectedColumn().prepare(cfDef.cfm);
 -                CFDefinition.Name name = cfDef.get(id);
 -                if (name == null)
++                ColumnIdentifier id = deletion.affectedColumn().prepare(cfm);
++                ColumnDefinition def = cfm.getColumnDefinition(id);
 +                if (def == null)
-                     throw new InvalidRequestException(String.format("Unknown identifier %s", deletion.affectedColumn()));
+                     throw new InvalidRequestException(String.format("Unknown identifier %s", id));
  
                  // 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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a446e80e/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/IndexTarget.java
index 58a8c92,0000000..dc77bcc
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java
+++ b/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java
@@@ -1,42 -1,0 +1,60 @@@
 +/*
 + * 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 org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.cql3.ColumnIdentifier;
 +
 +public class IndexTarget
 +{
 +    public final ColumnIdentifier column;
 +    public final boolean isCollectionKeys;
 +
 +    private IndexTarget(ColumnIdentifier column, boolean isCollectionKeys)
 +    {
 +        this.column = column;
 +        this.isCollectionKeys = isCollectionKeys;
 +    }
 +
-     public static IndexTarget of(ColumnIdentifier c)
++    public static class Raw
 +    {
-         return new IndexTarget(c, false);
-     }
++        private final ColumnIdentifier.Raw column;
++        public final boolean isCollectionKeys;
 +
-     public static IndexTarget keysOf(ColumnIdentifier c)
-     {
-         return new IndexTarget(c, true);
++        private Raw(ColumnIdentifier.Raw column, boolean isCollectionKeys)
++        {
++            this.column = column;
++            this.isCollectionKeys = isCollectionKeys;
++        }
++
++        public static Raw of(ColumnIdentifier.Raw c)
++        {
++            return new Raw(c, false);
++        }
++
++        public static Raw keysOf(ColumnIdentifier.Raw c)
++        {
++            return new Raw(c, true);
++        }
++
++        public IndexTarget prepare(CFMetaData cfm)
++        {
++            return new IndexTarget(column.prepare(cfm), isCollectionKeys);
++        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a446e80e/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index 69984b6,c098c92..974ccc8
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@@ -22,7 -22,10 +22,8 @@@ import java.util.*
  
  import com.google.common.base.Function;
  import com.google.common.collect.Iterables;
+ import org.apache.cassandra.db.marshal.AbstractType;
  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;
@@@ -245,19 -272,20 +246,20 @@@ public abstract class ModificationState
              if (rel.onToken)
                  throw new InvalidRequestException(String.format("The token function cannot be used in WHERE clauses for UPDATE and DELETE statements: %s", relation));
  
-             ColumnDefinition def = cfm.getColumnDefinition(rel.getEntity());
+             ColumnIdentifier id = rel.getEntity().prepare(cfm);
 -            CFDefinition.Name name = cfDef.get(id);
 -            if (name == null)
++            ColumnDefinition def = cfm.getColumnDefinition(id);
 +            if (def == null)
-                 throw new InvalidRequestException(String.format("Unknown key identifier %s", rel.getEntity()));
+                 throw new InvalidRequestException(String.format("Unknown key identifier %s", id));
  
 -            switch (name.kind)
 +            switch (def.kind)
              {
 -                case KEY_ALIAS:
 -                case COLUMN_ALIAS:
 +                case PARTITION_KEY:
 +                case CLUSTERING_COLUMN:
                      Restriction restriction;
  
 -                    if (rel.operator() == Relation.Type.EQ)
 +                    if (rel.operator() == Operator.EQ)
                      {
 -                        Term t = rel.getValue().prepare(name);
 +                        Term t = rel.getValue().prepare(keyspace(), def);
                          t.collectMarkerSpecification(names);
                          restriction = new SingleColumnRestriction.EQ(t, false);
                      }
@@@ -746,21 -817,24 +748,22 @@@
                  }
                  else
                  {
-                     for (Pair<ColumnIdentifier, ColumnCondition.Raw> entry : conditions)
+                     for (Pair<ColumnIdentifier.Raw, ColumnCondition.Raw> entry : conditions)
                      {
-                         ColumnDefinition def = metadata.getColumnDefinition(entry.left);
 -                        ColumnIdentifier id = entry.left.prepare(cfDef.cfm);
 -                        CFDefinition.Name name = cfDef.get(id);
 -                        if (name == null)
++                        ColumnIdentifier id = entry.left.prepare(metadata);
++                        ColumnDefinition def = metadata.getColumnDefinition(id);
 +                        if (def == null)
-                             throw new InvalidRequestException(String.format("Unknown identifier %s", entry.left));
+                             throw new InvalidRequestException(String.format("Unknown identifier %s", id));
  
 -                        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 column '%s' cannot have IF conditions", entry.left));
+                                 throw new InvalidRequestException(String.format("PRIMARY KEY column '%s' cannot have IF conditions", id));
 -                            case VALUE_ALIAS:
 -                            case COLUMN_METADATA:
 -                            case STATIC:
 +                            default:
                                  stmt.addCondition(condition);
                                  break;
                          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a446e80e/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index aa01c93,77d94e3..2632ee2
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -98,13 -99,13 +98,13 @@@ public class SelectStatement implement
      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 Parameters defaultParameters = new Parameters(Collections.<ColumnIdentifier.Raw, 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();
          }
      };
  
@@@ -1295,25 -1245,84 +1295,25 @@@
  
          assert orderingIndexes != null;
  
 -        // optimization when only *one* order condition was given
 -        // 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().prepare(cfDef.cfm));
 -            Collections.sort(cqlRows.rows, new SingleColumnComparator(orderingIndexes.get(ordering), ordering.type));
 -            return;
 -        }
 -
 -        // builds a 'composite' type for multi-column comparison from the comparators of the ordering components
 -        // and passes collected position information and built composite comparator to CompositeComparator to do
 -        // an actual comparison of the CQL rows.
 -        List<AbstractType<?>> types = new ArrayList<AbstractType<?>>(parameters.orderings.size());
 -        int[] positions = new int[parameters.orderings.size()];
 +        List<Integer> idToSort = new ArrayList<Integer>();
 +        List<Comparator<ByteBuffer>> sorters = new ArrayList<Comparator<ByteBuffer>>();
  
-         for (ColumnIdentifier identifier : parameters.orderings.keySet())
 -        int idx = 0;
+         for (ColumnIdentifier.Raw identifier : parameters.orderings.keySet())
          {
-             ColumnDefinition orderingColumn = cfm.getColumnDefinition(identifier);
 -            CFDefinition.Name orderingColumn = cfDef.get(identifier.prepare(cfDef.cfm));
 -            types.add(orderingColumn.type);
 -            positions[idx++] = orderingIndexes.get(orderingColumn);
++            ColumnDefinition orderingColumn = cfm.getColumnDefinition(identifier.prepare(cfm));
 +            idToSort.add(orderingIndexes.get(orderingColumn.name));
 +            sorters.add(orderingColumn.type);
          }
  
 -        Collections.sort(cqlRows.rows, new CompositeComparator(types, positions));
 +        Comparator<List<ByteBuffer>> comparator = idToSort.size() == 1
 +                                                ? new SingleColumnComparator(idToSort.get(0), sorters.get(0))
 +                                                : new CompositeComparator(sorters, idToSort);
 +        Collections.sort(cqlRows.rows, comparator);
      }
  
 -    private void handleGroup(Selection.ResultSetBuilder result,
 -                             ByteBuffer[] keyComponents,
 -                             ColumnGroupMap columns,
 -                             ColumnGroupMap staticGroup) throws InvalidRequestException
 +    private static boolean isReversedType(ColumnDefinition def)
      {
 -        // Respect requested order
 -        result.newRow();
 -        for (CFDefinition.Name name : selection.getColumns())
 -        {
 -            switch (name.kind)
 -            {
 -                case KEY_ALIAS:
 -                    result.add(keyComponents[name.position]);
 -                    break;
 -                case COLUMN_ALIAS:
 -                    result.add(columns.getKeyComponent(name.position));
 -                    break;
 -                case VALUE_ALIAS:
 -                    // This should not happen for SPARSE
 -                    throw new AssertionError();
 -                case COLUMN_METADATA:
 -                    addValue(result, name, columns);
 -                    break;
 -                case STATIC:
 -                    addValue(result, name, staticGroup);
 -                    break;
 -            }
 -        }
 -    }
 -
 -    private static void addValue(Selection.ResultSetBuilder result, CFDefinition.Name name, ColumnGroupMap group)
 -    {
 -        if (group == null)
 -        {
 -            result.add((ByteBuffer)null);
 -            return;
 -        }
 -
 -        if (name.type.isCollection())
 -        {
 -            List<Pair<ByteBuffer, Column>> collection = group.getCollection(name.name.key);
 -            result.add(collection == null ? null : ((CollectionType)name.type).serialize(collection));
 -        }
 -        else
 -        {
 -            result.add(group.getSimple(name.name.key));
 -        }
 -    }
 -
 -    private static boolean isReversedType(CFDefinition.Name name)
 -    {
 -        return name.type instanceof ReversedType;
 +        return def.type instanceof ReversedType;
      }
  
      private boolean columnFilterIsIdentity()
@@@ -1424,28 -1411,29 +1424,29 @@@
                  if (relation.isMultiColumn())
                  {
                      MultiColumnRelation rel = (MultiColumnRelation) relation;
 -                    List<CFDefinition.Name> names = new ArrayList<>(rel.getEntities().size());
 +                    List<ColumnDefinition> names = new ArrayList<>(rel.getEntities().size());
-                     for (ColumnIdentifier entity : rel.getEntities())
+                     for (ColumnIdentifier.Raw rawEntity : rel.getEntities())
                      {
+                         ColumnIdentifier entity = rawEntity.prepare(cfm);
 -                        boolean[] queriable = processRelationEntity(stmt, relation, entity, cfDef);
 +                        ColumnDefinition def = cfm.getColumnDefinition(entity);
 +                        boolean[] queriable = processRelationEntity(stmt, indexManager, relation, entity, def);
                          hasQueriableIndex |= queriable[0];
                          hasQueriableClusteringColumnIndex |= queriable[1];
 -                        Name name = cfDef.get(entity);
 -                        names.add(name);
 -                        hasMultiColumnRelations |= Kind.COLUMN_ALIAS.equals(name.kind);
 +                        names.add(def);
 +                        hasMultiColumnRelations |= ColumnDefinition.Kind.CLUSTERING_COLUMN.equals(def.kind);
                      }
                      updateRestrictionsForRelation(stmt, names, rel, boundNames);
                  }
                  else
                  {
                      SingleColumnRelation rel = (SingleColumnRelation) relation;
-                     ColumnIdentifier entity = rel.getEntity();
+                     ColumnIdentifier entity = rel.getEntity().prepare(cfm);
 -                    boolean[] queriable = processRelationEntity(stmt, relation, entity, cfDef);
 +                    ColumnDefinition def = cfm.getColumnDefinition(entity);
 +                    boolean[] queriable = processRelationEntity(stmt, indexManager, relation, entity, def);
                      hasQueriableIndex |= queriable[0];
                      hasQueriableClusteringColumnIndex |= queriable[1];
 -                    Name name = cfDef.get(entity);
 -                    hasSingleColumnRelations |= Kind.COLUMN_ALIAS.equals(name.kind);
 -                    updateRestrictionsForRelation(stmt, name, rel, boundNames);
 +                    hasSingleColumnRelations |= ColumnDefinition.Kind.CLUSTERING_COLUMN.equals(def.kind);
 +                    updateRestrictionsForRelation(stmt, def, rel, boundNames);
                  }
              }
              if (hasSingleColumnRelations && hasMultiColumnRelations)
@@@ -1876,9 -1834,9 +1877,9 @@@
              for (Relation relation : whereClause)
              {
                  SingleColumnRelation singleColumnRelation = (SingleColumnRelation) relation;
-                 if (singleColumnRelation.onToken && !cfm.getColumnDefinition(singleColumnRelation.getEntity()).equals(iter.next()))
 -                if (singleColumnRelation.onToken && !cfDef.get(singleColumnRelation.getEntity().prepare(cfDef.cfm)).equals(iter.next()))
++                if (singleColumnRelation.onToken && !cfm.getColumnDefinition(singleColumnRelation.getEntity().prepare(cfm)).equals(iter.next()))
                      throw new InvalidRequestException(String.format("The token function arguments must be in the partition key order: %s",
 -                                                                    Joiner.on(',').join(cfDef.partitionKeys())));
 +                                                                    Joiner.on(',').join(cfm.partitionKeyColumns())));
              }
          }
  
@@@ -1973,46 -1931,71 +1974,47 @@@
          {
              verifyOrderingIsAllowed(stmt);
  
 -            // If we order an IN query, we'll have to do a manual sort post-query. Currently, this sorting requires that we
 -            // have queried the column on which we sort (TODO: we should update it to add the column on which we sort to the one
 -            // queried automatically, and then removing it from the resultSet afterwards if needed)
 +            // If we order post-query (see orderResults), the sorted column needs to be in the ResultSet for sorting, even if we don't
 +            // ultimately ship them to the client (CASSANDRA-4911).
              if (stmt.keyIsInRelation)
              {
 -                stmt.orderingIndexes = new HashMap<CFDefinition.Name, Integer>();
 +                stmt.orderingIndexes = new HashMap<>();
-                 for (ColumnIdentifier column : stmt.parameters.orderings.keySet())
+                 for (ColumnIdentifier.Raw rawColumn : stmt.parameters.orderings.keySet())
                  {
 -                    ColumnIdentifier column = rawColumn.prepare(cfDef.cfm);
 -                    final CFDefinition.Name name = cfDef.get(column);
 -                    if (name == null)
++                    ColumnIdentifier column = rawColumn.prepare(cfm);
 +                    final ColumnDefinition def = cfm.getColumnDefinition(column);
 +                    if (def == null)
                          handleUnrecognizedOrderingColumn(column);
  
 -                    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);
 -                            }
 -                        }));
 -                    }
 -                    else
 -                    {
 -                        boolean hasColumn = false;
 -                        for (int i = 0; i < selectClause.size(); i++)
 -                        {
 -                            RawSelector selector = selectClause.get(i);
 -                            if (name.name.equals(selector.selectable))
 -                            {
 -                                stmt.orderingIndexes.put(name, i);
 -                                hasColumn = true;
 -                                break;
 -                            }
 -                        }
 -
 -                        if (!hasColumn)
 -                            throw new InvalidRequestException("ORDER BY could not be used on columns missing in select clause.");
 -                    }
 +                    int index = indexOf(def, stmt.selection);
 +                    if (index < 0)
 +                        index = stmt.selection.addColumnForOrdering(def);
 +                    stmt.orderingIndexes.put(def.name, index);
                  }
              }
 -            stmt.isReversed = isReversed(stmt, cfDef);
 +            stmt.isReversed = isReversed(stmt, cfm);
          }
  
 -        private boolean isReversed(SelectStatement stmt, CFDefinition cfDef) throws InvalidRequestException
 +        private boolean isReversed(SelectStatement stmt, CFMetaData cfm) throws InvalidRequestException
          {
 -            Boolean[] reversedMap = new Boolean[cfDef.clusteringColumnsCount()];
 +            Boolean[] reversedMap = new Boolean[cfm.clusteringColumns().size()];
              int i = 0;
-             for (Map.Entry<ColumnIdentifier, Boolean> entry : stmt.parameters.orderings.entrySet())
+             for (Map.Entry<ColumnIdentifier.Raw, Boolean> entry : stmt.parameters.orderings.entrySet())
              {
-                 ColumnIdentifier column = entry.getKey();
 -                ColumnIdentifier column = entry.getKey().prepare(cfDef.cfm);
++                ColumnIdentifier column = entry.getKey().prepare(cfm);
                  boolean reversed = entry.getValue();
  
 -                CFDefinition.Name name = cfDef.get(column);
 -                if (name == null)
 +                ColumnDefinition def = cfm.getColumnDefinition(column);
 +                if (def == null)
                      handleUnrecognizedOrderingColumn(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
@@@ -2107,8 -2059,8 +2109,8 @@@
                  // We only call this when sliceRestriction != null, i.e. for compact table with non composite comparator,
                  // so it can't be a MultiColumnRelation.
                  SingleColumnRelation rel = (SingleColumnRelation)r;
-                 if (cfm.getColumnDefinition(rel.getEntity()).kind == ColumnDefinition.Kind.CLUSTERING_COLUMN
 -                if (cfDef.get(rel.getEntity().prepare(cfDef.cfm)).kind == CFDefinition.Name.Kind.COLUMN_ALIAS
 -                    && (rel.operator() == Relation.Type.GT || rel.operator() == Relation.Type.LT))
++                if (cfm.getColumnDefinition(rel.getEntity().prepare(cfm)).kind == ColumnDefinition.Kind.CLUSTERING_COLUMN
 +                    && (rel.operator() == Operator.GT || rel.operator() == Operator.LT))
                      return rel;
              }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a446e80e/src/java/org/apache/cassandra/cql3/statements/Selectable.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/Selectable.java
index 448301c,633bf71..b7e8094
--- a/src/java/org/apache/cassandra/cql3/statements/Selectable.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Selectable.java
@@@ -65,23 -90,25 +90,60 @@@ public interface Selectabl
              }
              return sb.append(")").toString();
          }
+ 
+         public static class Raw implements Selectable.Raw
+         {
+             private final String functionName;
+             private final List<Selectable.Raw> args;
+ 
+             public Raw(String functionName, List<Selectable.Raw> args)
+             {
+                 this.functionName = functionName;
+                 this.args = args;
+             }
+ 
+             public WithFunction prepare(CFMetaData cfm)
+             {
+                 List<Selectable> preparedArgs = new ArrayList<>(args.size());
+                 for (Selectable.Raw arg : args)
+                     preparedArgs.add(arg.prepare(cfm));
+                 return new WithFunction(functionName, preparedArgs);
+             }
+         }
      }
 +
 +    public static class WithFieldSelection implements Selectable
 +    {
 +        public final Selectable selected;
 +        public final ColumnIdentifier field;
 +
 +        public WithFieldSelection(Selectable selected, ColumnIdentifier field)
 +        {
 +            this.selected = selected;
 +            this.field = field;
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("%s.%s", selected, field);
 +        }
++
++        public static class Raw implements Selectable.Raw
++        {
++            private final Selectable.Raw selected;
++            private final ColumnIdentifier.Raw field;
++
++            public Raw(Selectable.Raw selected, ColumnIdentifier.Raw field)
++            {
++                this.selected = selected;
++                this.field = field;
++            }
++
++            public WithFieldSelection prepare(CFMetaData cfm)
++            {
++                return new WithFieldSelection(selected.prepare(cfm), field.prepare(cfm));
++            }
++        }
 +    }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a446e80e/src/java/org/apache/cassandra/cql3/statements/Selection.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/Selection.java
index de05f44,0135a76..82efeef
--- a/src/java/org/apache/cassandra/cql3/statements/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Selection.java
@@@ -107,64 -95,50 +107,72 @@@ public abstract class Selectio
          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)
 -        Selectable selectable = raw.selectable.prepare(cfDef.cfm);
 -        return makeSelector(cfDef, selectable, raw.alias, names, metadata);
++        Selectable selectable = raw.selectable.prepare(cfm);
++        return makeSelector(cfm, selectable, raw.alias, defs, metadata);
+     }
+ 
 -    private static Selector makeSelector(CFDefinition cfDef, Selectable selectable, ColumnIdentifier alias, List<CFDefinition.Name> names, List<ColumnSpecification> metadata) throws InvalidRequestException
++    private static Selector makeSelector(CFMetaData cfm, Selectable selectable, ColumnIdentifier alias, List<ColumnDefinition> defs, List<ColumnSpecification> metadata) throws InvalidRequestException
+     {
+         if (selectable instanceof ColumnIdentifier)
          {
-             ColumnDefinition def = cfm.getColumnDefinition((ColumnIdentifier)raw.selectable);
 -            CFDefinition.Name name = cfDef.get((ColumnIdentifier)selectable);
 -            if (name == null)
++            ColumnDefinition def = cfm.getColumnDefinition((ColumnIdentifier)selectable);
 +            if (def == null)
-                 throw new InvalidRequestException(String.format("Undefined name %s in selection clause", raw.selectable));
+                 throw new InvalidRequestException(String.format("Undefined name %s in selection clause", selectable));
++
              if (metadata != null)
-                 metadata.add(raw.alias == null ? def : makeAliasSpec(cfm, def.type, raw.alias));
 -                metadata.add(alias == null ? name : makeAliasSpec(cfDef, name.type, alias));
 -            return new SimpleSelector(name.toString(), addAndGetIndex(name, names), name.type);
++                metadata.add(alias == null ? def : makeAliasSpec(cfm, def.type, alias));
 +            return new SimpleSelector(def.name.toString(), addAndGetIndex(def, defs), def.type);
          }
-         else if (raw.selectable instanceof Selectable.WritetimeOrTTL)
+         else if (selectable instanceof Selectable.WritetimeOrTTL)
          {
-             Selectable.WritetimeOrTTL tot = (Selectable.WritetimeOrTTL)raw.selectable;
+             Selectable.WritetimeOrTTL tot = (Selectable.WritetimeOrTTL)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.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)
-                 metadata.add(makeWritetimeOrTTLSpec(cfm, tot, raw.alias));
 -                metadata.add(makeWritetimeOrTTLSpec(cfDef, tot, alias));
 -            return new WritetimeOrTTLSelector(name.toString(), addAndGetIndex(name, names), tot.isWritetime);
++                metadata.add(makeWritetimeOrTTLSpec(cfm, tot, alias));
 +            return new WritetimeOrTTLSelector(def.name.toString(), addAndGetIndex(def, defs), tot.isWritetime);
 +        }
-         else if (raw.selectable instanceof Selectable.WithFieldSelection)
++        else if (selectable instanceof Selectable.WithFieldSelection)
 +        {
-             Selectable.WithFieldSelection withField = (Selectable.WithFieldSelection)raw.selectable;
-             Selector selected = makeSelector(cfm, new RawSelector(withField.selected, null), defs, null);
++            Selectable.WithFieldSelection withField = (Selectable.WithFieldSelection)selectable;
++            Selector selected = makeSelector(cfm, withField.selected, null, defs, null);
 +            AbstractType<?> type = selected.getType();
 +            if (!(type instanceof UserType))
 +                throw new InvalidRequestException(String.format("Invalid field selection: %s of type %s is not a user type", withField.selected, type.asCQL3Type()));
 +
 +            UserType ut = (UserType)type;
 +            for (int i = 0; i < ut.size(); i++)
 +            {
 +                if (!ut.fieldName(i).equals(withField.field.bytes))
 +                    continue;
 +
 +                if (metadata != null)
-                     metadata.add(makeFieldSelectSpec(cfm, withField, ut.fieldType(i), raw.alias));
++                    metadata.add(makeFieldSelectSpec(cfm, withField, ut.fieldType(i), alias));
 +                return new FieldSelector(ut, i, selected);
 +            }
 +            throw new InvalidRequestException(String.format("%s of type %s has no field %s", withField.selected, type.asCQL3Type(), withField.field));
          }
          else
          {
-             Selectable.WithFunction withFun = (Selectable.WithFunction)raw.selectable;
+             Selectable.WithFunction withFun = (Selectable.WithFunction)selectable;
              List<Selector> args = new ArrayList<Selector>(withFun.args.size());
--            for (Selectable rawArg : withFun.args)
-                 args.add(makeSelector(cfm, new RawSelector(rawArg, null), defs, null));
 -                args.add(makeSelector(cfDef, rawArg, null, names, null));
++            for (Selectable arg : withFun.args)
++                args.add(makeSelector(cfm, arg, 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(cfm, withFun, returnType, raw.alias);
 -            ColumnSpecification spec = makeFunctionSpec(cfDef, withFun, returnType, alias);
 -            Function fun = Functions.get(withFun.functionName, args, spec);
++
++            ColumnSpecification spec = makeFunctionSpec(cfm, withFun, returnType, alias);
 +            Function fun = Functions.get(cfm.ksName, withFun.functionName, args, spec);
              if (metadata != null)
                  metadata.add(spec);
              return new FunctionSelector(fun, args);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a446e80e/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
index 8eeab71,022af26..b11173a
--- a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
@@@ -22,9 -22,8 +22,10 @@@ 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.composites.Composite;
+ import org.apache.cassandra.db.marshal.AbstractType;
  import org.apache.cassandra.exceptions.*;
  import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.utils.Pair;
@@@ -135,8 -138,8 +136,8 @@@ public class UpdateStatement extends Mo
  
              for (int i = 0; i < columnNames.size(); i++)
              {
-                 ColumnDefinition def = cfm.getColumnDefinition(columnNames.get(i));
 -                CFDefinition.Name name = cfDef.get(columnNames.get(i).prepare(cfDef.cfm));
 -                if (name == null)
++                ColumnDefinition def = cfm.getColumnDefinition(columnNames.get(i).prepare(cfm));
 +                if (def == null)
                      throw new InvalidRequestException(String.format("Unknown identifier %s", columnNames.get(i)));
  
                  for (int j = 0; j < i; j++)
@@@ -190,25 -195,27 +191,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(ModificationStatement.StatementType.UPDATE, cfDef.cfm, attrs);
 +            UpdateStatement stmt = new UpdateStatement(ModificationStatement.StatementType.UPDATE, boundNames.size(), cfm, attrs);
  
-             for (Pair<ColumnIdentifier, Operation.RawUpdate> entry : updates)
+             for (Pair<ColumnIdentifier.Raw, Operation.RawUpdate> entry : updates)
              {
-                 ColumnDefinition def = cfm.getColumnDefinition(entry.left);
 -                CFDefinition.Name name = cfDef.get(entry.left.prepare(cfDef.cfm));
 -                if (name == null)
++                ColumnDefinition def = cfm.getColumnDefinition(entry.left.prepare(cfm));
 +                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 VALUE_ALIAS:
 -                    case COLUMN_METADATA:
 -                    case STATIC:
 +                    default:
                          stmt.addOperation(operation);
                          break;
                  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a446e80e/test/unit/org/apache/cassandra/SchemaLoader.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/SchemaLoader.java
index 1f6ffa9,7dea52c..4a1c104
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@@ -343,12 -354,18 +343,12 @@@ public class SchemaLoade
          final Map<String, String> indexOptions = Collections.singletonMap(
                                                        SecondaryIndex.CUSTOM_INDEX_OPTION_NAME,
                                                        PerRowSecondaryIndexTest.TestIndex.class.getName());
 -        return standardCFMD(ksName, cfName)
 -                .keyValidator(AsciiType.instance)
 -                .columnMetadata(new HashMap<ByteBuffer, ColumnDefinition>()
 -                {{
 -                        ByteBuffer cName = ByteBuffer.wrap("indexed".getBytes(StandardCharsets.UTF_8));
 -                        put(cName, new ColumnDefinition(cName,
 -                                AsciiType.instance,
 -                                IndexType.CUSTOM,
 -                                indexOptions,
 -                                ByteBufferUtil.bytesToHex(cName),
 -                                null, ColumnDefinition.Type.REGULAR));
 -                }});
 +
-         CFMetaData cfm =  CFMetaData.sparseCFMetaData(ksName, cfName, BytesType.instance).keyValidator(AsciiType.instance);
++        CFMetaData cfm =  CFMetaData.sparseCFMetaData(ksName, cfName, AsciiType.instance).keyValidator(AsciiType.instance);
 +
 +        ByteBuffer cName = ByteBufferUtil.bytes("indexed");
 +        return cfm.addOrReplaceColumnDefinition(ColumnDefinition.regularDef(cfm, cName, AsciiType.instance, null)
 +                                                                .setIndex("indexe1", IndexType.CUSTOM, indexOptions));
      }
  
      private static void useCompression(List<KSMetaData> schema)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a446e80e/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
index 0000000,86d2dac..662800b
mode 000000,100644..100644
--- a/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
@@@ -1,0 -1,58 +1,57 @@@
+ /*
+  * 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 org.apache.cassandra.utils.ByteBufferUtil;
+ import org.junit.Test;
+ 
+ import org.apache.cassandra.SchemaLoader;
+ 
 -import static org.apache.cassandra.cql3.QueryProcessor.processInternal;
+ import static org.junit.Assert.assertEquals;
+ 
+ public class ThriftCompatibilityTest extends SchemaLoader
+ {
+     private static UntypedResultSet execute(String query) throws Throwable
+     {
+         try
+         {
 -            return processInternal(String.format(query));
++            return QueryProcessor.executeInternal(String.format(query));
+         }
+         catch (RuntimeException exc)
+         {
+             if (exc.getCause() != null)
+                 throw exc.getCause();
+             throw exc;
+         }
+     }
+ 
+     /** Test For CASSANDRA-8178 */
+     @Test
+     public void testNonTextComparator() throws Throwable
+     {
+         // the comparator is IntegerType, and there is a column named 42 with a UTF8Type validation type
+         execute("INSERT INTO \"Keyspace1\".\"JdbcInteger\" (key, \"42\") VALUES (0x00000001, 'abc')");
+         execute("UPDATE \"Keyspace1\".\"JdbcInteger\" SET \"42\" = 'abc' WHERE key = 0x00000001");
+         execute("DELETE \"42\" FROM \"Keyspace1\".\"JdbcInteger\" WHERE key = 0x00000000");
+         UntypedResultSet results = execute("SELECT key, \"42\" FROM \"Keyspace1\".\"JdbcInteger\"");
+         assertEquals(1, results.size());
+         UntypedResultSet.Row row = results.iterator().next();
+         assertEquals(ByteBufferUtil.bytes(1), row.getBytes("key"));
+         assertEquals("abc", row.getString("42"));
+     }
+ }