You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2016/06/15 07:39:02 UTC

[1/4] cassandra git commit: Allow terms in selection clauses

Repository: cassandra
Updated Branches:
  refs/heads/trunk 60e5e0ef4 -> 4ed00607d


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/test/unit/org/apache/cassandra/transport/SerDeserTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/transport/SerDeserTest.java b/test/unit/org/apache/cassandra/transport/SerDeserTest.java
index a85ce59..44d72a1 100644
--- a/test/unit/org/apache/cassandra/transport/SerDeserTest.java
+++ b/test/unit/org/apache/cassandra/transport/SerDeserTest.java
@@ -148,6 +148,11 @@ public class SerDeserTest
         return UTF8Type.instance.decompose(str);
     }
 
+    private static FieldIdentifier field(String field)
+    {
+        return FieldIdentifier.forQuoted(field);
+    }
+
     private static ColumnIdentifier ci(String name)
     {
         return new ColumnIdentifier(name, false);
@@ -175,6 +180,7 @@ public class SerDeserTest
         udtSerDeserTest(4);
     }
 
+
     public void udtSerDeserTest(int version) throws Exception
     {
         ListType<?> lt = ListType.getInstance(Int32Type.instance, true);
@@ -183,15 +189,15 @@ public class SerDeserTest
 
         UserType udt = new UserType("ks",
                                     bb("myType"),
-                                    Arrays.asList(bb("f1"), bb("f2"), bb("f3"), bb("f4")),
+                                    Arrays.asList(field("f1"), field("f2"), field("f3"), field("f4")),
                                     Arrays.asList(LongType.instance, lt, st, mt),
                                     true);
 
-        Map<ColumnIdentifier, Term.Raw> value = new HashMap<>();
-        value.put(ci("f1"), lit(42));
-        value.put(ci("f2"), new Lists.Literal(Arrays.<Term.Raw>asList(lit(3), lit(1))));
-        value.put(ci("f3"), new Sets.Literal(Arrays.<Term.Raw>asList(lit("foo"), lit("bar"))));
-        value.put(ci("f4"), new Maps.Literal(Arrays.<Pair<Term.Raw, Term.Raw>>asList(
+        Map<FieldIdentifier, Term.Raw> value = new HashMap<>();
+        value.put(field("f1"), lit(42));
+        value.put(field("f2"), new Lists.Literal(Arrays.<Term.Raw>asList(lit(3), lit(1))));
+        value.put(field("f3"), new Sets.Literal(Arrays.<Term.Raw>asList(lit("foo"), lit("bar"))));
+        value.put(field("f4"), new Maps.Literal(Arrays.<Pair<Term.Raw, Term.Raw>>asList(
                                    Pair.<Term.Raw, Term.Raw>create(lit("foo"), lit(24)),
                                    Pair.<Term.Raw, Term.Raw>create(lit("bar"), lit(12)))));
 


[2/4] cassandra git commit: Allow terms in selection clauses

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/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 7b329ac..26b25de 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
@@ -122,7 +122,7 @@ public class DeleteStatement extends ModificationStatement
                       Attributes.Raw attrs,
                       List<Operation.RawDeletion> deletions,
                       WhereClause whereClause,
-                      List<Pair<ColumnIdentifier.Raw, ColumnCondition.Raw>> conditions,
+                      List<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>> conditions,
                       boolean ifExists)
         {
             super(name, StatementType.DELETE, attrs, conditions, false, ifExists);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java b/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java
index 8cdf2c8..9756a4c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java
+++ b/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java
@@ -68,36 +68,36 @@ public class IndexTarget
 
     public static class Raw
     {
-        private final ColumnIdentifier.Raw column;
+        private final ColumnDefinition.Raw column;
         private final Type type;
 
-        private Raw(ColumnIdentifier.Raw column, Type type)
+        private Raw(ColumnDefinition.Raw column, Type type)
         {
             this.column = column;
             this.type = type;
         }
 
-        public static Raw simpleIndexOn(ColumnIdentifier.Raw c)
+        public static Raw simpleIndexOn(ColumnDefinition.Raw c)
         {
             return new Raw(c, Type.SIMPLE);
         }
 
-        public static Raw valuesOf(ColumnIdentifier.Raw c)
+        public static Raw valuesOf(ColumnDefinition.Raw c)
         {
             return new Raw(c, Type.VALUES);
         }
 
-        public static Raw keysOf(ColumnIdentifier.Raw c)
+        public static Raw keysOf(ColumnDefinition.Raw c)
         {
             return new Raw(c, Type.KEYS);
         }
 
-        public static Raw keysAndValuesOf(ColumnIdentifier.Raw c)
+        public static Raw keysAndValuesOf(ColumnDefinition.Raw c)
         {
             return new Raw(c, Type.KEYS_AND_VALUES);
         }
 
-        public static Raw fullCollection(ColumnIdentifier.Raw c)
+        public static Raw fullCollection(ColumnDefinition.Raw c)
         {
             return new Raw(c, Type.FULL);
         }
@@ -109,13 +109,9 @@ public class IndexTarget
             // same syntax as an index on a regular column (i.e. the 'values' in
             // 'CREATE INDEX on table(values(collection));' is optional). So we correct the target type
             // when the target column is a collection & the target type is SIMPLE.
-            ColumnIdentifier colId = column.prepare(cfm);
-            ColumnDefinition columnDef = cfm.getColumnDefinition(colId);
-            if (columnDef == null)
-                throw new InvalidRequestException("No column definition found for column " + colId);
-
+            ColumnDefinition columnDef = column.prepare(cfm);
             Type actualType = (type == Type.SIMPLE && columnDef.type.isCollection()) ? Type.VALUES : type;
-            return new IndexTarget(colId, actualType);
+            return new IndexTarget(columnDef.name, actualType);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/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 af89ba8..0a9be0c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -27,9 +27,9 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.ColumnDefinition.Raw;
 import org.apache.cassandra.config.ViewDefinition;
 import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.cql3.ColumnIdentifier.Raw;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
 import org.apache.cassandra.cql3.selection.Selection;
@@ -549,10 +549,10 @@ public abstract class ModificationStatement implements CQLStatement
 
         }
 
-        Selection.ResultSetBuilder builder = selection.resultSetBuilder(false);
+        Selection.ResultSetBuilder builder = selection.resultSetBuilder(options, false);
         SelectStatement.forSelection(cfm, selection).processPartition(partition, options, builder, FBUtilities.nowInSeconds());
 
-        return builder.build(options.getProtocolVersion());
+        return builder.build();
     }
 
     public ResultMessage executeInternal(QueryState queryState, QueryOptions options) throws RequestValidationException, RequestExecutionException
@@ -746,21 +746,21 @@ public abstract class ModificationStatement implements CQLStatement
     {
         protected final StatementType type;
         private final Attributes.Raw attrs;
-        private final List<Pair<ColumnIdentifier.Raw, ColumnCondition.Raw>> conditions;
+        private final List<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>> conditions;
         private final boolean ifNotExists;
         private final boolean ifExists;
 
         protected Parsed(CFName name,
                          StatementType type,
                          Attributes.Raw attrs,
-                         List<Pair<ColumnIdentifier.Raw, ColumnCondition.Raw>> conditions,
+                         List<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>> conditions,
                          boolean ifNotExists,
                          boolean ifExists)
         {
             super(name);
             this.type = type;
             this.attrs = attrs;
-            this.conditions = conditions == null ? Collections.<Pair<ColumnIdentifier.Raw, ColumnCondition.Raw>>emptyList() : conditions;
+            this.conditions = conditions == null ? Collections.<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>>emptyList() : conditions;
             this.ifNotExists = ifNotExists;
             this.ifExists = ifExists;
         }
@@ -832,16 +832,13 @@ public abstract class ModificationStatement implements CQLStatement
 
             ColumnConditions.Builder builder = ColumnConditions.newBuilder();
 
-            for (Pair<ColumnIdentifier.Raw, ColumnCondition.Raw> entry : conditions)
+            for (Pair<ColumnDefinition.Raw, ColumnCondition.Raw> entry : conditions)
             {
-                ColumnIdentifier id = entry.left.prepare(metadata);
-                ColumnDefinition def = metadata.getColumnDefinition(id);
-                checkNotNull(metadata.getColumnDefinition(id), "Unknown identifier %s in IF conditions", id);
-
+                ColumnDefinition def = entry.left.prepare(metadata);
                 ColumnCondition condition = entry.right.prepare(keyspace(), def, metadata);
                 condition.collectMarkerSpecification(boundNames);
 
-                checkFalse(def.isPrimaryKeyColumn(), "PRIMARY KEY column '%s' cannot have IF conditions", id);
+                checkFalse(def.isPrimaryKeyColumn(), "PRIMARY KEY column '%s' cannot have IF conditions", def.name);
                 builder.add(condition);
             }
             return builder.build();
@@ -884,8 +881,7 @@ public abstract class ModificationStatement implements CQLStatement
          */
         protected static ColumnDefinition getColumnDefinition(CFMetaData cfm, Raw rawId)
         {
-            ColumnIdentifier id = rawId.prepare(cfm);
-            return checkNotNull(cfm.getColumnDefinition(id), "Unknown identifier %s", id);
+            return rawId.prepare(cfm);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/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 a4bacda..5f37e5e 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -100,7 +100,7 @@ public class SelectStatement implements CQLStatement
     private final ColumnFilter queriedColumns;
 
     // Used by forSelection below
-    private static final Parameters defaultParameters = new Parameters(Collections.<ColumnIdentifier.Raw, Boolean>emptyMap(), false, false, false);
+    private static final Parameters defaultParameters = new Parameters(Collections.<ColumnDefinition.Raw, Boolean>emptyMap(), false, false, false);
 
     public SelectStatement(CFMetaData cfm,
                            int boundTerms,
@@ -386,7 +386,7 @@ public class SelectStatement implements CQLStatement
             ClientWarn.instance.warn("Aggregation query used on multiple partition keys (IN restriction)");
         }
 
-        Selection.ResultSetBuilder result = selection.resultSetBuilder(parameters.isJson);
+        Selection.ResultSetBuilder result = selection.resultSetBuilder(options, parameters.isJson);
         while (!pager.isExhausted())
         {
             try (PartitionIterator iter = pager.fetchPage(pageSize))
@@ -400,7 +400,7 @@ public class SelectStatement implements CQLStatement
                 }
             }
         }
-        return new ResultMessage.Rows(result.build(options.getProtocolVersion()));
+        return new ResultMessage.Rows(result.build());
     }
 
     private ResultMessage.Rows processResults(PartitionIterator partitions,
@@ -732,7 +732,7 @@ public class SelectStatement implements CQLStatement
                               int nowInSec,
                               int userLimit) throws InvalidRequestException
     {
-        Selection.ResultSetBuilder result = selection.resultSetBuilder(parameters.isJson);
+        Selection.ResultSetBuilder result = selection.resultSetBuilder(options, parameters.isJson);
         while (partitions.hasNext())
         {
             try (RowIterator partition = partitions.next())
@@ -741,7 +741,7 @@ public class SelectStatement implements CQLStatement
             }
         }
 
-        ResultSet cqlRows = result.build(options.getProtocolVersion());
+        ResultSet cqlRows = result.build();
 
         orderResults(cqlRows);
 
@@ -779,7 +779,7 @@ public class SelectStatement implements CQLStatement
         {
             if (!staticRow.isEmpty() && (!restrictions.hasClusteringColumnsRestriction() || cfm.isStaticCompactTable()))
             {
-                result.newRow(protocolVersion);
+                result.newRow();
                 for (ColumnDefinition def : selection.getColumns())
                 {
                     switch (def.kind)
@@ -801,7 +801,7 @@ public class SelectStatement implements CQLStatement
         while (partition.hasNext())
         {
             Row row = partition.next();
-            result.newRow(protocolVersion);
+            result.newRow();
             // Respect selection order
             for (ColumnDefinition def : selection.getColumns())
             {
@@ -894,7 +894,7 @@ public class SelectStatement implements CQLStatement
 
             Selection selection = selectClause.isEmpty()
                                   ? Selection.wildcard(cfm)
-                                  : Selection.fromSelectors(cfm, selectClause);
+                                  : Selection.fromSelectors(cfm, selectClause, boundNames);
 
             StatementRestrictions restrictions = prepareRestrictions(cfm, boundNames, selection, forView);
 
@@ -949,23 +949,14 @@ public class SelectStatement implements CQLStatement
                                                           Selection selection,
                                                           boolean forView) throws InvalidRequestException
         {
-            try
-            {
-                return new StatementRestrictions(StatementType.SELECT,
-                                                 cfm,
-                                                 whereClause,
-                                                 boundNames,
-                                                 selection.containsOnlyStaticColumns(),
-                                                 selection.containsAComplexColumn(),
-                                                 parameters.allowFiltering,
-                                                 forView);
-            }
-            catch (UnrecognizedEntityException e)
-            {
-                if (containsAlias(e.entity))
-                    throw invalidRequest("Aliases aren't allowed in the where clause ('%s')", e.relation);
-                throw e;
-            }
+            return new StatementRestrictions(StatementType.SELECT,
+                                             cfm,
+                                             whereClause,
+                                             boundNames,
+                                             selection.containsOnlyStaticColumns(),
+                                             selection.containsAComplexColumn(),
+                                             parameters.allowFiltering,
+                                             forView);
         }
 
         /** Returns a Term for the limit or null if no limit is set */
@@ -1011,12 +1002,6 @@ public class SelectStatement implements CQLStatement
                           "SELECT DISTINCT queries must request all the partition key columns (missing %s)", def.name);
         }
 
-        private void handleUnrecognizedOrderingColumn(ColumnIdentifier column) throws InvalidRequestException
-        {
-            checkFalse(containsAlias(column), "Aliases are not allowed in order by clause ('%s')", column);
-            checkFalse(true, "Order by on unknown column %s", column);
-        }
-
         private Comparator<List<ByteBuffer>> getOrderingComparator(CFMetaData cfm,
                                                                    Selection selection,
                                                                    StatementRestrictions restrictions)
@@ -1030,10 +1015,9 @@ public class SelectStatement implements CQLStatement
             List<Integer> idToSort = new ArrayList<Integer>();
             List<Comparator<ByteBuffer>> sorters = new ArrayList<Comparator<ByteBuffer>>();
 
-            for (ColumnIdentifier.Raw raw : parameters.orderings.keySet())
+            for (ColumnDefinition.Raw raw : parameters.orderings.keySet())
             {
-                ColumnIdentifier identifier = raw.prepare(cfm);
-                ColumnDefinition orderingColumn = cfm.getColumnDefinition(identifier);
+                ColumnDefinition orderingColumn = raw.prepare(cfm);
                 idToSort.add(orderingIndexes.get(orderingColumn.name));
                 sorters.add(orderingColumn.type);
             }
@@ -1048,12 +1032,9 @@ public class SelectStatement implements CQLStatement
             // even if we don't
             // ultimately ship them to the client (CASSANDRA-4911).
             Map<ColumnIdentifier, Integer> orderingIndexes = new HashMap<>();
-            for (ColumnIdentifier.Raw raw : parameters.orderings.keySet())
+            for (ColumnDefinition.Raw raw : parameters.orderings.keySet())
             {
-                ColumnIdentifier column = raw.prepare(cfm);
-                final ColumnDefinition def = cfm.getColumnDefinition(column);
-                if (def == null)
-                    handleUnrecognizedOrderingColumn(column);
+                final ColumnDefinition def = raw.prepare(cfm);
                 int index = selection.getResultSetIndex(def);
                 if (index < 0)
                     index = selection.addColumnForOrdering(def);
@@ -1066,17 +1047,13 @@ public class SelectStatement implements CQLStatement
         {
             Boolean[] reversedMap = new Boolean[cfm.clusteringColumns().size()];
             int i = 0;
-            for (Map.Entry<ColumnIdentifier.Raw, Boolean> entry : parameters.orderings.entrySet())
+            for (Map.Entry<ColumnDefinition.Raw, Boolean> entry : parameters.orderings.entrySet())
             {
-                ColumnIdentifier column = entry.getKey().prepare(cfm);
+                ColumnDefinition def = entry.getKey().prepare(cfm);
                 boolean reversed = entry.getValue();
 
-                ColumnDefinition def = cfm.getColumnDefinition(column);
-                if (def == null)
-                    handleUnrecognizedOrderingColumn(column);
-
                 checkTrue(def.isClusteringColumn(),
-                          "Order by is currently only supported on the clustered columns of the PRIMARY KEY, got %s", column);
+                          "Order by is currently only supported on the clustered columns of the PRIMARY KEY, got %s", def.name);
 
                 checkTrue(i++ == def.position(),
                           "Order by currently only support the ordering of columns following their declared order in the PRIMARY KEY");
@@ -1116,17 +1093,6 @@ public class SelectStatement implements CQLStatement
             }
         }
 
-        private boolean containsAlias(final ColumnIdentifier name)
-        {
-            return Iterables.any(selectClause, new Predicate<RawSelector>()
-                                               {
-                                                   public boolean apply(RawSelector raw)
-                                                   {
-                                                       return name.equals(raw.alias);
-                                                   }
-                                               });
-        }
-
         private ColumnSpecification limitReceiver()
         {
             return new ColumnSpecification(keyspace(), columnFamily(), new ColumnIdentifier("[limit]", true), Int32Type.instance);
@@ -1152,12 +1118,12 @@ public class SelectStatement implements CQLStatement
     public static class Parameters
     {
         // Public because CASSANDRA-9858
-        public final Map<ColumnIdentifier.Raw, Boolean> orderings;
+        public final Map<ColumnDefinition.Raw, Boolean> orderings;
         public final boolean isDistinct;
         public final boolean allowFiltering;
         public final boolean isJson;
 
-        public Parameters(Map<ColumnIdentifier.Raw, Boolean> orderings,
+        public Parameters(Map<ColumnDefinition.Raw, Boolean> orderings,
                           boolean isDistinct,
                           boolean allowFiltering,
                           boolean isJson)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/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 4675731..3657f94 100644
--- a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
@@ -113,7 +113,7 @@ public class UpdateStatement extends ModificationStatement
 
     public static class ParsedInsert extends ModificationStatement.Parsed
     {
-        private final List<ColumnIdentifier.Raw> columnNames;
+        private final List<ColumnDefinition.Raw> columnNames;
         private final List<Term.Raw> columnValues;
 
         /**
@@ -127,7 +127,7 @@ public class UpdateStatement extends ModificationStatement
          */
         public ParsedInsert(CFName name,
                             Attributes.Raw attrs,
-                            List<ColumnIdentifier.Raw> columnNames,
+                            List<ColumnDefinition.Raw> columnNames,
                             List<Term.Raw> columnValues,
                             boolean ifNotExists)
         {
@@ -233,9 +233,7 @@ public class UpdateStatement extends ModificationStatement
                 Term.Raw raw = prepared.getRawTermForColumn(def);
                 if (def.isPrimaryKeyColumn())
                 {
-                    whereClause.add(new SingleColumnRelation(new ColumnIdentifier.ColumnIdentifierValue(def.name),
-                                                             Operator.EQ,
-                                                             raw));
+                    whereClause.add(new SingleColumnRelation(ColumnDefinition.Raw.forColumn(def), Operator.EQ, raw));
                 }
                 else
                 {
@@ -269,7 +267,7 @@ public class UpdateStatement extends ModificationStatement
     public static class ParsedUpdate extends ModificationStatement.Parsed
     {
         // Provided for an UPDATE
-        private final List<Pair<ColumnIdentifier.Raw, Operation.RawUpdate>> updates;
+        private final List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> updates;
         private final WhereClause whereClause;
 
         /**
@@ -284,9 +282,9 @@ public class UpdateStatement extends ModificationStatement
          * */
         public ParsedUpdate(CFName name,
                             Attributes.Raw attrs,
-                            List<Pair<ColumnIdentifier.Raw, Operation.RawUpdate>> updates,
+                            List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> updates,
                             WhereClause whereClause,
-                            List<Pair<ColumnIdentifier.Raw, ColumnCondition.Raw>> conditions,
+                            List<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>> conditions,
                             boolean ifExists)
         {
             super(name, StatementType.UPDATE, attrs, conditions, false, ifExists);
@@ -302,7 +300,7 @@ public class UpdateStatement extends ModificationStatement
         {
             Operations operations = new Operations(type);
 
-            for (Pair<ColumnIdentifier.Raw, Operation.RawUpdate> entry : updates)
+            for (Pair<ColumnDefinition.Raw, Operation.RawUpdate> entry : updates)
             {
                 ColumnDefinition def = getColumnDefinition(cfm, entry.left);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/db/marshal/AbstractType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractType.java b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
index 67717bb..2b5503b 100644
--- a/src/java/org/apache/cassandra/db/marshal/AbstractType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
@@ -30,7 +30,9 @@ import java.util.Map;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.cql3.AssignmentTestable;
 import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.exceptions.SyntaxException;
@@ -54,7 +56,7 @@ import static org.apache.cassandra.db.marshal.AbstractType.ComparisonType.CUSTOM
  * represent a valid ByteBuffer for the type being compared.
  */
 @Unmetered
-public abstract class AbstractType<T> implements Comparator<ByteBuffer>
+public abstract class AbstractType<T> implements Comparator<ByteBuffer>, AssignmentTestable
 {
     private static final Logger logger = LoggerFactory.getLogger(AbstractType.class);
 
@@ -476,4 +478,24 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>
                 throw new IllegalArgumentException(this + " cannot be used in comparisons, so cannot be used as a clustering column");
         }
     }
+
+    public final AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
+    {
+        // We should ignore the fact that the output type is frozen in our comparison as functions do not support
+        // frozen types for arguments
+        AbstractType<?> receiverType = receiver.type;
+        if (isFreezable() && !isMultiCell())
+            receiverType = receiverType.freeze();
+
+        if (isReversed())
+            receiverType = ReversedType.getInstance(receiverType);
+
+        if (equals(receiverType))
+            return AssignmentTestable.TestResult.EXACT_MATCH;
+
+        if (receiverType.isValueCompatibleWith(this))
+            return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
+
+        return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/db/marshal/TypeParser.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/TypeParser.java b/src/java/org/apache/cassandra/db/marshal/TypeParser.java
index ba8ad13..78af800 100644
--- a/src/java/org/apache/cassandra/db/marshal/TypeParser.java
+++ b/src/java/org/apache/cassandra/db/marshal/TypeParser.java
@@ -23,6 +23,7 @@ import java.lang.reflect.Method;
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import org.apache.cassandra.cql3.FieldIdentifier;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
@@ -538,16 +539,16 @@ public class TypeParser
         return sb.toString();
     }
 
-    public static String stringifyUserTypeParameters(String keysace, ByteBuffer typeName, List<ByteBuffer> columnNames,
+    public static String stringifyUserTypeParameters(String keysace, ByteBuffer typeName, List<FieldIdentifier> fields,
                                                      List<AbstractType<?>> columnTypes, boolean ignoreFreezing)
     {
         StringBuilder sb = new StringBuilder();
         sb.append('(').append(keysace).append(",").append(ByteBufferUtil.bytesToHex(typeName));
 
-        for (int i = 0; i < columnNames.size(); i++)
+        for (int i = 0; i < fields.size(); i++)
         {
             sb.append(',');
-            sb.append(ByteBufferUtil.bytesToHex(columnNames.get(i))).append(":");
+            sb.append(ByteBufferUtil.bytesToHex(fields.get(i).bytes)).append(":");
             sb.append(columnTypes.get(i).toString(ignoreFreezing));
         }
         sb.append(')');

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/db/marshal/UserType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/UserType.java b/src/java/org/apache/cassandra/db/marshal/UserType.java
index d2cf94b..7803ee2 100644
--- a/src/java/org/apache/cassandra/db/marshal/UserType.java
+++ b/src/java/org/apache/cassandra/db/marshal/UserType.java
@@ -47,11 +47,11 @@ public class UserType extends TupleType
 
     public final String keyspace;
     public final ByteBuffer name;
-    private final List<ByteBuffer> fieldNames;
+    private final List<FieldIdentifier> fieldNames;
     private final List<String> stringFieldNames;
     private final boolean isMultiCell;
 
-    public UserType(String keyspace, ByteBuffer name, List<ByteBuffer> fieldNames, List<AbstractType<?>> fieldTypes, boolean isMultiCell)
+    public UserType(String keyspace, ByteBuffer name, List<FieldIdentifier> fieldNames, List<AbstractType<?>> fieldTypes, boolean isMultiCell)
     {
         super(fieldTypes, false);
         assert fieldNames.size() == fieldTypes.size();
@@ -61,17 +61,8 @@ public class UserType extends TupleType
         this.stringFieldNames = new ArrayList<>(fieldNames.size());
         this.isMultiCell = isMultiCell;
 
-        for (ByteBuffer fieldName : fieldNames)
-        {
-            try
-            {
-                stringFieldNames.add(ByteBufferUtil.string(fieldName, StandardCharsets.UTF_8));
-            }
-            catch (CharacterCodingException ex)
-            {
-                throw new AssertionError("Got non-UTF8 field name for user-defined type: " + ByteBufferUtil.bytesToHex(fieldName), ex);
-            }
-        }
+        for (FieldIdentifier fieldName : fieldNames)
+            stringFieldNames.add(fieldName.toString());
     }
 
     public static UserType getInstance(TypeParser parser) throws ConfigurationException, SyntaxException
@@ -79,11 +70,11 @@ public class UserType extends TupleType
         Pair<Pair<String, ByteBuffer>, List<Pair<ByteBuffer, AbstractType>>> params = parser.getUserTypeParameters();
         String keyspace = params.left.left;
         ByteBuffer name = params.left.right;
-        List<ByteBuffer> columnNames = new ArrayList<>(params.right.size());
+        List<FieldIdentifier> columnNames = new ArrayList<>(params.right.size());
         List<AbstractType<?>> columnTypes = new ArrayList<>(params.right.size());
         for (Pair<ByteBuffer, AbstractType> p : params.right)
         {
-            columnNames.add(p.left);
+            columnNames.add(new FieldIdentifier(p.left));
             columnTypes.add(p.right);
         }
 
@@ -118,7 +109,7 @@ public class UserType extends TupleType
         return types;
     }
 
-    public ByteBuffer fieldName(int i)
+    public FieldIdentifier fieldName(int i)
     {
         return fieldNames.get(i);
     }
@@ -128,7 +119,7 @@ public class UserType extends TupleType
         return stringFieldNames.get(i);
     }
 
-    public List<ByteBuffer> fieldNames()
+    public List<FieldIdentifier> fieldNames()
     {
         return fieldNames;
     }
@@ -138,23 +129,15 @@ public class UserType extends TupleType
         return UTF8Type.instance.compose(name);
     }
 
-    public short fieldPosition(ColumnIdentifier field)
-    {
-        return fieldPosition(field.bytes);
-    }
-
-    public short fieldPosition(ByteBuffer fieldName)
+    public int fieldPosition(FieldIdentifier fieldName)
     {
-        for (short i = 0; i < fieldNames.size(); i++)
-            if (fieldName.equals(fieldNames.get(i)))
-                return i;
-        return -1;
+        return fieldNames.indexOf(fieldName);
     }
 
-    public CellPath cellPathForField(ByteBuffer fieldName)
+    public CellPath cellPathForField(FieldIdentifier fieldName)
     {
         // we use the field position instead of the field name to allow for field renaming in ALTER TYPE statements
-        return CellPath.create(ByteBufferUtil.bytes(fieldPosition(fieldName)));
+        return CellPath.create(ByteBufferUtil.bytes((short)fieldPosition(fieldName)));
     }
 
     public ShortType nameComparator()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/db/view/View.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/View.java b/src/java/org/apache/cassandra/db/view/View.java
index 845a6ab..771c242 100644
--- a/src/java/org/apache/cassandra/db/view/View.java
+++ b/src/java/org/apache/cassandra/db/view/View.java
@@ -262,12 +262,12 @@ public class View
             if (rel.isMultiColumn())
             {
                 sb.append(((MultiColumnRelation) rel).getEntities().stream()
-                        .map(ColumnIdentifier.Raw::toCQLString)
+                        .map(ColumnDefinition.Raw::toString)
                         .collect(Collectors.joining(", ", "(", ")")));
             }
             else
             {
-                sb.append(((SingleColumnRelation) rel).getEntity().toCQLString());
+                sb.append(((SingleColumnRelation) rel).getEntity());
             }
 
             sb.append(" ").append(rel.operator()).append(" ");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/exceptions/UnrecognizedEntityException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/UnrecognizedEntityException.java b/src/java/org/apache/cassandra/exceptions/UnrecognizedEntityException.java
deleted file mode 100644
index e8392e9..0000000
--- a/src/java/org/apache/cassandra/exceptions/UnrecognizedEntityException.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.exceptions;
-
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.cql3.Relation;
-
-/**
- * Exception thrown when an entity is not recognized within a relation.
- */
-public final class UnrecognizedEntityException extends InvalidRequestException
-{
-    /**
-     * The unrecognized entity.
-     */
-    public final ColumnIdentifier entity;
-
-    /**
-     * The entity relation.
-     */
-    public final Relation relation;
-
-    /**
-     * Creates a new <code>UnrecognizedEntityException</code>.
-     * @param entity the unrecognized entity
-     * @param relation the entity relation
-     */
-    public UnrecognizedEntityException(ColumnIdentifier entity, Relation relation)
-    {
-        super(String.format("Undefined name %s in where clause ('%s')", entity, relation));
-        this.entity = entity;
-        this.relation = relation;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java b/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java
index 0503080..68f907e 100644
--- a/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java
+++ b/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java
@@ -28,6 +28,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.*;
 import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.FieldIdentifier;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.cql3.functions.FunctionName;
@@ -823,10 +824,10 @@ public final class LegacySchemaMigrator
                               SystemKeyspace.LEGACY_USERTYPES);
         UntypedResultSet.Row row = query(query, keyspaceName, typeName).one();
 
-        List<ByteBuffer> names =
+        List<FieldIdentifier> names =
             row.getList("field_names", UTF8Type.instance)
                .stream()
-               .map(ByteBufferUtil::bytes)
+               .map(t -> FieldIdentifier.forInternalString(t))
                .collect(Collectors.toList());
 
         List<AbstractType<?>> types =

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
index ed6e76f..4dc273a 100644
--- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
+++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
@@ -433,7 +433,7 @@ public final class SchemaKeyspace
     {
         RowUpdateBuilder adder = new RowUpdateBuilder(Types, timestamp, mutation)
                                  .clustering(type.getNameAsString())
-                                 .frozenList("field_names", type.fieldNames().stream().map(SchemaKeyspace::bbToString).collect(toList()))
+                                 .frozenList("field_names", type.fieldNames().stream().map(FieldIdentifier::toString).collect(toList()))
                                  .frozenList("field_types", type.fieldTypes().stream().map(AbstractType::asCQL3Type).map(CQL3Type::toString).collect(toList()));
 
         adder.build();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/schema/Types.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/Types.java b/src/java/org/apache/cassandra/schema/Types.java
index 00801b5..25efd70 100644
--- a/src/java/org/apache/cassandra/schema/Types.java
+++ b/src/java/org/apache/cassandra/schema/Types.java
@@ -24,6 +24,7 @@ import javax.annotation.Nullable;
 
 import com.google.common.collect.*;
 
+import org.apache.cassandra.cql3.FieldIdentifier;
 import org.apache.cassandra.cql3.CQL3Type;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.UserType;
@@ -303,9 +304,9 @@ public final class Types implements Iterable<UserType>
 
             UserType prepare(String keyspace, Types types)
             {
-                List<ByteBuffer> preparedFieldNames =
+                List<FieldIdentifier> preparedFieldNames =
                     fieldNames.stream()
-                              .map(ByteBufferUtil::bytes)
+                              .map(t -> FieldIdentifier.forInternalString(t))
                               .collect(toList());
 
                 List<AbstractType<?>> preparedFieldTypes =

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/transport/DataType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/DataType.java b/src/java/org/apache/cassandra/transport/DataType.java
index 7abcba7..eb1f1f4 100644
--- a/src/java/org/apache/cassandra/transport/DataType.java
+++ b/src/java/org/apache/cassandra/transport/DataType.java
@@ -28,8 +28,9 @@ import com.google.common.annotations.VisibleForTesting;
 
 import io.netty.buffer.ByteBuf;
 
-import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.cql3.FieldIdentifier;
 import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.utils.Pair;
 
 public enum DataType implements OptionCodec.Codecable<DataType>
@@ -111,11 +112,11 @@ public enum DataType implements OptionCodec.Codecable<DataType>
                 String ks = CBUtil.readString(cb);
                 ByteBuffer name = UTF8Type.instance.decompose(CBUtil.readString(cb));
                 int n = cb.readUnsignedShort();
-                List<ByteBuffer> fieldNames = new ArrayList<>(n);
+                List<FieldIdentifier> fieldNames = new ArrayList<>(n);
                 List<AbstractType<?>> fieldTypes = new ArrayList<>(n);
                 for (int i = 0; i < n; i++)
                 {
-                    fieldNames.add(UTF8Type.instance.decompose(CBUtil.readString(cb)));
+                    fieldNames.add(FieldIdentifier.forInternalString(CBUtil.readString(cb)));
                     fieldTypes.add(DataType.toType(codec.decodeOne(cb, version)));
                 }
                 return new UserType(ks, name, fieldNames, fieldTypes, true);
@@ -163,7 +164,7 @@ public enum DataType implements OptionCodec.Codecable<DataType>
                 cb.writeShort(udt.size());
                 for (int i = 0; i < udt.size(); i++)
                 {
-                    CBUtil.writeString(UTF8Type.instance.compose(udt.fieldName(i)), cb);
+                    CBUtil.writeString(udt.fieldName(i).toString(), cb);
                     codec.writeOne(DataType.fromType(udt.fieldType(i), version), cb, version);
                 }
                 break;
@@ -203,7 +204,7 @@ public enum DataType implements OptionCodec.Codecable<DataType>
                 size += 2;
                 for (int i = 0; i < udt.size(); i++)
                 {
-                    size += CBUtil.sizeOfString(UTF8Type.instance.compose(udt.fieldName(i)));
+                    size += CBUtil.sizeOfString(udt.fieldName(i).toString());
                     size += codec.oneSerializedSize(DataType.fromType(udt.fieldType(i), version), version);
                 }
                 return size;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/test/unit/org/apache/cassandra/cql3/CQL3TypeLiteralTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CQL3TypeLiteralTest.java b/test/unit/org/apache/cassandra/cql3/CQL3TypeLiteralTest.java
index 73e0fca..43dc267 100644
--- a/test/unit/org/apache/cassandra/cql3/CQL3TypeLiteralTest.java
+++ b/test/unit/org/apache/cassandra/cql3/CQL3TypeLiteralTest.java
@@ -629,11 +629,11 @@ public class CQL3TypeLiteralTest
     static UserType randomUserType(int level)
     {
         int typeCount = 2 + randInt(5);
-        List<ByteBuffer> names = new ArrayList<>();
+        List<FieldIdentifier> names = new ArrayList<>();
         List<AbstractType<?>> types = new ArrayList<>();
         for (int i = 0; i < typeCount; i++)
         {
-            names.add(UTF8Type.instance.fromString('f' + randLetters(i)));
+            names.add(FieldIdentifier.forQuoted('f' + randLetters(i)));
             types.add(randomNestedType(level));
         }
         return new UserType("ks", UTF8Type.instance.fromString("u" + randInt(1000000)), names, types, true);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index 1e7d05f..bca9e7b 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -676,6 +676,11 @@ public abstract class CQLTester
         return currentTable == null ? query : String.format(query, KEYSPACE + "." + currentTable);
     }
 
+    protected ResultMessage.Prepared prepare(String query) throws Throwable
+    {
+        return QueryProcessor.prepare(formatQuery(query), ClientState.forInternalCalls(), false);
+    }
+
     protected UntypedResultSet execute(String query, Object... values) throws Throwable
     {
         query = formatQuery(query);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/test/unit/org/apache/cassandra/cql3/selection/TermSelectionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/selection/TermSelectionTest.java b/test/unit/org/apache/cassandra/cql3/selection/TermSelectionTest.java
new file mode 100644
index 0000000..065fdbd
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/selection/TermSelectionTest.java
@@ -0,0 +1,338 @@
+/*
+ * 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.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.*;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.transport.messages.ResultMessage;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class TermSelectionTest extends CQLTester
+{
+    // Helper method for testSelectLiteral()
+    private void assertConstantResult(UntypedResultSet result, Object constant)
+    {
+        assertRows(result,
+                   row(1, "one", constant),
+                   row(2, "two", constant),
+                   row(3, "three", constant));
+    }
+
+    @Test
+    public void testSelectLiteral() throws Throwable
+    {
+        createTable("CREATE TABLE %s (pk int, ck int, t text, PRIMARY KEY (pk, ck) )");
+        execute("INSERT INTO %s (pk, ck, t) VALUES (1, 1, 'one')");
+        execute("INSERT INTO %s (pk, ck, t) VALUES (1, 2, 'two')");
+        execute("INSERT INTO %s (pk, ck, t) VALUES (1, 3, 'three')");
+
+        assertInvalidMessage("Cannot infer type for term", "SELECT ck, t, 'a const' FROM %s");
+        assertConstantResult(execute("SELECT ck, t, (text)'a const' FROM %s"), "a const");
+
+        assertInvalidMessage("Cannot infer type for term", "SELECT ck, t, 42 FROM %s");
+        assertConstantResult(execute("SELECT ck, t, (int)42 FROM %s"), 42);
+
+        assertInvalidMessage("Cannot infer type for term", "SELECT ck, t, (1, 'foo') FROM %s");
+        assertConstantResult(execute("SELECT ck, t, (tuple<int, text>)(1, 'foo') FROM %s"), tuple(1, "foo"));
+
+        assertInvalidMessage("Cannot infer type for term", "SELECT ck, t, [1, 2, 3] FROM %s");
+        assertConstantResult(execute("SELECT ck, t, (list<int>)[1, 2, 3] FROM %s"), list(1, 2, 3));
+
+        assertInvalidMessage("Cannot infer type for term", "SELECT ck, t, {1, 2, 3} FROM %s");
+        assertConstantResult(execute("SELECT ck, t, (set<int>){1, 2, 3} FROM %s"), set(1, 2, 3));
+
+        assertInvalidMessage("Cannot infer type for term", "SELECT ck, t, {1: 'foo', 2: 'bar', 3: 'baz'} FROM %s");
+        assertConstantResult(execute("SELECT ck, t, (map<int, text>){1: 'foo', 2: 'bar', 3: 'baz'} FROM %s"), map(1, "foo", 2, "bar", 3, "baz"));
+
+        assertColumnNames(execute("SELECT ck, t, (int)42, (int)43 FROM %s"), "ck", "t", "(int)42", "(int)43");
+        assertRows(execute("SELECT ck, t, (int) 42, (int) 43 FROM %s"),
+                   row(1, "one", 42, 43),
+                   row(2, "two", 42, 43),
+                   row(3, "three", 42, 43));
+    }
+
+    @Test
+    public void testSelectUDTLiteral() throws Throwable
+    {
+        String type = createType("CREATE TYPE %s(a int, b text)");
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v " + type + ")");
+
+        execute("INSERT INTO %s(k, v) VALUES (?, ?)", 0, userType("a", 3, "b", "foo"));
+
+        assertInvalidMessage("Cannot infer type for term", "SELECT k, v, { a: 4, b: 'bar'} FROM %s");
+
+        assertRows(execute("SELECT k, v, (" + type + "){ a: 4, b: 'bar'} FROM %s"),
+            row(0, userType("a", 3, "b", "foo"), userType("a", 4, "b", "bar"))
+        );
+    }
+
+    @Test
+    public void testInvalidSelect() throws Throwable
+    {
+        // Creates a table just so we can reference it in the (invalid) SELECT below
+        createTable("CREATE TABLE %s (k int PRIMARY KEY)");
+
+        assertInvalidMessage("Cannot infer type for term", "SELECT ? FROM %s");
+        assertInvalidMessage("Cannot infer type for term", "SELECT k, ? FROM %s");
+
+        assertInvalidMessage("Cannot infer type for term", "SELECT k, null FROM %s");
+    }
+
+    private void assertColumnSpec(ColumnSpecification spec, String expectedName, AbstractType<?> expectedType)
+    {
+        assertEquals(expectedName, spec.name.toString());
+        assertEquals(expectedType, spec.type);
+    }
+
+    @Test
+    public void testSelectPrepared() throws Throwable
+    {
+        createTable("CREATE TABLE %s (pk int, ck int, t text, PRIMARY KEY (pk, ck) )");
+        execute("INSERT INTO %s (pk, ck, t) VALUES (1, 1, 'one')");
+        execute("INSERT INTO %s (pk, ck, t) VALUES (1, 2, 'two')");
+        execute("INSERT INTO %s (pk, ck, t) VALUES (1, 3, 'three')");
+
+        String query = "SELECT (int)?, (decimal):adecimal, (text)?, (tuple<int,text>):atuple, pk, ck, t FROM %s WHERE pk = ?";
+        ResultMessage.Prepared prepared = prepare(query);
+
+        List<ColumnSpecification> boundNames = prepared.metadata.names;
+
+        // 5 bound variables
+        assertEquals(5, boundNames.size());
+        assertColumnSpec(boundNames.get(0), "[selection]", Int32Type.instance);
+        assertColumnSpec(boundNames.get(1), "adecimal", DecimalType.instance);
+        assertColumnSpec(boundNames.get(2), "[selection]", UTF8Type.instance);
+        assertColumnSpec(boundNames.get(3), "atuple", TypeParser.parse("TupleType(Int32Type,UTF8Type)"));
+        assertColumnSpec(boundNames.get(4), "pk", Int32Type.instance);
+
+
+        List<ColumnSpecification> resultNames = prepared.resultMetadata.names;
+
+        // 7 result "columns"
+        assertEquals(7, resultNames.size());
+        assertColumnSpec(resultNames.get(0), "(int)?", Int32Type.instance);
+        assertColumnSpec(resultNames.get(1), "(decimal)?", DecimalType.instance);
+        assertColumnSpec(resultNames.get(2), "(text)?", UTF8Type.instance);
+        assertColumnSpec(resultNames.get(3), "(tuple<int, text>)?", TypeParser.parse("TupleType(Int32Type,UTF8Type)"));
+        assertColumnSpec(resultNames.get(4), "pk", Int32Type.instance);
+        assertColumnSpec(resultNames.get(5), "ck", Int32Type.instance);
+        assertColumnSpec(resultNames.get(6), "t", UTF8Type.instance);
+
+        assertRows(execute(query, 88, BigDecimal.TEN, "foo bar baz", tuple(42, "ursus"), 1),
+                   row(88, BigDecimal.TEN, "foo bar baz", tuple(42, "ursus"),
+                       1, 1, "one"),
+                   row(88, BigDecimal.TEN, "foo bar baz", tuple(42, "ursus"),
+                       1, 2, "two"),
+                   row(88, BigDecimal.TEN, "foo bar baz", tuple(42, "ursus"),
+                       1, 3, "three"));
+    }
+
+    @Test
+    public void testConstantFunctionArgs() throws Throwable
+    {
+        String fInt = createFunction(KEYSPACE,
+                                     "int,int",
+                                     "CREATE FUNCTION %s (val1 int, val2 int) " +
+                                     "CALLED ON NULL INPUT " +
+                                     "RETURNS int " +
+                                     "LANGUAGE java\n" +
+                                     "AS 'return Math.max(val1, val2);';");
+        String fFloat = createFunction(KEYSPACE,
+                                       "float,float",
+                                       "CREATE FUNCTION %s (val1 float, val2 float) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS float " +
+                                       "LANGUAGE java\n" +
+                                       "AS 'return Math.max(val1, val2);';");
+        String fText = createFunction(KEYSPACE,
+                                      "text,text",
+                                      "CREATE FUNCTION %s (val1 text, val2 text) " +
+                                      "CALLED ON NULL INPUT " +
+                                      "RETURNS text " +
+                                      "LANGUAGE java\n" +
+                                      "AS 'return val2;';");
+        String fAscii = createFunction(KEYSPACE,
+                                       "ascii,ascii",
+                                       "CREATE FUNCTION %s (val1 ascii, val2 ascii) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS ascii " +
+                                       "LANGUAGE java\n" +
+                                       "AS 'return val2;';");
+        String fTimeuuid = createFunction(KEYSPACE,
+                                          "timeuuid,timeuuid",
+                                          "CREATE FUNCTION %s (val1 timeuuid, val2 timeuuid) " +
+                                          "CALLED ON NULL INPUT " +
+                                          "RETURNS timeuuid " +
+                                          "LANGUAGE java\n" +
+                                          "AS 'return val2;';");
+
+        createTable("CREATE TABLE %s (pk int PRIMARY KEY, valInt int, valFloat float, valText text, valAscii ascii, valTimeuuid timeuuid)");
+        execute("INSERT INTO %s (pk, valInt, valFloat, valText, valAscii, valTimeuuid) " +
+                "VALUES (1, 10, 10.0, '100', '100', 2deb23e0-96b5-11e5-b26d-a939dd1405a3)");
+
+        assertRows(execute("SELECT pk, " + fInt + "(valInt, 100) FROM %s"),
+                   row(1, 100));
+        assertRows(execute("SELECT pk, " + fInt + "(valInt, (int)100) FROM %s"),
+                   row(1, 100));
+        assertInvalidMessage("Type error: (bigint)100 cannot be passed as argument 1 of function",
+                             "SELECT pk, " + fInt + "(valInt, (bigint)100) FROM %s");
+        assertRows(execute("SELECT pk, " + fFloat + "(valFloat, (float)100.00) FROM %s"),
+                   row(1, 100f));
+        assertRows(execute("SELECT pk, " + fText + "(valText, 'foo') FROM %s"),
+                   row(1, "foo"));
+        assertRows(execute("SELECT pk, " + fAscii + "(valAscii, (ascii)'foo') FROM %s"),
+                   row(1, "foo"));
+        assertRows(execute("SELECT pk, " + fTimeuuid + "(valTimeuuid, (timeuuid)34617f80-96b5-11e5-b26d-a939dd1405a3) FROM %s"),
+                   row(1, UUID.fromString("34617f80-96b5-11e5-b26d-a939dd1405a3")));
+
+        // ambiguous
+
+        String fAmbiguousFunc1 = createFunction(KEYSPACE,
+                                                "int,bigint",
+                                                "CREATE FUNCTION %s (val1 int, val2 bigint) " +
+                                                "CALLED ON NULL INPUT " +
+                                                "RETURNS bigint " +
+                                                "LANGUAGE java\n" +
+                                                "AS 'return Math.max((long)val1, val2);';");
+        assertRows(execute("SELECT pk, " + fAmbiguousFunc1 + "(valInt, 100) FROM %s"),
+                   row(1, 100L));
+        createFunctionOverload(fAmbiguousFunc1, "int,int",
+                                                "CREATE FUNCTION %s (val1 int, val2 int) " +
+                                                "CALLED ON NULL INPUT " +
+                                                "RETURNS bigint " +
+                                                "LANGUAGE java\n" +
+                                                "AS 'return (long)Math.max(val1, val2);';");
+        assertInvalidMessage("Ambiguous call to function cql_test_keyspace.function_",
+                             "SELECT pk, " + fAmbiguousFunc1 + "(valInt, 100) FROM %s");
+    }
+
+    @Test
+    public void testPreparedFunctionArgs() throws Throwable
+    {
+        createTable("CREATE TABLE %s (pk int, ck int, t text, i int, PRIMARY KEY (pk, ck) )");
+        execute("INSERT INTO %s (pk, ck, t, i) VALUES (1, 1, 'one', 50)");
+        execute("INSERT INTO %s (pk, ck, t, i) VALUES (1, 2, 'two', 100)");
+        execute("INSERT INTO %s (pk, ck, t, i) VALUES (1, 3, 'three', 150)");
+
+        String fIntMax = createFunction(KEYSPACE,
+                                        "int,int",
+                                        "CREATE FUNCTION %s (val1 int, val2 int) " +
+                                        "CALLED ON NULL INPUT " +
+                                        "RETURNS int " +
+                                        "LANGUAGE java\n" +
+                                        "AS 'return Math.max(val1, val2);';");
+
+        // weak typing
+
+        assertRows(execute("SELECT pk, ck, " + fIntMax + "(i, ?) FROM %s", 0),
+                   row(1, 1, 50),
+                   row(1, 2, 100),
+                   row(1, 3, 150));
+        assertRows(execute("SELECT pk, ck, " + fIntMax + "(i, ?) FROM %s", 100),
+                   row(1, 1, 100),
+                   row(1, 2, 100),
+                   row(1, 3, 150));
+        assertRows(execute("SELECT pk, ck, " + fIntMax + "(i, ?) FROM %s", 200),
+                   row(1, 1, 200),
+                   row(1, 2, 200),
+                   row(1, 3, 200));
+
+        // explicit typing
+
+        assertRows(execute("SELECT pk, ck, " + fIntMax + "(i, (int)?) FROM %s", 0),
+                   row(1, 1, 50),
+                   row(1, 2, 100),
+                   row(1, 3, 150));
+        assertRows(execute("SELECT pk, ck, " + fIntMax + "(i, (int)?) FROM %s", 100),
+                   row(1, 1, 100),
+                   row(1, 2, 100),
+                   row(1, 3, 150));
+        assertRows(execute("SELECT pk, ck, " + fIntMax + "(i, (int)?) FROM %s", 200),
+                   row(1, 1, 200),
+                   row(1, 2, 200),
+                   row(1, 3, 200));
+
+        // weak typing
+
+        assertRows(execute("SELECT pk, ck, " + fIntMax + "(i, ?) FROM %s WHERE pk = " + fIntMax + "(1,1)", 0),
+                   row(1, 1, 50),
+                   row(1, 2, 100),
+                   row(1, 3, 150));
+        assertRows(execute("SELECT pk, ck, " + fIntMax + "(i, ?) FROM %s WHERE pk = " + fIntMax + "(2,1)", 0));
+
+        assertRows(execute("SELECT pk, ck, " + fIntMax + "(i, ?) FROM %s WHERE pk = " + fIntMax + "(?,1)", 0, 1),
+                   row(1, 1, 50),
+                   row(1, 2, 100),
+                   row(1, 3, 150));
+        assertRows(execute("SELECT pk, ck, " + fIntMax + "(i, ?) FROM %s WHERE pk = " + fIntMax + "(?,1)", 0, 2));
+
+        // explicit typing
+
+        assertRows(execute("SELECT pk, ck, " + fIntMax + "(i, (int)?) FROM %s WHERE pk = " + fIntMax + "((int)1,(int)1)", 0),
+                   row(1, 1, 50),
+                   row(1, 2, 100),
+                   row(1, 3, 150));
+        assertRows(execute("SELECT pk, ck, " + fIntMax + "(i, (int)?) FROM %s WHERE pk = " + fIntMax + "((int)2,(int)1)", 0));
+
+        assertRows(execute("SELECT pk, ck, " + fIntMax + "(i, (int)?) FROM %s WHERE pk = " + fIntMax + "((int)?,(int)1)", 0, 1),
+                   row(1, 1, 50),
+                   row(1, 2, 100),
+                   row(1, 3, 150));
+        assertRows(execute("SELECT pk, ck, " + fIntMax + "(i, (int)?) FROM %s WHERE pk = " + fIntMax + "((int)?,(int)1)", 0, 2));
+
+        assertInvalidMessage("Invalid unset value for argument", "SELECT pk, ck, " + fIntMax + "(i, (int)?) FROM %s WHERE pk = " + fIntMax + "((int)1,(int)1)", unset());
+    }
+
+    @Test
+    public void testInsertUpdateDelete() throws Throwable
+    {
+        String fIntMax = createFunction(KEYSPACE,
+                                        "int,int",
+                                        "CREATE FUNCTION %s (val1 int, val2 int) " +
+                                        "CALLED ON NULL INPUT " +
+                                        "RETURNS int " +
+                                        "LANGUAGE java\n" +
+                                        "AS 'return Math.max(val1, val2);';");
+
+        createTable("CREATE TABLE %s (pk int, ck int, t text, i int, PRIMARY KEY (pk, ck) )");
+
+        execute("UPDATE %s SET i = " + fIntMax + "(100, 200) WHERE pk = 1 AND ck = 1");
+        assertRows(execute("SELECT i FROM %s WHERE pk = 1 AND ck = 1"),
+                   row(200));
+
+        execute("UPDATE %s SET i = " + fIntMax + "(100, 300) WHERE pk = 1 AND ck = " + fIntMax + "(1,2)");
+        assertRows(execute("SELECT i FROM %s WHERE pk = 1 AND ck = 2"),
+                   row(300));
+
+        execute("DELETE FROM %s WHERE pk = 1 AND ck = " + fIntMax + "(1,2)");
+        assertRows(execute("SELECT i FROM %s WHERE pk = 1 AND ck = 2"));
+
+        execute("INSERT INTO %s (pk, ck, i) VALUES (1, " + fIntMax + "(1,2), " + fIntMax + "(100, 300))");
+        assertRows(execute("SELECT i FROM %s WHERE pk = 1 AND ck = 2"),
+                   row(300));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
index fd09bc4..24a9528 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
@@ -148,9 +148,6 @@ public class AggregationTest extends CQLTester
         assertColumnNames(execute("SELECT COUNT(1) as myCount FROM %s"), "mycount");
         assertRows(execute("SELECT COUNT(1) as myCount FROM %s"), row(0L));
 
-        // Test invalid call
-        assertInvalidSyntaxMessage("Only COUNT(1) is supported, got COUNT(2)", "SELECT COUNT(2) FROM %s");
-
         // Test with other aggregates
         assertColumnNames(execute("SELECT COUNT(*), max(b), b FROM %s"), "count", "system.max(b)", "b");
         assertRows(execute("SELECT COUNT(*), max(b), b  FROM %s"), row(0L, null, null));
@@ -250,7 +247,7 @@ public class AggregationTest extends CQLTester
         assertRows(execute("SELECT count(b.x), max(b.x) as max, b.x, c.x as first FROM %s"),
                    row(3L, 8, 2, null));
 
-        assertInvalidMessage("Invalid field selection: max(b) of type blob is not a user type",
+        assertInvalidMessage("Invalid field selection: system.max(b) of type blob is not a user type",
                              "SELECT max(b).x as max FROM %s");
     }
 
@@ -353,7 +350,6 @@ public class AggregationTest extends CQLTester
 
         assertInvalidSyntax("SELECT max(b), max(c) FROM %s WHERE max(a) = 1");
         assertInvalidMessage("aggregate functions cannot be used as arguments of aggregate functions", "SELECT max(sum(c)) FROM %s");
-        assertInvalidSyntax("SELECT COUNT(2) FROM %s");
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
index 89b90a6..9ead942 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
@@ -397,10 +397,10 @@ public class DeleteTest extends CQLTester
                                  "DELETE FROM %s WHERE partitionKey = ? AND partitionKey = ?", 0, 1);
 
             // unknown identifiers
-            assertInvalidMessage("Unknown identifier unknown",
+            assertInvalidMessage("Undefined column name unknown",
                                  "DELETE unknown FROM %s WHERE partitionKey = ?", 0);
 
-            assertInvalidMessage("Undefined name partitionkey1 in where clause ('partitionkey1 = ?')",
+            assertInvalidMessage("Undefined column name partitionkey1",
                                  "DELETE FROM %s WHERE partitionKey1 = ?", 0);
 
             // Invalid operator in the where clause
@@ -486,13 +486,13 @@ public class DeleteTest extends CQLTester
                                  "DELETE FROM %s WHERE partitionKey = ? AND clustering = ? AND clustering = ?", 0, 1, 1);
 
             // unknown identifiers
-            assertInvalidMessage("Unknown identifier value1",
+            assertInvalidMessage("Undefined column name value1",
                                  "DELETE value1 FROM %s WHERE partitionKey = ? AND clustering = ?", 0, 1);
 
-            assertInvalidMessage("Undefined name partitionkey1 in where clause ('partitionkey1 = ?')",
+            assertInvalidMessage("Undefined column name partitionkey1",
                                  "DELETE FROM %s WHERE partitionKey1 = ? AND clustering = ?", 0, 1);
 
-            assertInvalidMessage("Undefined name clustering_3 in where clause ('clustering_3 = ?')",
+            assertInvalidMessage("Undefined column name clustering_3",
                                  "DELETE FROM %s WHERE partitionKey = ? AND clustering_3 = ?", 0, 1);
 
             // Invalid operator in the where clause
@@ -616,13 +616,13 @@ public class DeleteTest extends CQLTester
                                  "DELETE FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ? AND clustering_1 = ?", 0, 1, 1, 1);
 
             // unknown identifiers
-            assertInvalidMessage("Unknown identifier value1",
+            assertInvalidMessage("Undefined column name value1",
                                  "DELETE value1 FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ?", 0, 1, 1);
 
-            assertInvalidMessage("Undefined name partitionkey1 in where clause ('partitionkey1 = ?')",
+            assertInvalidMessage("Undefined column name partitionkey1",
                                  "DELETE FROM %s WHERE partitionKey1 = ? AND clustering_1 = ? AND clustering_2 = ?", 0, 1, 1);
 
-            assertInvalidMessage("Undefined name clustering_3 in where clause ('clustering_3 = ?')",
+            assertInvalidMessage("Undefined column name clustering_3",
                                  "DELETE FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_3 = ?", 0, 1, 1);
 
             // Invalid operator in the where clause

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
index aa738bb..5314d6a 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
@@ -97,10 +97,10 @@ public class InsertTest extends CQLTester
                              "INSERT INTO %s (partitionKey, clustering, clustering, value) VALUES (0, 0, 0, 2)");
 
         // unknown identifiers
-        assertInvalidMessage("Unknown identifier clusteringx",
+        assertInvalidMessage("Undefined column name clusteringx",
                              "INSERT INTO %s (partitionKey, clusteringx, value) VALUES (0, 0, 2)");
 
-        assertInvalidMessage("Unknown identifier valuex",
+        assertInvalidMessage("Undefined column name valuex",
                              "INSERT INTO %s (partitionKey, clustering, valuex) VALUES (0, 0, 2)");
     }
 
@@ -145,10 +145,10 @@ public class InsertTest extends CQLTester
                              "INSERT INTO %s (partitionKey, clustering, clustering, value) VALUES (0, 0, 0, 2)");
 
         // unknown identifiers
-        assertInvalidMessage("Unknown identifier clusteringx",
+        assertInvalidMessage("Undefined column name clusteringx",
                              "INSERT INTO %s (partitionKey, clusteringx, value) VALUES (0, 0, 2)");
 
-        assertInvalidMessage("Unknown identifier valuex",
+        assertInvalidMessage("Undefined column name valuex",
                              "INSERT INTO %s (partitionKey, clustering, valuex) VALUES (0, 0, 2)");
     }
 
@@ -190,10 +190,10 @@ public class InsertTest extends CQLTester
                              "INSERT INTO %s (partitionKey, clustering_1, clustering_1, clustering_2, value) VALUES (0, 0, 0, 0, 2)");
 
         // unknown identifiers
-        assertInvalidMessage("Unknown identifier clustering_1x",
+        assertInvalidMessage("Undefined column name clustering_1x",
                              "INSERT INTO %s (partitionKey, clustering_1x, clustering_2, value) VALUES (0, 0, 0, 2)");
 
-        assertInvalidMessage("Unknown identifier valuex",
+        assertInvalidMessage("Undefined column name valuex",
                              "INSERT INTO %s (partitionKey, clustering_1, clustering_2, valuex) VALUES (0, 0, 0, 2)");
     }
 
@@ -243,10 +243,10 @@ public class InsertTest extends CQLTester
                              "INSERT INTO %s (partitionKey, clustering_1, clustering_1, clustering_2, value) VALUES (0, 0, 0, 0, 2)");
 
         // unknown identifiers
-        assertInvalidMessage("Unknown identifier clustering_1x",
+        assertInvalidMessage("Undefined column name clustering_1x",
                              "INSERT INTO %s (partitionKey, clustering_1x, clustering_2, value) VALUES (0, 0, 0, 2)");
 
-        assertInvalidMessage("Unknown identifier valuex",
+        assertInvalidMessage("Undefined column name valuex",
                              "INSERT INTO %s (partitionKey, clustering_1, clustering_2, valuex) VALUES (0, 0, 0, 2)");
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
index 0db0039..1239b7a 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
@@ -1909,11 +1909,11 @@ public class SelectMultiColumnRelationTest extends CQLTester
     public void testInvalidColumnNames() throws Throwable
     {
         createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
-        assertInvalidMessage("Undefined name e in where clause ('(b, e) = (0, 0)')", "SELECT * FROM %s WHERE (b, e) = (0, 0)");
-        assertInvalidMessage("Undefined name e in where clause ('(b, e) IN ((0, 1), (2, 4))')", "SELECT * FROM %s WHERE (b, e) IN ((0, 1), (2, 4))");
-        assertInvalidMessage("Undefined name e in where clause ('(b, e) > (0, 1)')", "SELECT * FROM %s WHERE (b, e) > (0, 1) and b <= 2");
-        assertInvalidMessage("Aliases aren't allowed in the where clause ('(b, e) = (0, 0)')", "SELECT c AS e FROM %s WHERE (b, e) = (0, 0)");
-        assertInvalidMessage("Aliases aren't allowed in the where clause ('(b, e) IN ((0, 1), (2, 4))')", "SELECT c AS e FROM %s WHERE (b, e) IN ((0, 1), (2, 4))");
-        assertInvalidMessage("Aliases aren't allowed in the where clause ('(b, e) > (0, 1)')", "SELECT c AS e FROM %s WHERE (b, e) > (0, 1) and b <= 2");
+        assertInvalidMessage("Undefined column name e", "SELECT * FROM %s WHERE (b, e) = (0, 0)");
+        assertInvalidMessage("Undefined column name e", "SELECT * FROM %s WHERE (b, e) IN ((0, 1), (2, 4))");
+        assertInvalidMessage("Undefined column name e", "SELECT * FROM %s WHERE (b, e) > (0, 1) and b <= 2");
+        assertInvalidMessage("Undefined column name e", "SELECT c AS e FROM %s WHERE (b, e) = (0, 0)");
+        assertInvalidMessage("Undefined column name e", "SELECT c AS e FROM %s WHERE (b, e) IN ((0, 1), (2, 4))");
+        assertInvalidMessage("Undefined column name e", "SELECT c AS e FROM %s WHERE (b, e) > (0, 1) and b <= 2");
     }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderedPartitionerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderedPartitionerTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderedPartitionerTest.java
index 0b2ed9f..89f3e65 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderedPartitionerTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderedPartitionerTest.java
@@ -483,9 +483,9 @@ public class SelectOrderedPartitionerTest extends CQLTester
     public void testTokenFunctionWithInvalidColumnNames() throws Throwable
     {
         createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY ((a, b), c))");
-        assertInvalidMessage("Undefined name e in where clause ('token(a, e) = token(0, 0)')", "SELECT * FROM %s WHERE token(a, e) = token(0, 0)");
-        assertInvalidMessage("Undefined name e in where clause ('token(a, e) > token(0, 1)')", "SELECT * FROM %s WHERE token(a, e) > token(0, 1)");
-        assertInvalidMessage("Aliases aren't allowed in the where clause ('token(a, e) = token(0, 0)')", "SELECT b AS e FROM %s WHERE token(a, e) = token(0, 0)");
-        assertInvalidMessage("Aliases aren't allowed in the where clause ('token(a, e) > token(0, 1)')", "SELECT b AS e FROM %s WHERE token(a, e) > token(0, 1)");
+        assertInvalidMessage("Undefined column name e", "SELECT * FROM %s WHERE token(a, e) = token(0, 0)");
+        assertInvalidMessage("Undefined column name e", "SELECT * FROM %s WHERE token(a, e) > token(0, 1)");
+        assertInvalidMessage("Undefined column name e", "SELECT b AS e FROM %s WHERE token(a, e) = token(0, 0)");
+        assertInvalidMessage("Undefined column name e", "SELECT b AS e FROM %s WHERE token(a, e) > token(0, 1)");
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
index 4beb1fb..0e2517b 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
@@ -620,16 +620,16 @@ public class SelectSingleColumnRelationTest extends CQLTester
     public void testInvalidColumnNames() throws Throwable
     {
         createTable("CREATE TABLE %s (a int, b int, c map<int, int>, PRIMARY KEY (a, b))");
-        assertInvalidMessage("Undefined name d in where clause ('d = 0')", "SELECT * FROM %s WHERE d = 0");
-        assertInvalidMessage("Undefined name d in where clause ('d IN [0, 1]')", "SELECT * FROM %s WHERE d IN (0, 1)");
-        assertInvalidMessage("Undefined name d in where clause ('d > 0')", "SELECT * FROM %s WHERE d > 0 and d <= 2");
-        assertInvalidMessage("Undefined name d in where clause ('d CONTAINS 0')", "SELECT * FROM %s WHERE d CONTAINS 0");
-        assertInvalidMessage("Undefined name d in where clause ('d CONTAINS KEY 0')", "SELECT * FROM %s WHERE d CONTAINS KEY 0");
-        assertInvalidMessage("Aliases aren't allowed in the where clause ('d = 0')", "SELECT a AS d FROM %s WHERE d = 0");
-        assertInvalidMessage("Aliases aren't allowed in the where clause ('d IN [0, 1]')", "SELECT b AS d FROM %s WHERE d IN (0, 1)");
-        assertInvalidMessage("Aliases aren't allowed in the where clause ('d > 0')", "SELECT b AS d FROM %s WHERE d > 0 and d <= 2");
-        assertInvalidMessage("Aliases aren't allowed in the where clause ('d CONTAINS 0')", "SELECT c AS d FROM %s WHERE d CONTAINS 0");
-        assertInvalidMessage("Aliases aren't allowed in the where clause ('d CONTAINS KEY 0')", "SELECT c AS d FROM %s WHERE d CONTAINS KEY 0");
-        assertInvalidMessage("Undefined name d in selection clause", "SELECT d FROM %s WHERE a = 0");
+        assertInvalidMessage("Undefined column name d", "SELECT * FROM %s WHERE d = 0");
+        assertInvalidMessage("Undefined column name d", "SELECT * FROM %s WHERE d IN (0, 1)");
+        assertInvalidMessage("Undefined column name d", "SELECT * FROM %s WHERE d > 0 and d <= 2");
+        assertInvalidMessage("Undefined column name d", "SELECT * FROM %s WHERE d CONTAINS 0");
+        assertInvalidMessage("Undefined column name d", "SELECT * FROM %s WHERE d CONTAINS KEY 0");
+        assertInvalidMessage("Undefined column name d", "SELECT a AS d FROM %s WHERE d = 0");
+        assertInvalidMessage("Undefined column name d", "SELECT b AS d FROM %s WHERE d IN (0, 1)");
+        assertInvalidMessage("Undefined column name d", "SELECT b AS d FROM %s WHERE d > 0 and d <= 2");
+        assertInvalidMessage("Undefined column name d", "SELECT c AS d FROM %s WHERE d CONTAINS 0");
+        assertInvalidMessage("Undefined column name d", "SELECT c AS d FROM %s WHERE d CONTAINS KEY 0");
+        assertInvalidMessage("Undefined column name d", "SELECT d FROM %s WHERE a = 0");
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
index f91ec5a..7c0ca7c 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
@@ -1159,11 +1159,11 @@ public class SelectTest extends CQLTester
         assertEquals(ByteBuffer.wrap(new byte[4]), rs.one().getBlob(rs.metadata().get(0).name.toString()));
 
         // test that select throws a meaningful exception for aliases in where clause
-        assertInvalidMessage("Aliases aren't allowed in the where clause",
+        assertInvalidMessage("Undefined column name user_id",
                              "SELECT id AS user_id, name AS user_name FROM %s WHERE user_id = 0");
 
         // test that select throws a meaningful exception for aliases in order by clause
-        assertInvalidMessage("Aliases are not allowed in order by clause",
+        assertInvalidMessage("Undefined column name user_name",
                              "SELECT id AS user_id, name AS user_name FROM %s WHERE id IN (0) ORDER BY user_name");
     }
 
@@ -1401,11 +1401,11 @@ public class SelectTest extends CQLTester
         for (int i = 0; i < 5; i++)
             execute("INSERT INTO %s (id, name) VALUES (?, ?) USING TTL 10 AND TIMESTAMP 0", i, Integer.toString(i));
 
-        assertInvalidMessage("Aliases aren't allowed in the where clause",
+        assertInvalidMessage("Undefined column name user_id",
                              "SELECT id AS user_id, name AS user_name FROM %s WHERE user_id = 0");
 
         // test that select throws a meaningful exception for aliases in order by clause
-        assertInvalidMessage("Aliases are not allowed in order by clause",
+        assertInvalidMessage("Undefined column name user_name",
                              "SELECT id AS user_id, name AS user_name FROM %s WHERE id IN (0) ORDER BY user_name");
 
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
index b2e59a3..9c42fc2 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
@@ -196,13 +196,13 @@ public class UpdateTest extends CQLTester
                                  "UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ? AND clustering_1 = ?", 7, 0, 1, 1);
 
             // unknown identifiers
-            assertInvalidMessage("Unknown identifier value1",
+            assertInvalidMessage("Undefined column name value1",
                                  "UPDATE %s SET value1 = ? WHERE partitionKey = ? AND clustering_1 = ?", 7, 0, 1);
 
-            assertInvalidMessage("Undefined name partitionkey1 in where clause ('partitionkey1 = ?')",
+            assertInvalidMessage("Undefined column name partitionkey1",
                                  "UPDATE %s SET value = ? WHERE partitionKey1 = ? AND clustering_1 = ?", 7, 0, 1);
 
-            assertInvalidMessage("Undefined name clustering_3 in where clause ('clustering_3 = ?')",
+            assertInvalidMessage("Undefined column name clustering_3",
                                  "UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_3 = ?", 7, 0, 1);
 
             // Invalid operator in the where clause
@@ -383,13 +383,13 @@ public class UpdateTest extends CQLTester
                                  "UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ? AND clustering_1 = ?", 7, 0, 1, 1, 1);
 
             // unknown identifiers
-            assertInvalidMessage("Unknown identifier value1",
+            assertInvalidMessage("Undefined column name value1",
                                  "UPDATE %s SET value1 = ? WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ?", 7, 0, 1, 1);
 
-            assertInvalidMessage("Undefined name partitionkey1 in where clause ('partitionkey1 = ?')",
+            assertInvalidMessage("Undefined column name partitionkey1",
                                  "UPDATE %s SET value = ? WHERE partitionKey1 = ? AND clustering_1 = ? AND clustering_2 = ?", 7, 0, 1, 1);
 
-            assertInvalidMessage("Undefined name clustering_3 in where clause ('clustering_3 = ?')",
+            assertInvalidMessage("Undefined column name clustering_3",
                                  "UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ? AND clustering_3 = ?", 7, 0, 1, 1);
 
             // Invalid operator in the where clause

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java b/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java
index 9432c90..2de671c 100644
--- a/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java
+++ b/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java
@@ -31,6 +31,7 @@ import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.FieldIdentifier;
 import org.apache.cassandra.cql3.functions.*;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.*;
@@ -95,6 +96,11 @@ public class LegacySchemaMigratorTest
         expected.forEach(LegacySchemaMigratorTest::verifyIndexBuildStatus);
     }
 
+    private static FieldIdentifier field(String field)
+    {
+        return FieldIdentifier.forQuoted(field);
+    }
+
     private static void loadLegacySchemaTables()
     {
         KeyspaceMetadata systemKeyspace = Schema.instance.getKSMetaData(SystemKeyspace.NAME);
@@ -310,19 +316,19 @@ public class LegacySchemaMigratorTest
 
         UserType udt1 = new UserType(keyspace,
                                      bytes("udt1"),
-                                     new ArrayList<ByteBuffer>() {{ add(bytes("col1")); add(bytes("col2")); }},
+                                     new ArrayList<FieldIdentifier>() {{ add(field("col1")); add(field("col2")); }},
                                      new ArrayList<AbstractType<?>>() {{ add(UTF8Type.instance); add(Int32Type.instance); }},
                                      true);
 
         UserType udt2 = new UserType(keyspace,
                                      bytes("udt2"),
-                                     new ArrayList<ByteBuffer>() {{ add(bytes("col3")); add(bytes("col4")); }},
+                                     new ArrayList<FieldIdentifier>() {{ add(field("col3")); add(field("col4")); }},
                                      new ArrayList<AbstractType<?>>() {{ add(BytesType.instance); add(BooleanType.instance); }},
                                      true);
 
         UserType udt3 = new UserType(keyspace,
                                      bytes("udt3"),
-                                     new ArrayList<ByteBuffer>() {{ add(bytes("col5")); }},
+                                     new ArrayList<FieldIdentifier>() {{ add(field("col5")); }},
                                      new ArrayList<AbstractType<?>>() {{ add(AsciiType.instance); }},
                                      true);
 
@@ -433,13 +439,13 @@ public class LegacySchemaMigratorTest
 
         UserType udt1 = new UserType(keyspace,
                                      bytes("udt1"),
-                                     new ArrayList<ByteBuffer>() {{ add(bytes("col1")); add(bytes("col2")); }},
+                                     new ArrayList<FieldIdentifier>() {{ add(field("col1")); add(field("col2")); }},
                                      new ArrayList<AbstractType<?>>() {{ add(UTF8Type.instance); add(Int32Type.instance); }},
                                      true);
 
         UserType udt2 = new UserType(keyspace,
                                      bytes("udt2"),
-                                     new ArrayList<ByteBuffer>() {{ add(bytes("col1")); add(bytes("col2")); }},
+                                     new ArrayList<FieldIdentifier>() {{ add(field("col1")); add(field("col2")); }},
                                      new ArrayList<AbstractType<?>>() {{ add(ListType.getInstance(udt1, false)); add(Int32Type.instance); }},
                                      true);
 
@@ -482,13 +488,13 @@ public class LegacySchemaMigratorTest
 
         UserType udt1 = new UserType(keyspace,
                                      bytes("udt1"),
-                                     new ArrayList<ByteBuffer>() {{ add(bytes("col1")); add(bytes("col2")); }},
+                                     new ArrayList<FieldIdentifier>() {{ add(field("col1")); add(field("col2")); }},
                                      new ArrayList<AbstractType<?>>() {{ add(UTF8Type.instance); add(Int32Type.instance); }},
                                      true);
 
         UserType udt2 = new UserType(keyspace,
                                      bytes("udt2"),
-                                     new ArrayList<ByteBuffer>() {{ add(bytes("col1")); add(bytes("col2")); }},
+                                     new ArrayList<FieldIdentifier>() {{ add(field("col1")); add(field("col2")); }},
                                      new ArrayList<AbstractType<?>>() {{ add(ListType.getInstance(udt1, false)); add(Int32Type.instance); }},
                                      true);
 
@@ -728,7 +734,7 @@ public class LegacySchemaMigratorTest
 
         for (int i = 0; i < type.size(); i++)
         {
-            adder.addListEntry("field_names", type.fieldName(i))
+            adder.addListEntry("field_names", type.fieldName(i).toString())
                  .addListEntry("field_types", type.fieldType(i).toString());
         }
 


[4/4] cassandra git commit: Allow terms in selection clauses

Posted by sl...@apache.org.
Allow terms in selection clauses

patch by slebresne; reviewed by blerer for CASSANDRA-10783


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

Branch: refs/heads/trunk
Commit: 4ed00607df060a3cdcc685c4fc0c1967f62b37f1
Parents: 60e5e0e
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Tue May 24 15:49:48 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Jun 15 09:33:06 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |  10 +
 doc/cql3/CQL.textile                            |   8 +-
 pylib/cqlshlib/cql3handling.py                  |   1 +
 src/antlr/Cql.g                                 |   1 +
 src/antlr/Parser.g                              | 103 +++---
 .../cassandra/config/ColumnDefinition.java      | 202 ++++++++++-
 .../apache/cassandra/config/ViewDefinition.java |   4 +-
 .../apache/cassandra/cql3/AbstractMarker.java   |   7 +
 .../apache/cassandra/cql3/ColumnCondition.java  |  40 +--
 .../apache/cassandra/cql3/ColumnIdentifier.java | 143 +-------
 .../org/apache/cassandra/cql3/Constants.java    |  24 +-
 .../apache/cassandra/cql3/FieldIdentifier.java  |  98 ++++++
 src/java/org/apache/cassandra/cql3/Json.java    |  11 +
 src/java/org/apache/cassandra/cql3/Lists.java   |  13 +
 src/java/org/apache/cassandra/cql3/Maps.java    |  19 +-
 .../cassandra/cql3/MultiColumnRelation.java     |  20 +-
 .../org/apache/cassandra/cql3/Operation.java    |  41 ++-
 .../org/apache/cassandra/cql3/Relation.java     |  27 +-
 src/java/org/apache/cassandra/cql3/Sets.java    |  12 +
 .../cassandra/cql3/SingleColumnRelation.java    |  26 +-
 src/java/org/apache/cassandra/cql3/Term.java    |  14 +-
 .../apache/cassandra/cql3/TokenRelation.java    |  16 +-
 src/java/org/apache/cassandra/cql3/Tuples.java  |  24 ++
 .../org/apache/cassandra/cql3/TypeCast.java     |   5 +
 .../org/apache/cassandra/cql3/UserTypes.java    |  44 +--
 .../cassandra/cql3/functions/FunctionCall.java  |  10 +
 .../selection/AbstractFunctionSelector.java     |  23 +-
 .../selection/AggregateFunctionSelector.java    |   4 +-
 .../cassandra/cql3/selection/FieldSelector.java |  11 +-
 .../cql3/selection/ScalarFunctionSelector.java  |   6 +-
 .../cassandra/cql3/selection/Selectable.java    | 350 ++++++++++++++-----
 .../cassandra/cql3/selection/Selection.java     |  36 +-
 .../cassandra/cql3/selection/Selector.java      |  32 +-
 .../cql3/selection/SelectorFactories.java       |  31 +-
 .../cql3/selection/SimpleSelector.java          |   3 +-
 .../cassandra/cql3/selection/TermSelector.java  |  92 +++++
 .../cql3/selection/WritetimeOrTTLSelector.java  |   3 +-
 .../cql3/statements/AlterTableStatement.java    |  23 +-
 .../statements/AlterTableStatementColumn.java   |  24 +-
 .../cql3/statements/AlterTypeStatement.java     |  44 +--
 .../cql3/statements/CreateTypeStatement.java    |  18 +-
 .../cql3/statements/CreateViewStatement.java    |  53 ++-
 .../cql3/statements/DeleteStatement.java        |   2 +-
 .../cassandra/cql3/statements/IndexTarget.java  |  22 +-
 .../cql3/statements/ModificationStatement.java  |  24 +-
 .../cql3/statements/SelectStatement.java        |  84 ++---
 .../cql3/statements/UpdateStatement.java        |  16 +-
 .../cassandra/db/marshal/AbstractType.java      |  24 +-
 .../apache/cassandra/db/marshal/TypeParser.java |   7 +-
 .../apache/cassandra/db/marshal/UserType.java   |  41 +--
 src/java/org/apache/cassandra/db/view/View.java |   4 +-
 .../exceptions/UnrecognizedEntityException.java |  49 ---
 .../cassandra/schema/LegacySchemaMigrator.java  |   5 +-
 .../apache/cassandra/schema/SchemaKeyspace.java |   2 +-
 src/java/org/apache/cassandra/schema/Types.java |   5 +-
 .../apache/cassandra/transport/DataType.java    |  11 +-
 .../cassandra/cql3/CQL3TypeLiteralTest.java     |   4 +-
 .../org/apache/cassandra/cql3/CQLTester.java    |   5 +
 .../cql3/selection/TermSelectionTest.java       | 338 ++++++++++++++++++
 .../validation/operations/AggregationTest.java  |   6 +-
 .../cql3/validation/operations/DeleteTest.java  |  16 +-
 .../cql3/validation/operations/InsertTest.java  |  16 +-
 .../SelectMultiColumnRelationTest.java          |  12 +-
 .../SelectOrderedPartitionerTest.java           |   8 +-
 .../SelectSingleColumnRelationTest.java         |  22 +-
 .../cql3/validation/operations/SelectTest.java  |   8 +-
 .../cql3/validation/operations/UpdateTest.java  |  12 +-
 .../schema/LegacySchemaMigratorTest.java        |  22 +-
 .../cassandra/transport/SerDeserTest.java       |  18 +-
 70 files changed, 1631 insertions(+), 829 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 8689b7d..dbcfa34 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.8
+ * Allow terms in selection clause (CASSANDRA-10783)
  * Add bind variables to trace (CASSANDRA-11719)
  * Switch counter shards' clock to timestamps (CASSANDRA-9811)
  * Introduce HdrHistogram and response/service/wait separation to stress tool (CASSANDRA-11853)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 076d024..f9430ac 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -38,6 +38,16 @@ Deprecation
      cause increased compaction load for a while after the migration so make sure you run
      tests before migrating. Read CASSANDRA-9666 for background on this.
 
+Upgrading
+---------
+    - The name "json" and "distinct" are not valid anymore a user-defined function
+      names (they are still valid as column name however). In the unlikely case where
+      you had defined functions with such names, you will need to recreate
+      those under a different name, change your code to use the new names and
+      drop the old versions, and this _before_ upgrade (see CASSANDRA-10783 for more
+      details).
+
+
 3.7
 ===
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/doc/cql3/CQL.textile
----------------------------------------------------------------------
diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile
index 275717c..b4a3403 100644
--- a/doc/cql3/CQL.textile
+++ b/doc/cql3/CQL.textile
@@ -853,10 +853,7 @@ bc(syntax)..
 
 <names-list> ::= '(' <identifier> ( ',' <identifier> )* ')'
 
-<value-list> ::= '(' <term-or-literal> ( ',' <term-or-literal> )* ')'
-
-<term-or-literal> ::= <term>
-                    | <collection-literal>
+<value-list> ::= '(' <term> ( ',' <term> )* ')'
 
 <option> ::= TIMESTAMP <integer>
            | TTL <integer>
@@ -1070,13 +1067,14 @@ bc(syntax)..
                   ( ALLOW FILTERING )?
 
 <select-clause> ::= DISTINCT? <selection-list>
-                  | COUNT '(' ( '*' | '1' ) ')' (AS <identifier>)?
 
 <selection-list> ::= <selector> (AS <identifier>)? ( ',' <selector> (AS <identifier>)? )*
                    | '*'
 
 <selector> ::= <identifier>
+             | <term>
              | WRITETIME '(' <identifier> ')'
+             | COUNT '(' '*' ')'
              | TTL '(' <identifier> ')'
              | CAST '(' <selector> AS <type> ')'
              | <function> '(' (<selector> (',' <selector>)*)? ')'

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index 2f88e27..31a8459 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -702,6 +702,7 @@ syntax_rules += r'''
              | "COUNT" "(" star=( "*" | "1" ) ")"
              | "CAST" "(" <selector> "AS" <storageType> ")"
              | <functionName> <selectionFunctionArguments>
+             | <term>
              ;
 <selectionFunctionArguments> ::= "(" ( <selector> ( "," <selector> )* )? ")"
                           ;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/antlr/Cql.g
----------------------------------------------------------------------
diff --git a/src/antlr/Cql.g b/src/antlr/Cql.g
index 7cc16a3..61bdc43 100644
--- a/src/antlr/Cql.g
+++ b/src/antlr/Cql.g
@@ -40,6 +40,7 @@ import Parser,Lexer;
     import java.util.Set;
 
     import org.apache.cassandra.auth.*;
+    import org.apache.cassandra.config.ColumnDefinition;
     import org.apache.cassandra.cql3.*;
     import org.apache.cassandra.cql3.restrictions.CustomIndexExpression;
     import org.apache.cassandra.cql3.statements.*;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/antlr/Parser.g
----------------------------------------------------------------------
diff --git a/src/antlr/Parser.g b/src/antlr/Parser.g
index deaf5d0..cdb2263 100644
--- a/src/antlr/Parser.g
+++ b/src/antlr/Parser.g
@@ -133,9 +133,9 @@ options {
         return res;
     }
 
-    public void addRawUpdate(List<Pair<ColumnIdentifier.Raw, Operation.RawUpdate>> operations, ColumnIdentifier.Raw key, Operation.RawUpdate update)
+    public void addRawUpdate(List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations, ColumnDefinition.Raw key, Operation.RawUpdate update)
     {
-        for (Pair<ColumnIdentifier.Raw, Operation.RawUpdate> p : operations)
+        for (Pair<ColumnDefinition.Raw, Operation.RawUpdate> p : operations)
         {
             if (p.left.equals(key) && !p.right.isCompatibleWith(update))
                 addRecognitionError("Multiple incompatible setting of column " + key);
@@ -247,7 +247,7 @@ selectStatement returns [SelectStatement.RawStatement expr]
         boolean isDistinct = false;
         Term.Raw limit = null;
         Term.Raw perPartitionLimit = null;
-        Map<ColumnIdentifier.Raw, Boolean> orderings = new LinkedHashMap<ColumnIdentifier.Raw, Boolean>();
+        Map<ColumnDefinition.Raw, Boolean> orderings = new LinkedHashMap<>();
         boolean allowFiltering = false;
         boolean isJson = false;
     }
@@ -280,15 +280,21 @@ selector returns [RawSelector s]
     : us=unaliasedSelector (K_AS c=noncol_ident { alias = c; })? { $s = new RawSelector(us, alias); }
     ;
 
+/*
+ * A single selection. The core of it is selecting a column, but we also allow any term and function, as well as
+ * sub-element selection for UDT.
+ */
 unaliasedSelector returns [Selectable.Raw s]
     @init { Selectable.Raw tmp = null; }
     :  ( c=cident                                  { tmp = c; }
-       | K_COUNT '(' countArgument ')'             { tmp = Selectable.WithFunction.Raw.newCountRowsFunction();}
+       | v=value                                   { tmp = new Selectable.WithTerm.Raw(v); }
+       | '(' ct=comparatorType ')' v=value         { tmp = new Selectable.WithTerm.Raw(new TypeCast(ct, v)); }
+       | K_COUNT '(' '\*' ')'                      { tmp = Selectable.WithFunction.Raw.newCountRowsFunction(); }
        | K_WRITETIME '(' c=cident ')'              { tmp = new Selectable.WritetimeOrTTL.Raw(c, true); }
        | K_TTL       '(' c=cident ')'              { tmp = new Selectable.WritetimeOrTTL.Raw(c, false); }
        | K_CAST      '(' sn=unaliasedSelector K_AS t=native_type ')' {tmp = new Selectable.WithCast.Raw(sn, t);}
        | f=functionName args=selectionFunctionArgs { tmp = new Selectable.WithFunction.Raw(f, args); }
-       ) ( '.' fi=cident { tmp = new Selectable.WithFieldSelection.Raw(tmp, fi); } )* { $s = tmp; }
+       ) ( '.' fi=fident { tmp = new Selectable.WithFieldSelection.Raw(tmp, fi); } )* { $s = tmp; }
     ;
 
 selectionFunctionArgs returns [List<Selectable.Raw> a]
@@ -298,11 +304,6 @@ selectionFunctionArgs returns [List<Selectable.Raw> a]
       ')' { $a = args; }
     ;
 
-countArgument
-    : '\*'
-    | i=INTEGER { if (!i.getText().equals("1")) addRecognitionError("Only COUNT(1) is supported, got COUNT(" + i.getText() + ")");}
-    ;
-
 whereClause returns [WhereClause.Builder clause]
     @init{ $clause = new WhereClause.Builder(); }
     : relationOrExpression[$clause] (K_AND relationOrExpression[$clause])*
@@ -318,7 +319,7 @@ customIndexExpression [WhereClause.Builder clause]
     : 'expr(' idxName[name] ',' t=term ')' { clause.add(new CustomIndexExpression(name, t));}
     ;
 
-orderByClause[Map<ColumnIdentifier.Raw, Boolean> orderings]
+orderByClause[Map<ColumnDefinition.Raw, Boolean> orderings]
     @init{
         boolean reversed = false;
     }
@@ -340,8 +341,8 @@ insertStatement returns [ModificationStatement.Parsed expr]
 normalInsertStatement [CFName cf] returns [UpdateStatement.ParsedInsert expr]
     @init {
         Attributes.Raw attrs = new Attributes.Raw();
-        List<ColumnIdentifier.Raw> columnNames  = new ArrayList<ColumnIdentifier.Raw>();
-        List<Term.Raw> values = new ArrayList<Term.Raw>();
+        List<ColumnDefinition.Raw> columnNames  = new ArrayList<>();
+        List<Term.Raw> values = new ArrayList<>();
         boolean ifNotExists = false;
     }
     : '(' c1=cident { columnNames.add(c1); }  ( ',' cn=cident { columnNames.add(cn); } )* ')'
@@ -393,7 +394,7 @@ usingClauseObjective[Attributes.Raw attrs]
 updateStatement returns [UpdateStatement.ParsedUpdate expr]
     @init {
         Attributes.Raw attrs = new Attributes.Raw();
-        List<Pair<ColumnIdentifier.Raw, Operation.RawUpdate>> operations = new ArrayList<Pair<ColumnIdentifier.Raw, Operation.RawUpdate>>();
+        List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations = new ArrayList<>();
         boolean ifExists = false;
     }
     : K_UPDATE cf=columnFamilyName
@@ -406,13 +407,13 @@ updateStatement returns [UpdateStatement.ParsedUpdate expr]
                                                   attrs,
                                                   operations,
                                                   wclause.build(),
-                                                  conditions == null ? Collections.<Pair<ColumnIdentifier.Raw, ColumnCondition.Raw>>emptyList() : conditions,
+                                                  conditions == null ? Collections.<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>>emptyList() : conditions,
                                                   ifExists);
      }
     ;
 
-updateConditions returns [List<Pair<ColumnIdentifier.Raw, ColumnCondition.Raw>> conditions]
-    @init { conditions = new ArrayList<Pair<ColumnIdentifier.Raw, ColumnCondition.Raw>>(); }
+updateConditions returns [List<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>> conditions]
+    @init { conditions = new ArrayList<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>>(); }
     : columnCondition[conditions] ( K_AND columnCondition[conditions] )*
     ;
 
@@ -440,7 +441,7 @@ deleteStatement returns [DeleteStatement.Parsed expr]
                                             attrs,
                                             columnDeletions,
                                             wclause.build(),
-                                            conditions == null ? Collections.<Pair<ColumnIdentifier.Raw, ColumnCondition.Raw>>emptyList() : conditions,
+                                            conditions == null ? Collections.<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>>emptyList() : conditions,
                                             ifExists);
       }
     ;
@@ -454,7 +455,7 @@ deleteSelection returns [List<Operation.RawDeletion> operations]
 deleteOp returns [Operation.RawDeletion op]
     : c=cident                { $op = new Operation.ColumnDeletion(c); }
     | c=cident '[' t=term ']' { $op = new Operation.ElementDeletion(c, t); }
-    | c=cident '.' field=cident { $op = new Operation.FieldDeletion(c, field); }
+    | c=cident '.' field=fident { $op = new Operation.FieldDeletion(c, field); }
     ;
 
 usingClauseDelete[Attributes.Raw attrs]
@@ -673,7 +674,7 @@ createTypeStatement returns [CreateTypeStatement expr]
     ;
 
 typeColumns[CreateTypeStatement expr]
-    : k=noncol_ident v=comparatorType { $expr.addDefinition(k, v); }
+    : k=fident v=comparatorType { $expr.addDefinition(k, v); }
     ;
 
 
@@ -714,8 +715,8 @@ indexIdent [List<IndexTarget.Raw> targets]
 createMaterializedViewStatement returns [CreateViewStatement expr]
     @init {
         boolean ifNotExists = false;
-        List<ColumnIdentifier.Raw> partitionKeys = new ArrayList<>();
-        List<ColumnIdentifier.Raw> compositeKeys = new ArrayList<>();
+        List<ColumnDefinition.Raw> partitionKeys = new ArrayList<>();
+        List<ColumnDefinition.Raw> compositeKeys = new ArrayList<>();
     }
     : K_CREATE K_MATERIALIZED K_VIEW (K_IF K_NOT K_EXISTS { ifNotExists = true; })? cf=columnFamilyName K_AS
         K_SELECT sclause=selectClause K_FROM basecf=columnFamilyName
@@ -740,7 +741,7 @@ createTriggerStatement returns [CreateTriggerStatement expr]
     }
     : K_CREATE K_TRIGGER (K_IF K_NOT K_EXISTS { ifNotExists = true; } )? (name=cident)
         K_ON cf=columnFamilyName K_USING cls=STRING_LITERAL
-      { $expr = new CreateTriggerStatement(cf, name.toString(), $cls.text, ifNotExists); }
+      { $expr = new CreateTriggerStatement(cf, name.rawText(), $cls.text, ifNotExists); }
     ;
 
 /**
@@ -749,7 +750,7 @@ createTriggerStatement returns [CreateTriggerStatement expr]
 dropTriggerStatement returns [DropTriggerStatement expr]
      @init { boolean ifExists = false; }
     : K_DROP K_TRIGGER (K_IF K_EXISTS { ifExists = true; } )? (name=cident) K_ON cf=columnFamilyName
-      { $expr = new DropTriggerStatement(cf, name.toString(), ifExists); }
+      { $expr = new DropTriggerStatement(cf, name.rawText(), ifExists); }
     ;
 
 /**
@@ -773,7 +774,7 @@ alterTableStatement returns [AlterTableStatement expr]
     @init {
         AlterTableStatement.Type type = null;
         TableAttributes attrs = new TableAttributes();
-        Map<ColumnIdentifier.Raw, ColumnIdentifier.Raw> renames = new HashMap<ColumnIdentifier.Raw, ColumnIdentifier.Raw>();
+        Map<ColumnDefinition.Raw, ColumnDefinition.Raw> renames = new HashMap<ColumnDefinition.Raw, ColumnDefinition.Raw>();
         List<AlterTableStatementColumn> colNameList = new ArrayList<AlterTableStatementColumn>();
     }
     : K_ALTER K_COLUMNFAMILY cf=columnFamilyName
@@ -821,12 +822,12 @@ alterMaterializedViewStatement returns [AlterViewStatement expr]
  */
 alterTypeStatement returns [AlterTypeStatement expr]
     : K_ALTER K_TYPE name=userTypeName
-          ( K_ALTER f=noncol_ident K_TYPE v=comparatorType { $expr = AlterTypeStatement.alter(name, f, v); }
-          | K_ADD   f=noncol_ident v=comparatorType        { $expr = AlterTypeStatement.addition(name, f, v); }
+          ( K_ALTER f=fident K_TYPE v=comparatorType { $expr = AlterTypeStatement.alter(name, f, v); }
+          | K_ADD   f=fident v=comparatorType        { $expr = AlterTypeStatement.addition(name, f, v); }
           | K_RENAME
-               { Map<ColumnIdentifier, ColumnIdentifier> renames = new HashMap<ColumnIdentifier, ColumnIdentifier>(); }
-                 id1=noncol_ident K_TO toId1=noncol_ident { renames.put(id1, toId1); }
-                 ( K_AND idn=noncol_ident K_TO toIdn=noncol_ident { renames.put(idn, toIdn); } )*
+               { Map<FieldIdentifier, FieldIdentifier> renames = new HashMap<>(); }
+                 id1=fident K_TO toId1=fident { renames.put(id1, toId1); }
+                 ( K_AND idn=fident K_TO toIdn=fident { renames.put(idn, toIdn); } )*
                { $expr = AlterTypeStatement.renames(name, renames); }
           )
     ;
@@ -1144,10 +1145,10 @@ userPassword[RoleOptions opts]
 // 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.Literal($t.text, false); }
-    | t=QUOTED_NAME        { $id = new ColumnIdentifier.Literal($t.text, true); }
-    | k=unreserved_keyword { $id = new ColumnIdentifier.Literal(k, false); }
+cident returns [ColumnDefinition.Raw id]
+    : t=IDENT              { $id = ColumnDefinition.Raw.forUnquoted($t.text); }
+    | t=QUOTED_NAME        { $id = ColumnDefinition.Raw.forQuoted($t.text); }
+    | k=unreserved_keyword { $id = ColumnDefinition.Raw.forUnquoted(k); }
     ;
 
 // Column identifiers where the comparator is known to be text
@@ -1157,6 +1158,12 @@ ident returns [ColumnIdentifier id]
     | k=unreserved_keyword { $id = ColumnIdentifier.getInterned(k, false); }
     ;
 
+fident returns [FieldIdentifier id]
+    : t=IDENT              { $id = FieldIdentifier.forUnquoted($t.text); }
+    | t=QUOTED_NAME        { $id = FieldIdentifier.forQuoted($t.text); }
+    | k=unreserved_keyword { $id = FieldIdentifier.forUnquoted(k); }
+    ;
+
 // Identifiers that do not refer to columns
 noncol_ident returns [ColumnIdentifier id]
     : t=IDENT              { $id = new ColumnIdentifier($t.text, false); }
@@ -1254,10 +1261,10 @@ collectionLiteral returns [Term.Raw value]
     ;
 
 usertypeLiteral returns [UserTypes.Literal ut]
-    @init{ Map<ColumnIdentifier, Term.Raw> m = new HashMap<ColumnIdentifier, Term.Raw>(); }
+    @init{ Map<FieldIdentifier, Term.Raw> m = new HashMap<>(); }
     @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=noncol_ident ':' v1=term { m.put(k1, v1); } ( ',' kn=noncol_ident ':' vn=term { m.put(kn, vn); } )* '}'
+    : '{' k1=fident ':' v1=term { m.put(k1, v1); } ( ',' kn=fident ':' vn=term { m.put(kn, vn); } )* '}'
     ;
 
 tupleLiteral returns [Tuples.Literal tt]
@@ -1311,17 +1318,17 @@ term returns [Term.Raw term]
     | '(' c=comparatorType ')' t=term  { $term = new TypeCast(c, t); }
     ;
 
-columnOperation[List<Pair<ColumnIdentifier.Raw, Operation.RawUpdate>> operations]
+columnOperation[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations]
     : key=cident columnOperationDifferentiator[operations, key]
     ;
 
-columnOperationDifferentiator[List<Pair<ColumnIdentifier.Raw, Operation.RawUpdate>> operations, ColumnIdentifier.Raw key]
+columnOperationDifferentiator[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations, ColumnDefinition.Raw key]
     : '=' normalColumnOperation[operations, key]
     | '[' k=term ']' collectionColumnOperation[operations, key, k]
-    | '.' field=cident udtColumnOperation[operations, key, field]
+    | '.' field=fident udtColumnOperation[operations, key, field]
     ;
 
-normalColumnOperation[List<Pair<ColumnIdentifier.Raw, Operation.RawUpdate>> operations, ColumnIdentifier.Raw key]
+normalColumnOperation[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations, ColumnDefinition.Raw key]
     : t=term ('+' c=cident )?
       {
           if (c == null)
@@ -1351,21 +1358,21 @@ normalColumnOperation[List<Pair<ColumnIdentifier.Raw, Operation.RawUpdate>> oper
       }
     ;
 
-collectionColumnOperation[List<Pair<ColumnIdentifier.Raw, Operation.RawUpdate>> operations, ColumnIdentifier.Raw key, Term.Raw k]
+collectionColumnOperation[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations, ColumnDefinition.Raw key, Term.Raw k]
     : '=' t=term
       {
           addRawUpdate(operations, key, new Operation.SetElement(k, t));
       }
     ;
 
-udtColumnOperation[List<Pair<ColumnIdentifier.Raw, Operation.RawUpdate>> operations, ColumnIdentifier.Raw key, ColumnIdentifier.Raw field]
+udtColumnOperation[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations, ColumnDefinition.Raw key, FieldIdentifier field]
     : '=' t=term
       {
           addRawUpdate(operations, key, new Operation.SetField(field, t));
       }
     ;
 
-columnCondition[List<Pair<ColumnIdentifier.Raw, ColumnCondition.Raw>> conditions]
+columnCondition[List<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>> conditions]
     // Note: we'll reject duplicates later
     : key=cident
         ( op=relationType t=term { conditions.add(Pair.create(key, ColumnCondition.Raw.simpleCondition(t, op))); }
@@ -1380,7 +1387,7 @@ columnCondition[List<Pair<ColumnIdentifier.Raw, ColumnCondition.Raw>> conditions
                 | marker=inMarker { conditions.add(Pair.create(key, ColumnCondition.Raw.collectionInCondition(element, marker))); }
                 )
             )
-        | '.' field=cident
+        | '.' field=fident
             ( op=relationType t=term { conditions.add(Pair.create(key, ColumnCondition.Raw.udtFieldCondition(t, field, op))); }
             | K_IN
                 ( values=singleColumnInValues { conditions.add(Pair.create(key, ColumnCondition.Raw.udtFieldInCondition(field, values))); }
@@ -1454,8 +1461,8 @@ inMarker returns [AbstractMarker.INRaw marker]
     | ':' name=noncol_ident { $marker = newINBindVariables(name); }
     ;
 
-tupleOfIdentifiers returns [List<ColumnIdentifier.Raw> ids]
-    @init { $ids = new ArrayList<ColumnIdentifier.Raw>(); }
+tupleOfIdentifiers returns [List<ColumnDefinition.Raw> ids]
+    @init { $ids = new ArrayList<ColumnDefinition.Raw>(); }
     : '(' n1=cident { $ids.add(n1); } (',' ni=cident { $ids.add(ni); })* ')'
     ;
 
@@ -1572,7 +1579,7 @@ non_type_ident returns [ColumnIdentifier id]
 
 unreserved_keyword returns [String str]
     : u=unreserved_function_keyword     { $str = u; }
-    | k=(K_TTL | K_COUNT | K_WRITETIME | K_KEY | K_CAST) { $str = $k.text; }
+    | k=(K_TTL | K_COUNT | K_WRITETIME | K_KEY | K_CAST | K_JSON | K_DISTINCT) { $str = $k.text; }
     ;
 
 unreserved_function_keyword returns [String str]
@@ -1608,7 +1615,6 @@ basic_unreserved_keyword returns [String str]
         | K_EXISTS
         | K_CUSTOM
         | K_TRIGGER
-        | K_DISTINCT
         | K_CONTAINS
         | K_STATIC
         | K_FROZEN
@@ -1622,7 +1628,6 @@ basic_unreserved_keyword returns [String str]
         | K_INITCOND
         | K_RETURNS
         | K_LANGUAGE
-        | K_JSON
         | K_CALLED
         | K_INPUT
         | K_LIKE

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/config/ColumnDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/ColumnDefinition.java b/src/java/org/apache/cassandra/config/ColumnDefinition.java
index a900ce7..713d684 100644
--- a/src/java/org/apache/cassandra/config/ColumnDefinition.java
+++ b/src/java/org/apache/cassandra/config/ColumnDefinition.java
@@ -27,11 +27,16 @@ import com.google.common.base.Objects;
 import com.google.common.collect.Collections2;
 
 import org.apache.cassandra.cql3.*;
+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.rows.*;
 import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
-public class ColumnDefinition extends ColumnSpecification implements Comparable<ColumnDefinition>
+public class ColumnDefinition extends ColumnSpecification implements Selectable, Comparable<ColumnDefinition>
 {
     public static final Comparator<Object> asymmetricColumnDataComparator =
         (a, b) -> ((ColumnData) a).column().compareTo((ColumnDefinition) b);
@@ -280,9 +285,15 @@ public class ColumnDefinition extends ColumnSpecification implements Comparable<
         }
         return result;
     }
+
     @Override
     public String toString()
     {
+        return name.toString();
+    }
+
+    public String debugString()
+    {
         return MoreObjects.toStringHelper(this)
                           .add("name", name)
                           .add("type", type)
@@ -419,4 +430,193 @@ public class ColumnDefinition extends ColumnSpecification implements Comparable<
              ? ((CollectionType)type).valueComparator()
              : type;
     }
+
+    public Selector.Factory newSelectorFactory(CFMetaData cfm, AbstractType<?> expectedType, List<ColumnDefinition> defs, VariableSpecifications boundNames) throws InvalidRequestException
+    {
+        return SimpleSelector.newFactory(this, addAndGetIndex(this, defs));
+    }
+
+    public AbstractType<?> getExactTypeIfKnown(String keyspace)
+    {
+        return type;
+    }
+
+    /**
+     * Because Thrift-created tables may have a non-text comparator, we cannot determine the proper 'key' until
+     * we know the comparator. ColumnDefinition.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 abstract class Raw extends Selectable.Raw
+    {
+        /**
+         * Creates a {@code ColumnDefinition.Raw} from an unquoted identifier string.
+         */
+        public static Raw forUnquoted(String text)
+        {
+            return new Literal(text, false);
+        }
+
+        /**
+         * Creates a {@code ColumnDefinition.Raw} from a quoted identifier string.
+         */
+        public static Raw forQuoted(String text)
+        {
+            return new Literal(text, true);
+        }
+
+        /**
+         * Creates a {@code ColumnDefinition.Raw} from a pre-existing {@code ColumnDefinition}
+         * (useful in the rare cases where we already have the column but need
+         * a {@code ColumnDefinition.Raw} for typing purposes).
+         */
+        public static Raw forColumn(ColumnDefinition column)
+        {
+            return new ForColumn(column);
+        }
+
+        /**
+         * Get the identifier corresponding to this raw column, without assuming this is an
+         * existing column (unlike {@link #prepare}).
+         */
+        public abstract ColumnIdentifier getIdentifier(CFMetaData cfm);
+
+        public abstract String rawText();
+
+        @Override
+        public abstract ColumnDefinition prepare(CFMetaData cfm);
+
+        @Override
+        public boolean processesSelection()
+        {
+            return false;
+        }
+
+        @Override
+        public final int hashCode()
+        {
+            return toString().hashCode();
+        }
+
+        @Override
+        public final boolean equals(Object o)
+        {
+            if(!(o instanceof Raw))
+                return false;
+
+            Raw that = (Raw)o;
+            return this.toString().equals(that.toString());
+        }
+
+        private static class Literal extends Raw
+        {
+            private final String text;
+
+            public Literal(String rawText, boolean keepCase)
+            {
+                this.text =  keepCase ? rawText : rawText.toLowerCase(Locale.US);
+            }
+
+            public ColumnIdentifier getIdentifier(CFMetaData cfm)
+            {
+                if (!cfm.isStaticCompactTable())
+                    return ColumnIdentifier.getInterned(text, true);
+
+                AbstractType<?> thriftColumnNameType = cfm.thriftColumnNameType();
+                if (thriftColumnNameType instanceof UTF8Type)
+                    return ColumnIdentifier.getInterned(text, true);
+
+                // We have a Thrift-created table with a non-text comparator. Check if we have a match column, otherwise assume we should use
+                // thriftColumnNameType
+                ByteBuffer bufferName = ByteBufferUtil.bytes(text);
+                for (ColumnDefinition def : cfm.allColumns())
+                {
+                    if (def.name.bytes.equals(bufferName))
+                        return def.name;
+                }
+                return ColumnIdentifier.getInterned(thriftColumnNameType.fromString(text), text);
+            }
+
+            public ColumnDefinition prepare(CFMetaData cfm)
+            {
+                if (!cfm.isStaticCompactTable())
+                    return find(cfm);
+
+                AbstractType<?> thriftColumnNameType = cfm.thriftColumnNameType();
+                if (thriftColumnNameType instanceof UTF8Type)
+                    return find(cfm);
+
+                // We have a Thrift-created table with a non-text comparator. Check if we have a match column, otherwise assume we should use
+                // thriftColumnNameType
+                ByteBuffer bufferName = ByteBufferUtil.bytes(text);
+                for (ColumnDefinition def : cfm.allColumns())
+                {
+                    if (def.name.bytes.equals(bufferName))
+                        return def;
+                }
+                return find(thriftColumnNameType.fromString(text), cfm);
+            }
+
+            private ColumnDefinition find(CFMetaData cfm)
+            {
+                return find(ByteBufferUtil.bytes(text), cfm);
+            }
+
+            private ColumnDefinition find(ByteBuffer id, CFMetaData cfm)
+            {
+                ColumnDefinition def = cfm.getColumnDefinition(id);
+                if (def == null)
+                    throw new InvalidRequestException(String.format("Undefined column name %s", toString()));
+                return def;
+            }
+
+            public String rawText()
+            {
+                return text;
+            }
+
+            @Override
+            public String toString()
+            {
+                return ColumnIdentifier.maybeQuote(text);
+            }
+        }
+
+        // Use internally in the rare case where we need a ColumnDefinition.Raw for type-checking but
+        // actually already have the column itself.
+        private static class ForColumn extends Raw
+        {
+            private final ColumnDefinition column;
+
+            private ForColumn(ColumnDefinition column)
+            {
+                this.column = column;
+            }
+
+            public ColumnIdentifier getIdentifier(CFMetaData cfm)
+            {
+                return column.name;
+            }
+
+            public ColumnDefinition prepare(CFMetaData cfm)
+            {
+                assert cfm.getColumnDefinition(column.name) != null; // Sanity check that we're not doing something crazy
+                return column;
+            }
+
+            public String rawText()
+            {
+                return column.name.toString();
+            }
+
+            @Override
+            public String toString()
+            {
+                return column.name.toCQLString();
+            }
+        }
+    }
+
+
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/config/ViewDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/ViewDefinition.java b/src/java/org/apache/cassandra/config/ViewDefinition.java
index 9cf8476..33cc175 100644
--- a/src/java/org/apache/cassandra/config/ViewDefinition.java
+++ b/src/java/org/apache/cassandra/config/ViewDefinition.java
@@ -139,8 +139,8 @@ public class ViewDefinition
 
         // convert whereClause to Relations, rename ids in Relations, then convert back to whereClause
         List<Relation> relations = whereClauseToRelations(whereClause);
-        ColumnIdentifier.Raw fromRaw = new ColumnIdentifier.Literal(from.toString(), true);
-        ColumnIdentifier.Raw toRaw = new ColumnIdentifier.Literal(to.toString(), true);
+        ColumnDefinition.Raw fromRaw = ColumnDefinition.Raw.forQuoted(from.toString());
+        ColumnDefinition.Raw toRaw = ColumnDefinition.Raw.forQuoted(to.toString());
         List<Relation> newRelations = relations.stream()
                 .map(r -> r.renameIdentifier(fromRaw, toRaw))
                 .collect(Collectors.toList());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/AbstractMarker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/AbstractMarker.java b/src/java/org/apache/cassandra/cql3/AbstractMarker.java
index 21644c8..3689ed1 100644
--- a/src/java/org/apache/cassandra/cql3/AbstractMarker.java
+++ b/src/java/org/apache/cassandra/cql3/AbstractMarker.java
@@ -21,6 +21,7 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.CollectionType;
 import org.apache.cassandra.db.marshal.ListType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -89,11 +90,17 @@ public abstract class AbstractMarker extends Term.NonTerminal
             return new Constants.Marker(bindIndex, receiver);
         }
 
+        @Override
         public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
         {
             return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
         }
 
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            return null;
+        }
+
         @Override
         public String getText()
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/ColumnCondition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ColumnCondition.java b/src/java/org/apache/cassandra/cql3/ColumnCondition.java
index 255dd0e..304f8bc 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnCondition.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnCondition.java
@@ -42,14 +42,14 @@ public class ColumnCondition
     private final Term collectionElement;
 
     // For UDT, when testing the equality of a specific field, null otherwise.
-    private final ColumnIdentifier field;
+    private final FieldIdentifier field;
 
     private final Term value;  // a single value or a marker for a list of IN values
     private final List<Term> inValues;
 
     public final Operator operator;
 
-    private ColumnCondition(ColumnDefinition column, Term collectionElement, ColumnIdentifier field, Term value, List<Term> inValues, Operator op)
+    private ColumnCondition(ColumnDefinition column, Term collectionElement, FieldIdentifier field, Term value, List<Term> inValues, Operator op)
     {
         this.column = column;
         this.collectionElement = collectionElement;
@@ -73,7 +73,7 @@ public class ColumnCondition
         return new ColumnCondition(column, collectionElement, null, value, null, op);
     }
 
-    public static ColumnCondition condition(ColumnDefinition column, ColumnIdentifier udtField, Term value, Operator op)
+    public static ColumnCondition condition(ColumnDefinition column, FieldIdentifier udtField, Term value, Operator op)
     {
         return new ColumnCondition(column, null, udtField, value, null, op);
     }
@@ -88,7 +88,7 @@ public class ColumnCondition
         return new ColumnCondition(column, collectionElement, null, null, inValues, Operator.IN);
     }
 
-    public static ColumnCondition inCondition(ColumnDefinition column, ColumnIdentifier udtField, List<Term> inValues)
+    public static ColumnCondition inCondition(ColumnDefinition column, FieldIdentifier udtField, List<Term> inValues)
     {
         return new ColumnCondition(column, null, udtField, null, inValues, Operator.IN);
     }
@@ -103,7 +103,7 @@ public class ColumnCondition
         return new ColumnCondition(column, collectionElement, null, inMarker, null, Operator.IN);
     }
 
-    public static ColumnCondition inCondition(ColumnDefinition column, ColumnIdentifier udtField, Term inMarker)
+    public static ColumnCondition inCondition(ColumnDefinition column, FieldIdentifier udtField, Term inMarker)
     {
         return new ColumnCondition(column, null, udtField, inMarker, null, Operator.IN);
     }
@@ -693,7 +693,7 @@ public class ColumnCondition
     /** A condition on a UDT field. IN operators are not supported here, see UDTFieldAccessInBound. */
     static class UDTFieldAccessBound extends Bound
     {
-        public final ColumnIdentifier field;
+        public final FieldIdentifier field;
         public final ByteBuffer value;
 
         private UDTFieldAccessBound(ColumnCondition condition, QueryOptions options) throws InvalidRequestException
@@ -714,7 +714,7 @@ public class ColumnCondition
             ByteBuffer cellValue;
             if (column.type.isMultiCell())
             {
-                Cell cell = getCell(row, column, userType.cellPathForField(field.bytes));
+                Cell cell = getCell(row, column, userType.cellPathForField(field));
                 cellValue = cell == null ? null : cell.value();
             }
             else
@@ -731,7 +731,7 @@ public class ColumnCondition
     /** An IN condition on a UDT field.  For example: IF user.name IN ('a', 'b') */
     static class UDTFieldAccessInBound extends Bound
     {
-        public final ColumnIdentifier field;
+        public final FieldIdentifier field;
         public final List<ByteBuffer> inValues;
 
         private UDTFieldAccessInBound(ColumnCondition condition, QueryOptions options) throws InvalidRequestException
@@ -759,7 +759,7 @@ public class ColumnCondition
             ByteBuffer cellValue;
             if (column.type.isMultiCell())
             {
-                Cell cell = getCell(row, column, userType.cellPathForField(field.bytes));
+                Cell cell = getCell(row, column, userType.cellPathForField(field));
                 cellValue = cell == null ? null : cell.value();
             }
             else
@@ -889,12 +889,12 @@ public class ColumnCondition
         private final Term.Raw collectionElement;
 
         // Can be null, only used with the syntax "IF udt.field = ..." (in which case it's 'field')
-        private final ColumnIdentifier.Raw udtField;
+        private final FieldIdentifier udtField;
 
         private final Operator operator;
 
         private Raw(Term.Raw value, List<Term.Raw> inValues, AbstractMarker.INRaw inMarker, Term.Raw collectionElement,
-                    ColumnIdentifier.Raw udtField, Operator op)
+                    FieldIdentifier udtField, Operator op)
         {
             this.value = value;
             this.inValues = inValues;
@@ -941,19 +941,19 @@ public class ColumnCondition
         }
 
         /** A condition on a UDT field. For example: "IF col.field = 'foo'" */
-        public static Raw udtFieldCondition(Term.Raw value, ColumnIdentifier.Raw udtField, Operator op)
+        public static Raw udtFieldCondition(Term.Raw value, FieldIdentifier udtField, Operator op)
         {
             return new Raw(value, null, null, null, udtField, op);
         }
 
         /** An IN condition on a collection element. For example: "IF col.field IN ('foo', 'bar', ...)" */
-        public static Raw udtFieldInCondition(ColumnIdentifier.Raw udtField, List<Term.Raw> inValues)
+        public static Raw udtFieldInCondition(FieldIdentifier udtField, List<Term.Raw> inValues)
         {
             return new Raw(null, inValues, null, null, udtField, Operator.IN);
         }
 
         /** An IN condition on a collection element with a single marker. For example: "IF col.field IN ?" */
-        public static Raw udtFieldInCondition(ColumnIdentifier.Raw udtField, AbstractMarker.INRaw inMarker)
+        public static Raw udtFieldInCondition(FieldIdentifier udtField, AbstractMarker.INRaw inMarker)
         {
             return new Raw(null, null, inMarker, null, udtField, Operator.IN);
         }
@@ -1001,26 +1001,24 @@ public class ColumnCondition
             else if (udtField != null)
             {
                 UserType userType = (UserType) receiver.type;
-                ColumnIdentifier fieldIdentifier = udtField.prepare(cfm);
-
-                int fieldPosition = userType.fieldPosition(fieldIdentifier);
+                int fieldPosition = userType.fieldPosition(udtField);
                 if (fieldPosition == -1)
-                    throw new InvalidRequestException(String.format("Unknown field %s for column %s", fieldIdentifier, receiver.name));
+                    throw new InvalidRequestException(String.format("Unknown field %s for column %s", udtField, receiver.name));
 
                 ColumnSpecification fieldReceiver = UserTypes.fieldSpecOf(receiver, fieldPosition);
                 if (operator == Operator.IN)
                 {
                     if (inValues == null)
-                        return ColumnCondition.inCondition(receiver, udtField.prepare(cfm), inMarker.prepare(keyspace, fieldReceiver));
+                        return ColumnCondition.inCondition(receiver, udtField, inMarker.prepare(keyspace, fieldReceiver));
 
                     List<Term> terms = new ArrayList<>(inValues.size());
                     for (Term.Raw value : inValues)
                         terms.add(value.prepare(keyspace, fieldReceiver));
-                    return ColumnCondition.inCondition(receiver, udtField.prepare(cfm), terms);
+                    return ColumnCondition.inCondition(receiver, udtField, terms);
                 }
                 else
                 {
-                    return ColumnCondition.condition(receiver, udtField.prepare(cfm), value.prepare(keyspace, fieldReceiver), operator);
+                    return ColumnCondition.condition(receiver, udtField, value.prepare(keyspace, fieldReceiver), operator);
                 }
             }
             else

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/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 f202145..467c672 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
@@ -31,8 +31,6 @@ import org.apache.cassandra.cache.IMeasurableMemory;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 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.UTF8Type;
@@ -44,7 +42,7 @@ import org.apache.cassandra.utils.memory.AbstractAllocator;
  * 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, Comparable<ColumnIdentifier>
+public class ColumnIdentifier implements IMeasurableMemory, Comparable<ColumnIdentifier>
 {
     private static final Pattern PATTERN_DOUBLE_QUOTE = Pattern.compile("\"", Pattern.LITERAL);
     private static final String ESCAPED_DOUBLE_QUOTE = Matcher.quoteReplacement("\"\"");
@@ -157,7 +155,7 @@ public class ColumnIdentifier extends Selectable implements IMeasurableMemory, C
 
     /**
      * Returns a string representation of the identifier that is safe to use directly in CQL queries.
-     * In necessary, the string will be double-quoted, and any quotes inside the string will be escaped.
+     * If necessary, the string will be double-quoted, and any quotes inside the string will be escaped.
      */
     public String toCQLString()
     {
@@ -183,15 +181,6 @@ public class ColumnIdentifier extends Selectable implements IMeasurableMemory, C
         return interned ? this : new ColumnIdentifier(allocator.clone(bytes), text, false);
     }
 
-    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, addAndGetIndex(def, defs));
-    }
-
     public int compareTo(ColumnIdentifier that)
     {
         int c = Long.compare(this.prefixComparison, that.prefixComparison);
@@ -202,134 +191,6 @@ public class ColumnIdentifier extends Selectable implements IMeasurableMemory, C
         return ByteBufferUtil.compareUnsigned(this.bytes, that.bytes);
     }
 
-    /**
-     * 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 interface Raw extends Selectable.Raw
-    {
-
-        public ColumnIdentifier prepare(CFMetaData cfm);
-
-        /**
-         * Returns a string representation of the identifier that is safe to use directly in CQL queries.
-         * In necessary, the string will be double-quoted, and any quotes inside the string will be escaped.
-         */
-        public String toCQLString();
-    }
-
-    public static class Literal implements Raw
-    {
-        private final String rawText;
-        private final String text;
-
-        public Literal(String rawText, boolean keepCase)
-        {
-            this.rawText = rawText;
-            this.text =  keepCase ? rawText : rawText.toLowerCase(Locale.US);
-        }
-
-        public ColumnIdentifier prepare(CFMetaData cfm)
-        {
-            if (!cfm.isStaticCompactTable())
-                return getInterned(text, true);
-
-            AbstractType<?> thriftColumnNameType = cfm.thriftColumnNameType();
-            if (thriftColumnNameType instanceof UTF8Type)
-                return getInterned(text, true);
-
-            // We have a Thrift-created table with a non-text comparator. Check if we have a match column, otherwise assume we should use
-            // thriftColumnNameType
-            ByteBuffer bufferName = ByteBufferUtil.bytes(text);
-            for (ColumnDefinition def : cfm.allColumns())
-            {
-                if (def.name.bytes.equals(bufferName))
-                    return def.name;
-            }
-            return getInterned(thriftColumnNameType.fromString(rawText), text);
-        }
-
-        public boolean processesSelection()
-        {
-            return false;
-        }
-
-        @Override
-        public final int hashCode()
-        {
-            return text.hashCode();
-        }
-
-        @Override
-        public final boolean equals(Object o)
-        {
-            if(!(o instanceof Literal))
-                return false;
-
-            Literal that = (Literal) o;
-            return text.equals(that.text);
-        }
-
-        @Override
-        public String toString()
-        {
-            return text;
-        }
-
-        public String toCQLString()
-        {
-            return maybeQuote(text);
-        }
-    }
-
-    public static class ColumnIdentifierValue implements Raw
-    {
-        private final ColumnIdentifier identifier;
-
-        public ColumnIdentifierValue(ColumnIdentifier identifier)
-        {
-            this.identifier = identifier;
-        }
-
-        public ColumnIdentifier prepare(CFMetaData cfm)
-        {
-            return identifier;
-        }
-
-        public boolean processesSelection()
-        {
-            return false;
-        }
-
-        @Override
-        public final int hashCode()
-        {
-            return identifier.hashCode();
-        }
-
-        @Override
-        public final boolean equals(Object o)
-        {
-            if(!(o instanceof ColumnIdentifierValue))
-                return false;
-            ColumnIdentifierValue that = (ColumnIdentifierValue) o;
-            return identifier.equals(that.identifier);
-        }
-
-        @Override
-        public String toString()
-        {
-            return identifier.toString();
-        }
-
-        public String toCQLString()
-        {
-            return maybeQuote(identifier.text);
-        }
-    }
-
     @VisibleForTesting
     public static String maybeQuote(String text)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/Constants.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Constants.java b/src/java/org/apache/cassandra/cql3/Constants.java
index 2efce59..6589d85 100644
--- a/src/java/org/apache/cassandra/cql3/Constants.java
+++ b/src/java/org/apache/cassandra/cql3/Constants.java
@@ -23,11 +23,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.BytesType;
-import org.apache.cassandra.db.marshal.CounterColumnType;
-import org.apache.cassandra.db.marshal.LongType;
-import org.apache.cassandra.db.marshal.ReversedType;
+import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -41,7 +37,7 @@ public abstract class Constants
 
     public enum Type
     {
-        STRING, INTEGER, UUID, FLOAT, DATE, TIME, BOOLEAN, HEX;
+        STRING, INTEGER, UUID, FLOAT, BOOLEAN, HEX;
     }
 
     public static final Value UNSET_VALUE = new Value(ByteBufferUtil.UNSET_BYTE_BUFFER);
@@ -67,6 +63,11 @@ public abstract class Constants
         {
             return "NULL";
         }
+
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            return null;
+        }
     }
 
     public static final NullLiteral NULL_LITERAL = new NullLiteral();
@@ -159,6 +160,7 @@ public abstract class Constants
             }
         }
 
+        @Override
         public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
         {
             CQL3Type receiverType = receiver.type.asCQL3Type();
@@ -237,6 +239,16 @@ public abstract class Constants
             return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
         }
 
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            // Most constant are valid for more than one type (the extreme example being integer constants, which can
+            // be use for any numerical type, including date, time, ...) so they don't have an exact type. And in fact,
+            // for good or bad, any literal is valid for custom types, so we can never claim an exact type.
+            // But really, the reason it's fine to return null here is that getExactTypeIfKnown is only used to
+            // implement testAssignment() in Selectable and that method is overriden above.
+            return null;
+        }
+
         public String getRawText()
         {
             return text;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/FieldIdentifier.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/FieldIdentifier.java b/src/java/org/apache/cassandra/cql3/FieldIdentifier.java
new file mode 100644
index 0000000..5e0601c
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/FieldIdentifier.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3;
+
+import java.util.Locale;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.serializers.MarshalException;
+
+/**
+ * Identifies a field in a UDT.
+ */
+public class FieldIdentifier
+{
+    public final ByteBuffer bytes;
+
+    public FieldIdentifier(ByteBuffer bytes)
+    {
+        this.bytes = bytes;
+    }
+
+    /**
+     * Creates a {@code FieldIdentifier} from an unquoted identifier string.
+     */
+    public static FieldIdentifier forUnquoted(String text)
+    {
+        return new FieldIdentifier(convert(text.toLowerCase(Locale.US)));
+    }
+
+    /**
+     * Creates a {@code FieldIdentifier} from a quoted identifier string.
+     */
+    public static FieldIdentifier forQuoted(String text)
+    {
+        return new FieldIdentifier(convert(text));
+    }
+
+    /**
+     * Creates a {@code FieldIdentifier} from an internal string.
+     */
+    public static FieldIdentifier forInternalString(String text)
+    {
+        // If we store a field internally, we consider it as quoted, i.e. we preserve
+        // whatever case the text has.
+        return forQuoted(text);
+    }
+
+    private static ByteBuffer convert(String text)
+    {
+        try
+        {
+            return UTF8Type.instance.decompose(text);
+        }
+        catch (MarshalException e)
+        {
+            throw new SyntaxException(String.format("For field name %s: %s", text, e.getMessage()));
+        }
+    }
+
+    @Override
+    public String toString()
+    {
+        return UTF8Type.instance.compose(bytes);
+    }
+
+    @Override
+    public final int hashCode()
+    {
+        return bytes.hashCode();
+    }
+
+    @Override
+    public final boolean equals(Object o)
+    {
+        if(!(o instanceof FieldIdentifier))
+            return false;
+        FieldIdentifier that = (FieldIdentifier)o;
+        return this.bytes.equals(that.bytes);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/Json.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Json.java b/src/java/org/apache/cassandra/cql3/Json.java
index ab02fb6..298cde7 100644
--- a/src/java/org/apache/cassandra/cql3/Json.java
+++ b/src/java/org/apache/cassandra/cql3/Json.java
@@ -24,6 +24,7 @@ import java.util.*;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.serializers.MarshalException;
@@ -180,6 +181,11 @@ public class Json
             return TestResult.NOT_ASSIGNABLE;
         }
 
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            return null;
+        }
+
         public String getText()
         {
             return term.toString();
@@ -212,6 +218,11 @@ public class Json
             return TestResult.WEAKLY_ASSIGNABLE;
         }
 
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            return null;
+        }
+
         public String getText()
         {
             return marker.toString();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/Lists.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Lists.java b/src/java/org/apache/cassandra/cql3/Lists.java
index aab9c8f..ad0af6d 100644
--- a/src/java/org/apache/cassandra/cql3/Lists.java
+++ b/src/java/org/apache/cassandra/cql3/Lists.java
@@ -29,6 +29,7 @@ import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.db.marshal.ListType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -113,6 +114,18 @@ public abstract class Lists
             return AssignmentTestable.TestResult.testAll(keyspace, valueSpec, elements);
         }
 
+        @Override
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            for (Term.Raw term : elements)
+            {
+                AbstractType<?> type = term.getExactTypeIfKnown(keyspace);
+                if (type != null)
+                    return ListType.getInstance(type, false);
+            }
+            return null;
+        }
+
         public String getText()
         {
             return elements.stream().map(Term.Raw::getText).collect(Collectors.joining(", ", "[", "]"));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/Maps.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Maps.java b/src/java/org/apache/cassandra/cql3/Maps.java
index 4772369..952bff0 100644
--- a/src/java/org/apache/cassandra/cql3/Maps.java
+++ b/src/java/org/apache/cassandra/cql3/Maps.java
@@ -27,7 +27,7 @@ import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.rows.*;
-import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.serializers.CollectionSerializer;
 import org.apache.cassandra.serializers.MarshalException;
@@ -127,6 +127,23 @@ public abstract class Maps
             return res;
         }
 
+        @Override
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            AbstractType<?> keyType = null;
+            AbstractType<?> valueType = null;
+            for (Pair<Term.Raw, Term.Raw> entry : entries)
+            {
+                if (keyType == null)
+                    keyType = entry.left.getExactTypeIfKnown(keyspace);
+                if (valueType == null)
+                    valueType = entry.right.getExactTypeIfKnown(keyspace);
+                if (keyType != null && valueType != null)
+                    return MapType.getInstance(keyType, valueType, false);
+            }
+            return null;
+        }
+
         public String getText()
         {
             return entries.stream()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/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 2c1f97b..01f2a12 100644
--- a/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
+++ b/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
@@ -46,7 +46,7 @@ import static org.apache.cassandra.cql3.statements.RequestValidations.invalidReq
  */
 public class MultiColumnRelation extends Relation
 {
-    private final List<ColumnIdentifier.Raw> entities;
+    private final List<ColumnDefinition.Raw> entities;
 
     /** A Tuples.Literal or Tuples.Raw marker */
     private final Term.MultiColumnRaw valuesOrMarker;
@@ -56,7 +56,7 @@ public class MultiColumnRelation extends Relation
 
     private final Tuples.INRaw inMarker;
 
-    private MultiColumnRelation(List<ColumnIdentifier.Raw> entities, Operator relationType, Term.MultiColumnRaw valuesOrMarker, List<? extends Term.MultiColumnRaw> inValues, Tuples.INRaw inMarker)
+    private MultiColumnRelation(List<ColumnDefinition.Raw> entities, Operator relationType, Term.MultiColumnRaw valuesOrMarker, List<? extends Term.MultiColumnRaw> inValues, Tuples.INRaw inMarker)
     {
         this.entities = entities;
         this.relationType = relationType;
@@ -76,7 +76,7 @@ public class MultiColumnRelation extends Relation
      * @param valuesOrMarker a Tuples.Literal instance or a Tuples.Raw marker
      * @return a new <code>MultiColumnRelation</code> instance
      */
-    public static MultiColumnRelation createNonInRelation(List<ColumnIdentifier.Raw> entities, Operator relationType, Term.MultiColumnRaw valuesOrMarker)
+    public static MultiColumnRelation createNonInRelation(List<ColumnDefinition.Raw> entities, Operator relationType, Term.MultiColumnRaw valuesOrMarker)
     {
         assert relationType != Operator.IN;
         return new MultiColumnRelation(entities, relationType, valuesOrMarker, null, null);
@@ -89,7 +89,7 @@ public class MultiColumnRelation extends Relation
      * @param inValues a list of Tuples.Literal instances or a Tuples.Raw markers
      * @return a new <code>MultiColumnRelation</code> instance
      */
-    public static MultiColumnRelation createInRelation(List<ColumnIdentifier.Raw> entities, List<? extends Term.MultiColumnRaw> inValues)
+    public static MultiColumnRelation createInRelation(List<ColumnDefinition.Raw> entities, List<? extends Term.MultiColumnRaw> inValues)
     {
         return new MultiColumnRelation(entities, Operator.IN, null, inValues, null);
     }
@@ -101,12 +101,12 @@ public class MultiColumnRelation extends Relation
      * @param inMarker a single IN marker
      * @return a new <code>MultiColumnRelation</code> instance
      */
-    public static MultiColumnRelation createSingleMarkerInRelation(List<ColumnIdentifier.Raw> entities, Tuples.INRaw inMarker)
+    public static MultiColumnRelation createSingleMarkerInRelation(List<ColumnDefinition.Raw> entities, Tuples.INRaw inMarker)
     {
         return new MultiColumnRelation(entities, Operator.IN, null, null, inMarker);
     }
 
-    public List<ColumnIdentifier.Raw> getEntities()
+    public List<ColumnDefinition.Raw> getEntities()
     {
         return entities;
     }
@@ -203,9 +203,9 @@ public class MultiColumnRelation extends Relation
     {
         List<ColumnDefinition> names = new ArrayList<>(getEntities().size());
         int previousPosition = -1;
-        for (ColumnIdentifier.Raw raw : getEntities())
+        for (ColumnDefinition.Raw raw : getEntities())
         {
-            ColumnDefinition def = toColumnDefinition(cfm, raw);
+            ColumnDefinition def = raw.prepare(cfm);
             checkTrue(def.isClusteringColumn(), "Multi-column relations can only be applied to clustering columns but was applied to: %s", def.name);
             checkFalse(names.contains(def), "Column \"%s\" appeared twice in a relation: %s", def.name, this);
 
@@ -219,12 +219,12 @@ public class MultiColumnRelation extends Relation
         return names;
     }
 
-    public Relation renameIdentifier(ColumnIdentifier.Raw from, ColumnIdentifier.Raw to)
+    public Relation renameIdentifier(ColumnDefinition.Raw from, ColumnDefinition.Raw to)
     {
         if (!entities.contains(from))
             return this;
 
-        List<ColumnIdentifier.Raw> newEntities = entities.stream().map(e -> e.equals(from) ? to : e).collect(Collectors.toList());
+        List<ColumnDefinition.Raw> newEntities = entities.stream().map(e -> e.equals(from) ? to : e).collect(Collectors.toList());
         return new MultiColumnRelation(newEntities, operator(), valuesOrMarker, inValues, inMarker);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/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 0147859..7d7d7b3 100644
--- a/src/java/org/apache/cassandra/cql3/Operation.java
+++ b/src/java/org/apache/cassandra/cql3/Operation.java
@@ -17,6 +17,7 @@
  */
 package org.apache.cassandra.cql3;
 
+import java.nio.ByteBuffer;
 import java.util.List;
 
 import org.apache.cassandra.config.CFMetaData;
@@ -133,7 +134,7 @@ public abstract class Operation
         /**
          * The name of the column affected by this delete operation.
          */
-        public ColumnIdentifier.Raw affectedColumn();
+        public ColumnDefinition.Raw affectedColumn();
 
         /**
          * This method validates the operation (i.e. validate it is well typed)
@@ -248,10 +249,10 @@ public abstract class Operation
 
     public static class SetField implements RawUpdate
     {
-        private final ColumnIdentifier.Raw field;
+        private final FieldIdentifier field;
         private final Term.Raw value;
 
-        public SetField(ColumnIdentifier.Raw field, Term.Raw value)
+        public SetField(FieldIdentifier field, Term.Raw value)
         {
             this.field = field;
             this.value = value;
@@ -264,13 +265,12 @@ public abstract class Operation
             else if (!receiver.type.isMultiCell())
                 throw new InvalidRequestException(String.format("Invalid operation (%s) for frozen UDT column %s", toString(receiver), receiver.name));
 
-            ColumnIdentifier fieldIdentifier = field.prepare(cfm);
-            int fieldPosition = ((UserType) receiver.type).fieldPosition(fieldIdentifier);
+            int fieldPosition = ((UserType) receiver.type).fieldPosition(field);
             if (fieldPosition == -1)
-                throw new InvalidRequestException(String.format("UDT column %s does not have a field named %s", receiver.name, fieldIdentifier));
+                throw new InvalidRequestException(String.format("UDT column %s does not have a field named %s", receiver.name, field));
 
             Term val = value.prepare(cfm.ksName, UserTypes.fieldSpecOf(receiver, fieldPosition));
-            return new UserTypes.SetterByField(receiver, fieldIdentifier, val);
+            return new UserTypes.SetterByField(receiver, field, val);
         }
 
         protected String toString(ColumnSpecification column)
@@ -414,14 +414,14 @@ public abstract class Operation
 
     public static class ColumnDeletion implements RawDeletion
     {
-        private final ColumnIdentifier.Raw id;
+        private final ColumnDefinition.Raw id;
 
-        public ColumnDeletion(ColumnIdentifier.Raw id)
+        public ColumnDeletion(ColumnDefinition.Raw id)
         {
             this.id = id;
         }
 
-        public ColumnIdentifier.Raw affectedColumn()
+        public ColumnDefinition.Raw affectedColumn()
         {
             return id;
         }
@@ -435,16 +435,16 @@ public abstract class Operation
 
     public static class ElementDeletion implements RawDeletion
     {
-        private final ColumnIdentifier.Raw id;
+        private final ColumnDefinition.Raw id;
         private final Term.Raw element;
 
-        public ElementDeletion(ColumnIdentifier.Raw id, Term.Raw element)
+        public ElementDeletion(ColumnDefinition.Raw id, Term.Raw element)
         {
             this.id = id;
             this.element = element;
         }
 
-        public ColumnIdentifier.Raw affectedColumn()
+        public ColumnDefinition.Raw affectedColumn()
         {
             return id;
         }
@@ -474,16 +474,16 @@ public abstract class Operation
 
     public static class FieldDeletion implements RawDeletion
     {
-        private final ColumnIdentifier.Raw id;
-        private final ColumnIdentifier.Raw field;
+        private final ColumnDefinition.Raw id;
+        private final FieldIdentifier field;
 
-        public FieldDeletion(ColumnIdentifier.Raw id, ColumnIdentifier.Raw field)
+        public FieldDeletion(ColumnDefinition.Raw id, FieldIdentifier field)
         {
             this.id = id;
             this.field = field;
         }
 
-        public ColumnIdentifier.Raw affectedColumn()
+        public ColumnDefinition.Raw affectedColumn()
         {
             return id;
         }
@@ -495,11 +495,10 @@ public abstract class Operation
             else if (!receiver.type.isMultiCell())
                 throw new InvalidRequestException(String.format("Frozen UDT column %s does not support field deletions", receiver.name));
 
-            ColumnIdentifier fieldIdentifier = field.prepare(cfm);
-            if (((UserType) receiver.type).fieldPosition(fieldIdentifier) == -1)
-                throw new InvalidRequestException(String.format("UDT column %s does not have a field named %s", receiver.name, fieldIdentifier));
+            if (((UserType) receiver.type).fieldPosition(field) == -1)
+                throw new InvalidRequestException(String.format("UDT column %s does not have a field named %s", receiver.name, field));
 
-            return new UserTypes.DeleterByField(receiver, fieldIdentifier);
+            return new UserTypes.DeleterByField(receiver, field);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/Relation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Relation.java b/src/java/org/apache/cassandra/cql3/Relation.java
index bc986f6..097b88e 100644
--- a/src/java/org/apache/cassandra/cql3/Relation.java
+++ b/src/java/org/apache/cassandra/cql3/Relation.java
@@ -25,12 +25,11 @@ import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.restrictions.Restriction;
 import org.apache.cassandra.cql3.statements.Bound;
 import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.UnrecognizedEntityException;
 
 import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
 
-public abstract class Relation {
-
+public abstract class Relation
+{
     protected Operator relationType;
 
     public Operator operator()
@@ -261,31 +260,11 @@ public abstract class Relation {
     }
 
     /**
-     * Converts the specified entity into a column definition.
-     *
-     * @param cfm the column family meta data
-     * @param entity the entity to convert
-     * @return the column definition corresponding to the specified entity
-     * @throws InvalidRequestException if the entity cannot be recognized
-     */
-    protected final ColumnDefinition toColumnDefinition(CFMetaData cfm,
-                                                        ColumnIdentifier.Raw entity) throws InvalidRequestException
-    {
-        ColumnIdentifier identifier = entity.prepare(cfm);
-        ColumnDefinition def = cfm.getColumnDefinition(identifier);
-
-        if (def == null)
-            throw new UnrecognizedEntityException(identifier, this);
-
-        return def;
-    }
-
-    /**
      * Renames an identifier in this Relation, if applicable.
      * @param from the old identifier
      * @param to the new identifier
      * @return this object, if the old identifier is not in the set of entities that this relation covers; otherwise
      *         a new Relation with "from" replaced by "to" is returned.
      */
-    public abstract Relation renameIdentifier(ColumnIdentifier.Raw from, ColumnIdentifier.Raw to);
+    public abstract Relation renameIdentifier(ColumnDefinition.Raw from, ColumnDefinition.Raw to);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/Sets.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Sets.java b/src/java/org/apache/cassandra/cql3/Sets.java
index 622bb23..e8617aa 100644
--- a/src/java/org/apache/cassandra/cql3/Sets.java
+++ b/src/java/org/apache/cassandra/cql3/Sets.java
@@ -122,6 +122,18 @@ public abstract class Sets
             return AssignmentTestable.TestResult.testAll(keyspace, valueSpec, elements);
         }
 
+        @Override
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            for (Term.Raw term : elements)
+            {
+                AbstractType<?> type = term.getExactTypeIfKnown(keyspace);
+                if (type != null)
+                    return SetType.getInstance(type, false);
+            }
+            return null;
+        }
+
         public String getText()
         {
             return elements.stream().map(Term.Raw::getText).collect(Collectors.joining(", ", "{", "}"));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/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 3cb07b9..07232d2 100644
--- a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
+++ b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
@@ -43,12 +43,12 @@ import static org.apache.cassandra.cql3.statements.RequestValidations.invalidReq
  */
 public final class SingleColumnRelation extends Relation
 {
-    private final ColumnIdentifier.Raw entity;
+    private final ColumnDefinition.Raw entity;
     private final Term.Raw mapKey;
     private final Term.Raw value;
     private final List<Term.Raw> inValues;
 
-    private SingleColumnRelation(ColumnIdentifier.Raw entity, Term.Raw mapKey, Operator type, Term.Raw value, List<Term.Raw> inValues)
+    private SingleColumnRelation(ColumnDefinition.Raw entity, Term.Raw mapKey, Operator type, Term.Raw value, List<Term.Raw> inValues)
     {
         this.entity = entity;
         this.mapKey = mapKey;
@@ -68,7 +68,7 @@ public final 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.Raw entity, Term.Raw mapKey, Operator type, Term.Raw value)
+    public SingleColumnRelation(ColumnDefinition.Raw entity, Term.Raw mapKey, Operator type, Term.Raw value)
     {
         this(entity, mapKey, type, value, null);
     }
@@ -80,7 +80,7 @@ public final 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.Raw entity, Operator type, Term.Raw value)
+    public SingleColumnRelation(ColumnDefinition.Raw entity, Operator type, Term.Raw value)
     {
         this(entity, null, type, value);
     }
@@ -95,12 +95,12 @@ public final class SingleColumnRelation extends Relation
         return inValues;
     }
 
-    public static SingleColumnRelation createInRelation(ColumnIdentifier.Raw entity, List<Term.Raw> inValues)
+    public static SingleColumnRelation createInRelation(ColumnDefinition.Raw entity, List<Term.Raw> inValues)
     {
         return new SingleColumnRelation(entity, null, Operator.IN, null, inValues);
     }
 
-    public ColumnIdentifier.Raw getEntity()
+    public ColumnDefinition.Raw getEntity()
     {
         return entity;
     }
@@ -134,7 +134,7 @@ public final class SingleColumnRelation extends Relation
         }
     }
 
-    public Relation renameIdentifier(ColumnIdentifier.Raw from, ColumnIdentifier.Raw to)
+    public Relation renameIdentifier(ColumnDefinition.Raw from, ColumnDefinition.Raw to)
     {
         return entity.equals(from)
                ? new SingleColumnRelation(to, mapKey, operator(), value, inValues)
@@ -158,7 +158,7 @@ public final class SingleColumnRelation extends Relation
     protected Restriction newEQRestriction(CFMetaData cfm,
                                            VariableSpecifications boundNames) throws InvalidRequestException
     {
-        ColumnDefinition columnDef = toColumnDefinition(cfm, entity);
+        ColumnDefinition columnDef = entity.prepare(cfm);
         if (mapKey == null)
         {
             Term term = toTerm(toReceivers(columnDef, cfm.isDense()), value, cfm.ksName, boundNames);
@@ -174,7 +174,7 @@ public final class SingleColumnRelation extends Relation
     protected Restriction newINRestriction(CFMetaData cfm,
                                            VariableSpecifications boundNames) throws InvalidRequestException
     {
-        ColumnDefinition columnDef = toColumnDefinition(cfm, entity);
+        ColumnDefinition columnDef = entity.prepare(cfm);
         List<? extends ColumnSpecification> receivers = toReceivers(columnDef, cfm.isDense());
         List<Term> terms = toTerms(receivers, inValues, cfm.ksName, boundNames);
         if (terms == null)
@@ -191,7 +191,7 @@ public final class SingleColumnRelation extends Relation
                                               Bound bound,
                                               boolean inclusive) throws InvalidRequestException
     {
-        ColumnDefinition columnDef = toColumnDefinition(cfm, entity);
+        ColumnDefinition columnDef = entity.prepare(cfm);
         Term term = toTerm(toReceivers(columnDef, cfm.isDense()), value, cfm.ksName, boundNames);
         return new SingleColumnRestriction.SliceRestriction(columnDef, bound, inclusive, term);
     }
@@ -201,7 +201,7 @@ public final class SingleColumnRelation extends Relation
                                                  VariableSpecifications boundNames,
                                                  boolean isKey) throws InvalidRequestException
     {
-        ColumnDefinition columnDef = toColumnDefinition(cfm, entity);
+        ColumnDefinition columnDef = entity.prepare(cfm);
         Term term = toTerm(toReceivers(columnDef, cfm.isDense()), value, cfm.ksName, boundNames);
         return new SingleColumnRestriction.ContainsRestriction(columnDef, term, isKey);
     }
@@ -210,7 +210,7 @@ public final class SingleColumnRelation extends Relation
     protected Restriction newIsNotRestriction(CFMetaData cfm,
                                               VariableSpecifications boundNames) throws InvalidRequestException
     {
-        ColumnDefinition columnDef = toColumnDefinition(cfm, entity);
+        ColumnDefinition columnDef = entity.prepare(cfm);
         // currently enforced by the grammar
         assert value == Constants.NULL_LITERAL : "Expected null literal for IS NOT relation: " + this.toString();
         return new SingleColumnRestriction.IsNotNullRestriction(columnDef);
@@ -222,7 +222,7 @@ public final class SingleColumnRelation extends Relation
         if (mapKey != null)
             throw invalidRequest("%s can't be used with collections.", operator());
 
-        ColumnDefinition columnDef = toColumnDefinition(cfm, entity);
+        ColumnDefinition columnDef = entity.prepare(cfm);
         Term term = toTerm(toReceivers(columnDef, cfm.isDense()), value, cfm.ksName, boundNames);
 
         return new SingleColumnRestriction.LikeRestriction(columnDef, operator, term);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/Term.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Term.java b/src/java/org/apache/cassandra/cql3/Term.java
index 5ae9c18..2c2eba6 100644
--- a/src/java/org/apache/cassandra/cql3/Term.java
+++ b/src/java/org/apache/cassandra/cql3/Term.java
@@ -18,11 +18,10 @@
 package org.apache.cassandra.cql3;
 
 import java.nio.ByteBuffer;
-import java.util.Collections;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 
 /**
@@ -100,6 +99,17 @@ public interface Term
          */
         public abstract String getText();
 
+        /**
+         * The type of the {@code term} if it can be infered.
+         *
+         * @param keyspace the keyspace on which the statement containing this term is on.
+         * @return the type of this {@code Term} if inferrable, or {@code null}
+         * otherwise (for instance, the type isn't inferable for a bind marker. Even for
+         * literals, the exact type is not inferrable since they are valid for many
+         * different types and so this will return {@code null} too).
+         */
+        public abstract AbstractType<?> getExactTypeIfKnown(String keyspace);
+
         @Override
         public String toString()
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/TokenRelation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/TokenRelation.java b/src/java/org/apache/cassandra/cql3/TokenRelation.java
index 8b8b9ce..42464ef 100644
--- a/src/java/org/apache/cassandra/cql3/TokenRelation.java
+++ b/src/java/org/apache/cassandra/cql3/TokenRelation.java
@@ -47,11 +47,11 @@ import static org.apache.cassandra.cql3.statements.RequestValidations.invalidReq
  */
 public final class TokenRelation extends Relation
 {
-    private final List<ColumnIdentifier.Raw> entities;
+    private final List<ColumnDefinition.Raw> entities;
 
     private final Term.Raw value;
 
-    public TokenRelation(List<ColumnIdentifier.Raw> entities, Operator type, Term.Raw value)
+    public TokenRelation(List<ColumnDefinition.Raw> entities, Operator type, Term.Raw value)
     {
         this.entities = entities;
         this.relationType = type;
@@ -128,12 +128,12 @@ public final class TokenRelation extends Relation
         return term;
     }
 
-    public Relation renameIdentifier(ColumnIdentifier.Raw from, ColumnIdentifier.Raw to)
+    public Relation renameIdentifier(ColumnDefinition.Raw from, ColumnDefinition.Raw to)
     {
         if (!entities.contains(from))
             return this;
 
-        List<ColumnIdentifier.Raw> newEntities = entities.stream().map(e -> e.equals(from) ? to : e).collect(Collectors.toList());
+        List<ColumnDefinition.Raw> newEntities = entities.stream().map(e -> e.equals(from) ? to : e).collect(Collectors.toList());
         return new TokenRelation(newEntities, operator(), value);
     }
 
@@ -152,11 +152,9 @@ public final class TokenRelation extends Relation
      */
     private List<ColumnDefinition> getColumnDefinitions(CFMetaData cfm) throws InvalidRequestException
     {
-        List<ColumnDefinition> columnDefs = new ArrayList<>();
-        for ( ColumnIdentifier.Raw raw : entities)
-        {
-            columnDefs.add(toColumnDefinition(cfm, raw));
-        }
+        List<ColumnDefinition> columnDefs = new ArrayList<>(entities.size());
+        for ( ColumnDefinition.Raw raw : entities)
+            columnDefs.add(raw.prepare(cfm));
         return columnDefs;
     }
 


[3/4] cassandra git commit: Allow terms in selection clauses

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/Tuples.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Tuples.java b/src/java/org/apache/cassandra/cql3/Tuples.java
index 19e20c2..ba9ddb6 100644
--- a/src/java/org/apache/cassandra/cql3/Tuples.java
+++ b/src/java/org/apache/cassandra/cql3/Tuples.java
@@ -136,6 +136,20 @@ public class Tuples
             }
         }
 
+        @Override
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            List<AbstractType<?>> types = new ArrayList<>(elements.size());
+            for (Term.Raw term : elements)
+            {
+                AbstractType<?> type = term.getExactTypeIfKnown(keyspace);
+                if (type == null)
+                    return null;
+                types.add(type);
+            }
+            return new TupleType(types);
+        }
+
         public String getText()
         {
             return elements.stream().map(Term.Raw::getText).collect(Collectors.joining(", ", "(", ")"));
@@ -326,6 +340,11 @@ public class Tuples
             return new ColumnSpecification(receivers.get(0).ksName, receivers.get(0).cfName, identifier, type);
         }
 
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            return null;
+        }
+
         public AbstractMarker prepare(String keyspace, List<? extends ColumnSpecification> receivers) throws InvalidRequestException
         {
             return new Tuples.Marker(bindIndex, makeReceiver(receivers));
@@ -365,6 +384,11 @@ public class Tuples
             return new ColumnSpecification(receivers.get(0).ksName, receivers.get(0).cfName, identifier, ListType.getInstance(type, false));
         }
 
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            return null;
+        }
+
         public AbstractMarker prepare(String keyspace, List<? extends ColumnSpecification> receivers) throws InvalidRequestException
         {
             return new InMarker(bindIndex, makeInReceiver(receivers));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/TypeCast.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/TypeCast.java b/src/java/org/apache/cassandra/cql3/TypeCast.java
index 890b34f..7b2f306 100644
--- a/src/java/org/apache/cassandra/cql3/TypeCast.java
+++ b/src/java/org/apache/cassandra/cql3/TypeCast.java
@@ -58,6 +58,11 @@ public class TypeCast extends Term.Raw
             return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
     }
 
+    public AbstractType<?> getExactTypeIfKnown(String keyspace)
+    {
+        return type.prepare(keyspace).getType();
+    }
+
     public String getText()
     {
         return "(" + type + ")" + term;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/UserTypes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/UserTypes.java b/src/java/org/apache/cassandra/cql3/UserTypes.java
index 3a54216..41b8eed 100644
--- a/src/java/org/apache/cassandra/cql3/UserTypes.java
+++ b/src/java/org/apache/cassandra/cql3/UserTypes.java
@@ -23,9 +23,7 @@ import java.util.*;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.marshal.TupleType;
-import org.apache.cassandra.db.marshal.UTF8Type;
-import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.db.rows.CellPath;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -44,15 +42,15 @@ public abstract class UserTypes
         UserType ut = (UserType)column.type;
         return new ColumnSpecification(column.ksName,
                                        column.cfName,
-                                       new ColumnIdentifier(column.name + "." + UTF8Type.instance.compose(ut.fieldName(field)), true),
+                                       new ColumnIdentifier(column.name + "." + ut.fieldName(field), true),
                                        ut.fieldType(field));
     }
 
     public static class Literal extends Term.Raw
     {
-        public final Map<ColumnIdentifier, Term.Raw> entries;
+        public final Map<FieldIdentifier, Term.Raw> entries;
 
-        public Literal(Map<ColumnIdentifier, Term.Raw> entries)
+        public Literal(Map<FieldIdentifier, Term.Raw> entries)
         {
             this.entries = entries;
         }
@@ -67,7 +65,7 @@ public abstract class UserTypes
             int foundValues = 0;
             for (int i = 0; i < ut.size(); i++)
             {
-                ColumnIdentifier field = new ColumnIdentifier(ut.fieldName(i), UTF8Type.instance);
+                FieldIdentifier field = ut.fieldName(i);
                 Term.Raw raw = entries.get(field);
                 if (raw == null)
                     raw = Constants.NULL_LITERAL;
@@ -83,9 +81,9 @@ public abstract class UserTypes
             if (foundValues != entries.size())
             {
                 // We had some field that are not part of the type
-                for (ColumnIdentifier id : entries.keySet())
+                for (FieldIdentifier id : entries.keySet())
                 {
-                    if (!ut.fieldNames().contains(id.bytes))
+                    if (!ut.fieldNames().contains(id))
                         throw new InvalidRequestException(String.format("Unknown field '%s' in value of user defined type %s", id, ut.getNameAsString()));
                 }
             }
@@ -102,7 +100,7 @@ public abstract class UserTypes
             UserType ut = (UserType)receiver.type;
             for (int i = 0; i < ut.size(); i++)
             {
-                ColumnIdentifier field = new ColumnIdentifier(ut.fieldName(i), UTF8Type.instance);
+                FieldIdentifier field = ut.fieldName(i);
                 Term.Raw value = entries.get(field);
                 if (value == null)
                     continue;
@@ -129,14 +127,19 @@ public abstract class UserTypes
             }
         }
 
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            return null;
+        }
+
         public String getText()
         {
             StringBuilder sb = new StringBuilder();
             sb.append("{");
-            Iterator<Map.Entry<ColumnIdentifier, Term.Raw>> iter = entries.entrySet().iterator();
+            Iterator<Map.Entry<FieldIdentifier, Term.Raw>> iter = entries.entrySet().iterator();
             while (iter.hasNext())
             {
-                Map.Entry<ColumnIdentifier, Term.Raw> entry = iter.next();
+                Map.Entry<FieldIdentifier, Term.Raw> entry = iter.next();
                 sb.append(entry.getKey()).append(": ").append(entry.getValue().getText());
                 if (iter.hasNext())
                     sb.append(", ");
@@ -294,10 +297,11 @@ public abstract class UserTypes
                 if (value == null)
                     return;
 
-                Iterator<ByteBuffer> fieldNameIter = userTypeValue.type.fieldNames().iterator();
+                Iterator<FieldIdentifier> fieldNameIter = userTypeValue.type.fieldNames().iterator();
                 for (ByteBuffer buffer : userTypeValue.elements)
                 {
-                    ByteBuffer fieldName = fieldNameIter.next();
+                    assert fieldNameIter.hasNext();
+                    FieldIdentifier fieldName = fieldNameIter.next();
                     if (buffer == null)
                         continue;
 
@@ -318,9 +322,9 @@ public abstract class UserTypes
 
     public static class SetterByField extends Operation
     {
-        private final ColumnIdentifier field;
+        private final FieldIdentifier field;
 
-        public SetterByField(ColumnDefinition column, ColumnIdentifier field, Term t)
+        public SetterByField(ColumnDefinition column, FieldIdentifier field, Term t)
         {
             super(column, t);
             this.field = field;
@@ -335,7 +339,7 @@ public abstract class UserTypes
             if (value == UNSET_VALUE)
                 return;
 
-            CellPath fieldPath = ((UserType) column.type).cellPathForField(field.bytes);
+            CellPath fieldPath = ((UserType) column.type).cellPathForField(field);
             if (value == null)
                 params.addTombstone(column, fieldPath);
             else
@@ -345,9 +349,9 @@ public abstract class UserTypes
 
     public static class DeleterByField extends Operation
     {
-        private final ColumnIdentifier field;
+        private final FieldIdentifier field;
 
-        public DeleterByField(ColumnDefinition column, ColumnIdentifier field)
+        public DeleterByField(ColumnDefinition column, FieldIdentifier field)
         {
             super(column, null);
             this.field = field;
@@ -358,7 +362,7 @@ public abstract class UserTypes
             // we should not get here for frozen UDTs
             assert column.type.isMultiCell() : "Attempted to delete a single field from a frozen UDT";
 
-            CellPath fieldPath = ((UserType) column.type).cellPathForField(field.bytes);
+            CellPath fieldPath = ((UserType) column.type).cellPathForField(field);
             params.addTombstone(column, fieldPath);
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/functions/FunctionCall.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/FunctionCall.java b/src/java/org/apache/cassandra/cql3/functions/FunctionCall.java
index 0fe86e4..3905c83 100644
--- a/src/java/org/apache/cassandra/cql3/functions/FunctionCall.java
+++ b/src/java/org/apache/cassandra/cql3/functions/FunctionCall.java
@@ -189,6 +189,16 @@ public class FunctionCall extends Term.NonTerminal
             }
         }
 
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            // We could implement this, but the method is only used in selection clause, where FunctionCall is not used 
+            // we use a Selectable.WithFunction instead). And if that method is later used in other places, better to
+            // let that future patch make sure this can be implemented properly (note in particular we don't have access
+            // to the receiver type, which FunctionResolver.get() takes) rather than provide an implementation that may
+            // not work in all cases.
+            throw new UnsupportedOperationException();
+        }
+
         public String getText()
         {
             return name + terms.stream().map(Term.Raw::getText).collect(Collectors.joining(", ", "(", ")"));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java b/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
index 868f752..498cf0f 100644
--- a/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
@@ -24,7 +24,9 @@ import java.util.List;
 import org.apache.commons.lang3.text.StrBuilder;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.statements.RequestValidations;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 
@@ -36,7 +38,7 @@ abstract class AbstractFunctionSelector<T extends Function> extends Selector
      * The list used to pass the function arguments is recycled to avoid the cost of instantiating a new list
      * with each function call.
      */
-    protected final List<ByteBuffer> args;
+    private final List<ByteBuffer> args;
     protected final List<Selector> argSelectors;
 
     public static Factory newFactory(final Function fun, final SelectorFactories factories) throws InvalidRequestException
@@ -80,10 +82,10 @@ abstract class AbstractFunctionSelector<T extends Function> extends Selector
                 factories.addFunctionsTo(functions);
             }
 
-            public Selector newInstance() throws InvalidRequestException
+            public Selector newInstance(QueryOptions options) throws InvalidRequestException
             {
-                return fun.isAggregate() ? new AggregateFunctionSelector(fun, factories.newInstances())
-                                         : new ScalarFunctionSelector(fun, factories.newInstances());
+                return fun.isAggregate() ? new AggregateFunctionSelector(fun, factories.newInstances(options))
+                                         : new ScalarFunctionSelector(fun, factories.newInstances(options));
             }
 
             public boolean isWritetimeSelectorFactory()
@@ -110,6 +112,19 @@ abstract class AbstractFunctionSelector<T extends Function> extends Selector
         this.args = Arrays.asList(new ByteBuffer[argSelectors.size()]);
     }
 
+    // Sets a given arg value. We should use that instead of directly setting the args list for the
+    // sake of validation.
+    protected void setArg(int i, ByteBuffer value) throws InvalidRequestException
+    {
+        RequestValidations.checkBindValueSet(value, "Invalid unset value for argument in call to function %s", fun.name().name);
+        args.set(i, value);
+    }
+
+    protected List<ByteBuffer> args()
+    {
+        return args;
+    }
+
     public AbstractType<?> getType()
     {
         return fun.returnType();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/selection/AggregateFunctionSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/AggregateFunctionSelector.java b/src/java/org/apache/cassandra/cql3/selection/AggregateFunctionSelector.java
index 27a8294..d768665 100644
--- a/src/java/org/apache/cassandra/cql3/selection/AggregateFunctionSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/AggregateFunctionSelector.java
@@ -41,10 +41,10 @@ final class AggregateFunctionSelector extends AbstractFunctionSelector<Aggregate
         {
             Selector s = argSelectors.get(i);
             s.addInput(protocolVersion, rs);
-            args.set(i, s.getOutput(protocolVersion));
+            setArg(i, s.getOutput(protocolVersion));
             s.reset();
         }
-        this.aggregate.addInput(protocolVersion, args);
+        this.aggregate.addInput(protocolVersion, args());
     }
 
     public ByteBuffer getOutput(int protocolVersion) throws InvalidRequestException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java b/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
index 965a01a..55ff50f 100644
--- a/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.cql3.selection;
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.UTF8Type;
@@ -38,9 +39,7 @@ final class FieldSelector extends Selector
         {
             protected String getColumnName()
             {
-                return String.format("%s.%s",
-                                     factory.getColumnName(),
-                                     UTF8Type.instance.getString(type.fieldName(field)));
+                return String.format("%s.%s", factory.getColumnName(), type.fieldName(field));
             }
 
             protected AbstractType<?> getReturnType()
@@ -53,9 +52,9 @@ final class FieldSelector extends Selector
                 factory.addColumnMapping(mapping, resultsColumn);
             }
 
-            public Selector newInstance() throws InvalidRequestException
+            public Selector newInstance(QueryOptions options) throws InvalidRequestException
             {
-                return new FieldSelector(type, field, factory.newInstance());
+                return new FieldSelector(type, field, factory.newInstance(options));
             }
 
             public boolean isAggregateSelectorFactory()
@@ -92,7 +91,7 @@ final class FieldSelector extends Selector
     @Override
     public String toString()
     {
-        return String.format("%s.%s", selected, UTF8Type.instance.getString(type.fieldName(field)));
+        return String.format("%s.%s", selected, type.fieldName(field));
     }
 
     private FieldSelector(UserType type, int field, Selector selected)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/selection/ScalarFunctionSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/ScalarFunctionSelector.java b/src/java/org/apache/cassandra/cql3/selection/ScalarFunctionSelector.java
index bb56bb8..50175c1 100644
--- a/src/java/org/apache/cassandra/cql3/selection/ScalarFunctionSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/ScalarFunctionSelector.java
@@ -54,14 +54,14 @@ final class ScalarFunctionSelector extends AbstractFunctionSelector<ScalarFuncti
         for (int i = 0, m = argSelectors.size(); i < m; i++)
         {
             Selector s = argSelectors.get(i);
-            args.set(i, s.getOutput(protocolVersion));
+            setArg(i, s.getOutput(protocolVersion));
             s.reset();
         }
-        return fun.execute(protocolVersion, args);
+        return fun.execute(protocolVersion, args());
     }
 
     ScalarFunctionSelector(Function fun, List<Selector> argSelectors)
     {
         super((ScalarFunction) fun, argSelectors);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/selection/Selectable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/Selectable.java b/src/java/org/apache/cassandra/cql3/selection/Selectable.java
index faf3f2d..1f1f07b 100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selectable.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selectable.java
@@ -21,22 +21,40 @@ package org.apache.cassandra.cql3.selection;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.nio.ByteBuffer;
 
 import org.apache.commons.lang3.text.StrBuilder;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.cql3.CQL3Type;
-import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.functions.*;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 
-public abstract class Selectable
+public interface Selectable extends AssignmentTestable
 {
-    public abstract Selector.Factory newSelectorFactory(CFMetaData cfm, List<ColumnDefinition> defs);
+    public Selector.Factory newSelectorFactory(CFMetaData cfm, AbstractType<?> expectedType, List<ColumnDefinition> defs, VariableSpecifications boundNames);
+
+    /**
+     * The type of the {@code Selectable} if it can be infered.
+     *
+     * @param keyspace the keyspace on which the statement for which this is a
+     * {@code Selectable} is on.
+     * @return the type of this {@code Selectable} if inferrable, or {@code null}
+     * otherwise (for instance, the type isn't inferable for a bind marker. Even for
+     * literals, the exact type is not inferrable since they are valid for many
+     * different types and so this will return {@code null} too).
+     */
+    public AbstractType<?> getExactTypeIfKnown(String keyspace);
+
+    // Term.Raw overrides this since some literals can be WEAKLY_ASSIGNABLE
+    default public TestResult testAssignment(String keyspace, ColumnSpecification receiver)
+    {
+        AbstractType<?> type = getExactTypeIfKnown(keyspace);
+        return type == null ? TestResult.NOT_ASSIGNABLE : type.testAssignment(keyspace, receiver);
+    }
 
-    protected static int addAndGetIndex(ColumnDefinition def, List<ColumnDefinition> l)
+    default int addAndGetIndex(ColumnDefinition def, List<ColumnDefinition> l)
     {
         int idx = l.indexOf(def);
         if (idx < 0)
@@ -47,57 +65,160 @@ public abstract class Selectable
         return idx;
     }
 
-    public static interface Raw
+    public static abstract class Raw
     {
-        public Selectable prepare(CFMetaData cfm);
+        public abstract Selectable prepare(CFMetaData cfm);
 
         /**
          * Returns true if any processing is performed on the selected column.
          **/
-        public boolean processesSelection();
+        public boolean processesSelection()
+        {
+            // ColumnIdentifier is the only case that returns false and override this
+            return true;
+        }
     }
 
-    public static class WritetimeOrTTL extends Selectable
+    public static class WithTerm implements Selectable
     {
-        public final ColumnIdentifier id;
+        /**
+         * The names given to unamed bind markers found in selection. In selection clause, we often don't have a good
+         * name for bind markers, typically if you have:
+         *   SELECT (int)? FROM foo;
+         * there isn't a good name for that marker. So we give the same name to all the markers. Note that we could try
+         * to differenciate the names by using some increasing number in the name (so [selection_1], [selection_2], ...)
+         * but it's actually not trivial to do in the current code and it's not really more helpful since if users wants
+         * to bind by position (which they will have to in this case), they can do so at the driver level directly. And
+         * so we don't bother.
+         * Note that users should really be using named bind markers if they want to be able to bind by names.
+         */
+        private static final ColumnIdentifier bindMarkerNameInSelection = new ColumnIdentifier("[selection]", true);
+
+        private final Term.Raw rawTerm;
+
+        public WithTerm(Term.Raw rawTerm)
+        {
+            this.rawTerm = rawTerm;
+        }
+
+        @Override
+        public TestResult testAssignment(String keyspace, ColumnSpecification receiver)
+        {
+            return rawTerm.testAssignment(keyspace, receiver);
+        }
+
+        public Selector.Factory newSelectorFactory(CFMetaData cfm, AbstractType<?> expectedType, List<ColumnDefinition> defs, VariableSpecifications boundNames) throws InvalidRequestException
+        {
+            /*
+             * expectedType will be null if we have no constraint on what the type should be. For instance, if this term is a bind marker:
+             *   - it will be null if we do "SELECT ? FROM foo"
+             *   - it won't be null (and be LongType) if we do "SELECT bigintAsBlob(?) FROM foo" because the function constrain it.
+             *
+             * In the first case, we have to error out: we need to infer the type of the metadata of a SELECT at preparation time, which we can't
+             * here (users will have to do "SELECT (varint)? FROM foo" for instance).
+             * But in the 2nd case, we're fine and can use the expectedType to "prepare" the bind marker/collect the bound type.
+             *
+             * Further, the term might not be a bind marker, in which case we sometimes can default to some most-general type. For instance, in
+             *   SELECT 3 FROM foo
+             * we'll just default the type to 'varint' as that's the most generic type for the literal '3' (this is mostly for convenience, the query
+             * is not terribly useful in practice and use can force the type as for the bind marker case through "SELECT (int)3 FROM foo").
+             * But note that not all literals can have such default type. For instance, there is no way to infer the type of a UDT literal in a vacuum,
+             * and so we simply error out if we have something like:
+             *   SELECT { foo: 'bar' } FROM foo
+             *
+             * Lastly, note that if the term is a terminal literal, we don't have to check it's compatibility with 'expectedType' as any incompatibility
+             * would have been found at preparation time.
+             */
+            AbstractType<?> type = getExactTypeIfKnown(cfm.ksName);
+            if (type == null)
+            {
+                type = expectedType;
+                if (type == null)
+                    throw new InvalidRequestException("Cannot infer type for term " + this + " in selection clause (try using a cast to force a type)");
+            }
+
+            // The fact we default the name to "[selection]" inconditionally means that any bind marker in a
+            // selection will have this name. Which isn't terribly helpful, but it's unclear how to provide
+            // something a lot more helpful and in practice user can bind those markers by position or, even better,
+            // use bind markers.
+            Term term = rawTerm.prepare(cfm.ksName, new ColumnSpecification(cfm.ksName, cfm.cfName, bindMarkerNameInSelection, type));
+            term.collectMarkerSpecification(boundNames);
+            return TermSelector.newFactory(rawTerm.getText(), term, type);
+        }
+
+        @Override
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            return rawTerm.getExactTypeIfKnown(keyspace);
+        }
+ 
+        @Override
+        public String toString()
+        {
+            return rawTerm.toString();
+        }
+
+        public static class Raw extends Selectable.Raw
+        {
+            private final Term.Raw term;
+
+            public Raw(Term.Raw term)
+            {
+                this.term = term;
+            }
+
+            public Selectable prepare(CFMetaData cfm)
+            {
+                return new WithTerm(term);
+            }
+        }
+    }
+
+    public static class WritetimeOrTTL implements Selectable
+    {
+        public final ColumnDefinition column;
         public final boolean isWritetime;
 
-        public WritetimeOrTTL(ColumnIdentifier id, boolean isWritetime)
+        public WritetimeOrTTL(ColumnDefinition column, boolean isWritetime)
         {
-            this.id = id;
+            this.column = column;
             this.isWritetime = isWritetime;
         }
 
         @Override
         public String toString()
         {
-            return (isWritetime ? "writetime" : "ttl") + "(" + id + ")";
+            return (isWritetime ? "writetime" : "ttl") + "(" + column.name + ")";
         }
 
         public Selector.Factory newSelectorFactory(CFMetaData cfm,
-                                                   List<ColumnDefinition> defs)
+                                                   AbstractType<?> expectedType,
+                                                   List<ColumnDefinition> defs,
+                                                   VariableSpecifications boundNames)
         {
-            ColumnDefinition def = cfm.getColumnDefinition(id);
-            if (def == null)
-                throw new InvalidRequestException(String.format("Undefined name %s in selection clause", id));
-            if (def.isPrimaryKeyColumn())
+            if (column.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())
+                                      column.name));
+            if (column.type.isCollection())
                 throw new InvalidRequestException(String.format("Cannot use selection function %s on collections",
                                                                 isWritetime ? "writeTime" : "ttl"));
 
-            return WritetimeOrTTLSelector.newFactory(def, addAndGetIndex(def, defs), isWritetime);
+            return WritetimeOrTTLSelector.newFactory(column, addAndGetIndex(column, defs), isWritetime);
+        }
+
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            return isWritetime ? LongType.instance : Int32Type.instance;
         }
 
-        public static class Raw implements Selectable.Raw
+        public static class Raw extends Selectable.Raw
         {
-            private final ColumnIdentifier.Raw id;
+            private final ColumnDefinition.Raw id;
             private final boolean isWritetime;
 
-            public Raw(ColumnIdentifier.Raw id, boolean isWritetime)
+            public Raw(ColumnDefinition.Raw id, boolean isWritetime)
             {
                 this.id = id;
                 this.isWritetime = isWritetime;
@@ -107,59 +228,42 @@ public abstract class Selectable
             {
                 return new WritetimeOrTTL(id.prepare(cfm), isWritetime);
             }
-
-            public boolean processesSelection()
-            {
-                return true;
-            }
         }
     }
 
-    public static class WithFunction extends Selectable
+    public static class WithFunction implements Selectable
     {
-        public final FunctionName functionName;
+        public final Function function;
         public final List<Selectable> args;
 
-        public WithFunction(FunctionName functionName, List<Selectable> args)
+        public WithFunction(Function function, List<Selectable> args)
         {
-            this.functionName = functionName;
+            this.function = function;
             this.args = args;
         }
 
         @Override
         public String toString()
         {
-            return new StrBuilder().append(functionName)
+            return new StrBuilder().append(function.name())
                                    .append("(")
                                    .appendWithSeparators(args, ", ")
                                    .append(")")
                                    .toString();
         }
 
-        public Selector.Factory newSelectorFactory(CFMetaData cfm, List<ColumnDefinition> defs)
+        public Selector.Factory newSelectorFactory(CFMetaData cfm, AbstractType<?> expectedType, List<ColumnDefinition> defs, VariableSpecifications boundNames)
         {
-            SelectorFactories factories  =
-                    SelectorFactories.createFactoriesAndCollectColumnDefinitions(args, cfm, defs);
-
-            // We need to circumvent the normal function lookup process for toJson() because instances of the function
-            // are not pre-declared (because it can accept any type of argument).
-            Function fun;
-            if (functionName.equalsNativeFunction(ToJsonFct.NAME))
-                fun = ToJsonFct.getInstance(factories.getReturnTypes());
-            else
-                fun = FunctionResolver.get(cfm.ksName, functionName, factories.newInstances(), cfm.ksName, cfm.cfName, null);
-
-            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));
+            SelectorFactories factories = SelectorFactories.createFactoriesAndCollectColumnDefinitions(args, function.argTypes(), cfm, defs, boundNames);
+            return AbstractFunctionSelector.newFactory(function, factories);
+        }
 
-            return AbstractFunctionSelector.newFactory(fun, factories);
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            return function.returnType();
         }
 
-        public static class Raw implements Selectable.Raw
+        public static class Raw extends Selectable.Raw
         {
             private final FunctionName functionName;
             private final List<Selectable.Raw> args;
@@ -176,22 +280,79 @@ public abstract class Selectable
                                Collections.emptyList());
             }
 
-            public WithFunction prepare(CFMetaData cfm)
+            public Selectable 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 boolean processesSelection()
-            {
-                return true;
+                FunctionName name = functionName;
+                // We need to circumvent the normal function lookup process for toJson() because instances of the function
+                // are not pre-declared (because it can accept any type of argument). We also have to wait until we have the
+                // selector factories of the argument so we can access their final type.
+                if (functionName.equalsNativeFunction(ToJsonFct.NAME))
+                {
+                    return new WithToJSonFunction(preparedArgs);
+                }
+                // Also, COUNT(x) is equivalent to COUNT(*) for any non-null term x (since count(x) don't care about it's argument outside of check for nullness) and
+                // for backward compatibilty we want to support COUNT(1), but we actually have COUNT(x) method for every existing (simple) input types so currently COUNT(1)
+                // will throw as ambiguous (since 1 works for any type). So we have have to special case COUNT.
+                else if (functionName.equalsNativeFunction(FunctionName.nativeFunction("count"))
+                        && preparedArgs.size() == 1
+                        && (preparedArgs.get(0) instanceof WithTerm)
+                        && (((WithTerm)preparedArgs.get(0)).rawTerm instanceof Constants.Literal))
+                {
+                    // Note that 'null' isn't a Constants.Literal
+                    name = AggregateFcts.countRowsFunction.name();
+                    preparedArgs = Collections.emptyList();
+                }
+
+                Function fun = FunctionResolver.get(cfm.ksName, name, preparedArgs, cfm.ksName, cfm.cfName, null);
+
+                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 new WithFunction(fun, preparedArgs);
             }
         }
     }
 
-    public static class WithCast extends Selectable
+    public static class WithToJSonFunction implements Selectable
+    {
+        public final List<Selectable> args;
+
+        private WithToJSonFunction(List<Selectable> args)
+        {
+            this.args = args;
+        }
+
+        @Override
+        public String toString()
+        {
+            return new StrBuilder().append(ToJsonFct.NAME)
+                                   .append("(")
+                                   .appendWithSeparators(args, ", ")
+                                   .append(")")
+                                   .toString();
+        }
+
+        public Selector.Factory newSelectorFactory(CFMetaData cfm, AbstractType<?> expectedType, List<ColumnDefinition> defs, VariableSpecifications boundNames)
+        {
+            SelectorFactories factories = SelectorFactories.createFactoriesAndCollectColumnDefinitions(args, null, cfm, defs, boundNames);
+            Function fun = ToJsonFct.getInstance(factories.getReturnTypes());
+            return AbstractFunctionSelector.newFactory(fun, factories);
+        }
+
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            return UTF8Type.instance;
+        }
+    }
+
+    public static class WithCast implements Selectable
     {
         private final CQL3Type type;
         private final Selectable arg;
@@ -208,21 +369,19 @@ public abstract class Selectable
             return String.format("cast(%s as %s)", arg, type.toString().toLowerCase());
         }
 
-        public Selector.Factory newSelectorFactory(CFMetaData cfm, List<ColumnDefinition> defs)
+        public Selector.Factory newSelectorFactory(CFMetaData cfm, AbstractType<?> expectedType, List<ColumnDefinition> defs, VariableSpecifications boundNames)
         {
-            SelectorFactories factories  =
-                    SelectorFactories.createFactoriesAndCollectColumnDefinitions(Collections.singletonList(arg), cfm, defs);
+            List<Selectable> args = Collections.singletonList(arg);
+            SelectorFactories factories = SelectorFactories.createFactoriesAndCollectColumnDefinitions(args, null, cfm, defs, boundNames);
 
             Selector.Factory factory = factories.get(0);
 
             // If the user is trying to cast a type on its own type we simply ignore it.
             if (type.getType().equals(factory.getReturnType()))
-            {
                 return factory;
-            }
 
             FunctionName name = FunctionName.nativeFunction(CastFcts.getFunctionName(type));
-            Function fun = FunctionResolver.get(cfm.ksName, name, factories.newInstances(), cfm.ksName, cfm.cfName, null);
+            Function fun = FunctionResolver.get(cfm.ksName, name, args, cfm.ksName, cfm.cfName, null);
 
             if (fun == null)
             {
@@ -233,7 +392,12 @@ public abstract class Selectable
             return AbstractFunctionSelector.newFactory(fun, factories);
         }
 
-        public static class Raw implements Selectable.Raw
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            return type.getType();
+        }
+
+        public static class Raw extends Selectable.Raw
         {
             private final CQL3Type type;
             private final Selectable.Raw arg;
@@ -248,20 +412,15 @@ public abstract class Selectable
             {
                 return new WithCast(arg.prepare(cfm), type);
             }
-
-            public boolean processesSelection()
-            {
-                return true;
-            }
         }
     }
 
-    public static class WithFieldSelection extends Selectable
+    public static class WithFieldSelection implements Selectable
     {
         public final Selectable selected;
-        public final ColumnIdentifier field;
+        public final FieldIdentifier field;
 
-        public WithFieldSelection(Selectable selected, ColumnIdentifier field)
+        public WithFieldSelection(Selectable selected, FieldIdentifier field)
         {
             this.selected = selected;
             this.field = field;
@@ -273,10 +432,10 @@ public abstract class Selectable
             return String.format("%s.%s", selected, field);
         }
 
-        public Selector.Factory newSelectorFactory(CFMetaData cfm, List<ColumnDefinition> defs)
+        public Selector.Factory newSelectorFactory(CFMetaData cfm, AbstractType<?> expectedType, List<ColumnDefinition> defs, VariableSpecifications boundNames)
         {
-            Selector.Factory factory = selected.newSelectorFactory(cfm, defs);
-            AbstractType<?> type = factory.newInstance().getType();
+            Selector.Factory factory = selected.newSelectorFactory(cfm, null, defs, boundNames);
+            AbstractType<?> type = factory.getColumnSpecification(cfm).type;
             if (!type.isUDT())
             {
                 throw new InvalidRequestException(
@@ -286,7 +445,7 @@ public abstract class Selectable
             }
 
             UserType ut = (UserType) type;
-            int fieldIndex = ((UserType) type).fieldPosition(field);
+            int fieldIndex = ut.fieldPosition(field);
             if (fieldIndex == -1)
             {
                 throw new InvalidRequestException(String.format("%s of type %s has no field %s",
@@ -296,12 +455,26 @@ public abstract class Selectable
             return FieldSelector.newFactory(ut, fieldIndex, factory);
         }
 
-        public static class Raw implements Selectable.Raw
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            AbstractType<?> selectedType = selected.getExactTypeIfKnown(keyspace);
+            if (selectedType == null || !(selectedType instanceof UserType))
+                return null;
+
+            UserType ut = (UserType) selectedType;
+            int fieldIndex = ut.fieldPosition(field);
+            if (fieldIndex == -1)
+                return null;
+
+            return ut.fieldType(fieldIndex);
+        }
+
+        public static class Raw extends Selectable.Raw
         {
             private final Selectable.Raw selected;
-            private final ColumnIdentifier.Raw field;
+            private final FieldIdentifier field;
 
-            public Raw(Selectable.Raw selected, ColumnIdentifier.Raw field)
+            public Raw(Selectable.Raw selected, FieldIdentifier field)
             {
                 this.selected = selected;
                 this.field = field;
@@ -309,12 +482,7 @@ public abstract class Selectable
 
             public WithFieldSelection prepare(CFMetaData cfm)
             {
-                return new WithFieldSelection(selected.prepare(cfm), field.prepare(cfm));
-            }
-
-            public boolean processesSelection()
-            {
-                return true;
+                return new WithFieldSelection(selected.prepare(cfm), field);
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/selection/Selection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/Selection.java b/src/java/org/apache/cassandra/cql3/selection/Selection.java
index 664fd4f..2a11d27 100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selection.java
@@ -169,12 +169,12 @@ public abstract class Selection
         return false;
     }
 
-    public static Selection fromSelectors(CFMetaData cfm, List<RawSelector> rawSelectors) throws InvalidRequestException
+    public static Selection fromSelectors(CFMetaData cfm, List<RawSelector> rawSelectors, VariableSpecifications boundNames) throws InvalidRequestException
     {
         List<ColumnDefinition> defs = new ArrayList<>();
 
         SelectorFactories factories =
-                SelectorFactories.createFactoriesAndCollectColumnDefinitions(RawSelector.toSelectables(rawSelectors, cfm), cfm, defs);
+                SelectorFactories.createFactoriesAndCollectColumnDefinitions(RawSelector.toSelectables(rawSelectors, cfm), null, cfm, defs, boundNames);
         SelectionColumnMapping mapping = collectColumnMappings(cfm, rawSelectors, factories);
 
         return (processesSelection(rawSelectors) || rawSelectors.size() != defs.size())
@@ -221,7 +221,7 @@ public abstract class Selection
         return selectionColumns;
     }
 
-    protected abstract Selectors newSelectors() throws InvalidRequestException;
+    protected abstract Selectors newSelectors(QueryOptions options) throws InvalidRequestException;
 
     /**
      * @return the list of CQL3 columns value this SelectionClause needs.
@@ -239,9 +239,9 @@ public abstract class Selection
         return columnMapping;
     }
 
-    public ResultSetBuilder resultSetBuilder(boolean isJons) throws InvalidRequestException
+    public ResultSetBuilder resultSetBuilder(QueryOptions options, boolean isJons) throws InvalidRequestException
     {
-        return new ResultSetBuilder(isJons);
+        return new ResultSetBuilder(options, isJons);
     }
 
     public abstract boolean isAggregate();
@@ -287,6 +287,7 @@ public abstract class Selection
     public class ResultSetBuilder
     {
         private final ResultSet resultSet;
+        private final int protocolVersion;
 
         /**
          * As multiple thread can access a <code>Selection</code> instance each <code>ResultSetBuilder</code> will use
@@ -308,10 +309,11 @@ public abstract class Selection
 
         private final boolean isJson;
 
-        private ResultSetBuilder(boolean isJson) throws InvalidRequestException
+        private ResultSetBuilder(QueryOptions options, boolean isJson) throws InvalidRequestException
         {
             this.resultSet = new ResultSet(getResultMetadata(isJson).copy(), new ArrayList<List<ByteBuffer>>());
-            this.selectors = newSelectors();
+            this.protocolVersion = options.getProtocolVersion();
+            this.selectors = newSelectors(options);
             this.timestamps = collectTimestamps ? new long[columns.size()] : null;
             this.ttls = collectTTLs ? new int[columns.size()] : null;
             this.isJson = isJson;
@@ -361,36 +363,36 @@ public abstract class Selection
                  : c.value();
         }
 
-        public void newRow(int protocolVersion) throws InvalidRequestException
+        public void newRow() throws InvalidRequestException
         {
             if (current != null)
             {
                 selectors.addInputRow(protocolVersion, this);
                 if (!selectors.isAggregate())
                 {
-                    resultSet.addRow(getOutputRow(protocolVersion));
+                    resultSet.addRow(getOutputRow());
                     selectors.reset();
                 }
             }
             current = new ArrayList<>(columns.size());
         }
 
-        public ResultSet build(int protocolVersion) throws InvalidRequestException
+        public ResultSet build() throws InvalidRequestException
         {
             if (current != null)
             {
                 selectors.addInputRow(protocolVersion, this);
-                resultSet.addRow(getOutputRow(protocolVersion));
+                resultSet.addRow(getOutputRow());
                 selectors.reset();
                 current = null;
             }
 
             if (resultSet.isEmpty() && selectors.isAggregate())
-                resultSet.addRow(getOutputRow(protocolVersion));
+                resultSet.addRow(getOutputRow());
             return resultSet;
         }
 
-        private List<ByteBuffer> getOutputRow(int protocolVersion)
+        private List<ByteBuffer> getOutputRow()
         {
             List<ByteBuffer> outputRow = selectors.getOutputRow(protocolVersion);
             return isJson ? rowToJson(outputRow, protocolVersion, metadata)
@@ -415,7 +417,7 @@ public abstract class Selection
         public void reset();
     }
 
-    // Special cased selection for when no function is used (this save some allocations).
+    // Special cased selection for when only columns are selected.
     private static class SimpleSelection extends Selection
     {
         private final boolean isWildcard;
@@ -450,7 +452,7 @@ public abstract class Selection
             return false;
         }
 
-        protected Selectors newSelectors()
+        protected Selectors newSelectors(QueryOptions options)
         {
             return new Selectors()
             {
@@ -531,11 +533,11 @@ public abstract class Selection
             return factories.doesAggregation();
         }
 
-        protected Selectors newSelectors() throws InvalidRequestException
+        protected Selectors newSelectors(final QueryOptions options) throws InvalidRequestException
         {
             return new Selectors()
             {
-                private final List<Selector> selectors = factories.newInstances();
+                private final List<Selector> selectors = factories.newInstances(options);
 
                 public void reset()
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/selection/Selector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/Selector.java b/src/java/org/apache/cassandra/cql3/selection/Selector.java
index 87ab14c..c85dcd1 100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selector.java
@@ -21,13 +21,12 @@ import java.nio.ByteBuffer;
 import java.util.List;
 
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.cql3.AssignmentTestable;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
 import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.ReversedType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 
 /**
@@ -36,7 +35,7 @@ import org.apache.cassandra.exceptions.InvalidRequestException;
  * <p>Since the introduction of aggregation, <code>Selector</code>s cannot be called anymore by multiple threads 
  * as they have an internal state.</p>
  */
-public abstract class Selector implements AssignmentTestable
+public abstract class Selector
 {
     /**
      * A factory for <code>Selector</code> instances.
@@ -58,16 +57,19 @@ public abstract class Selector implements AssignmentTestable
         {
             return new ColumnSpecification(cfm.ksName,
                                            cfm.cfName,
-                                           ColumnIdentifier.getInterned(getColumnName(), true),
+                                           new ColumnIdentifier(getColumnName(), true), // note that the name is not necessarily
+                                                                                        // a true column name so we shouldn't intern it
                                            getReturnType());
         }
 
         /**
          * Creates a new <code>Selector</code> instance.
          *
+         * @param options the options of the query for which the instance is created (some selector
+         * depends on the bound values in particular).
          * @return a new <code>Selector</code> instance
          */
-        public abstract Selector newInstance() throws InvalidRequestException;
+        public abstract Selector newInstance(QueryOptions options) throws InvalidRequestException;
 
         /**
          * Checks if this factory creates selectors instances that creates aggregates.
@@ -183,24 +185,4 @@ public abstract class Selector implements AssignmentTestable
      * Reset the internal state of this <code>Selector</code>.
      */
     public abstract void reset();
-
-    public final AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
-    {
-        // We should ignore the fact that the output type is frozen in our comparison as functions do not support
-        // frozen types for arguments
-        AbstractType<?> receiverType = receiver.type;
-        if (getType().isFreezable() && !getType().isMultiCell())
-            receiverType = receiverType.freeze();
-
-        if (getType().isReversed())
-            receiverType = ReversedType.getInstance(receiverType);
-
-        if (receiverType.equals(getType()))
-            return AssignmentTestable.TestResult.EXACT_MATCH;
-
-        if (receiverType.isValueCompatibleWith(getType()))
-            return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
-
-        return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java b/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
index 97a1198..41bf193 100644
--- a/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
+++ b/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
@@ -23,6 +23,8 @@ import com.google.common.collect.Lists;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.VariableSpecifications;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.selection.Selector.Factory;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -57,29 +59,40 @@ final class SelectorFactories implements Iterable<Selector.Factory>
      * Creates a new <code>SelectorFactories</code> instance and collect the column definitions.
      *
      * @param selectables the <code>Selectable</code>s for which the factories must be created
+     * @param expectedTypes the returned types expected for each of the {@code selectables}, if there
+     * is any such expectations, or {@code null} otherwise. This will be {@code null} when called on
+     * the top-level selectables, but may not be for selectable nested within a function for instance
+     * (as the argument selectable will be expected to be of the type expected by the function).
      * @param cfm the Column Family Definition
      * @param defs the collector parameter for the column definitions
+     * @param boundNames the collector for the specification of bound markers in the selection
      * @return a new <code>SelectorFactories</code> instance
      * @throws InvalidRequestException if a problem occurs while creating the factories
      */
     public static SelectorFactories createFactoriesAndCollectColumnDefinitions(List<Selectable> selectables,
+                                                                               List<AbstractType<?>> expectedTypes,
                                                                                CFMetaData cfm,
-                                                                               List<ColumnDefinition> defs)
+                                                                               List<ColumnDefinition> defs,
+                                                                               VariableSpecifications boundNames)
                                                                                throws InvalidRequestException
     {
-        return new SelectorFactories(selectables, cfm, defs);
+        return new SelectorFactories(selectables, expectedTypes, cfm, defs, boundNames);
     }
 
     private SelectorFactories(List<Selectable> selectables,
+                              List<AbstractType<?>> expectedTypes,
                               CFMetaData cfm,
-                              List<ColumnDefinition> defs)
+                              List<ColumnDefinition> defs,
+                              VariableSpecifications boundNames)
                               throws InvalidRequestException
     {
         factories = new ArrayList<>(selectables.size());
 
-        for (Selectable selectable : selectables)
+        for (int i = 0; i < selectables.size(); i++)
         {
-            Factory factory = selectable.newSelectorFactory(cfm, defs);
+            Selectable selectable = selectables.get(i);
+            AbstractType<?> expectedType = expectedTypes == null ? null : expectedTypes.get(i);
+            Factory factory = selectable.newSelectorFactory(cfm, expectedType, defs, boundNames);
             containsWritetimeFactory |= factory.isWritetimeSelectorFactory();
             containsTTLFactory |= factory.isTTLSelectorFactory();
             if (factory.isAggregateSelectorFactory())
@@ -148,15 +161,15 @@ final class SelectorFactories implements Iterable<Selector.Factory>
 
     /**
      * Creates a list of new <code>Selector</code> instances.
+     *
+     * @param options the query options for the query being executed.
      * @return a list of new <code>Selector</code> instances.
      */
-    public List<Selector> newInstances() throws InvalidRequestException
+    public List<Selector> newInstances(QueryOptions options) throws InvalidRequestException
     {
         List<Selector> selectors = new ArrayList<>(factories.size());
         for (Selector.Factory factory : factories)
-        {
-            selectors.add(factory.newInstance());
-        }
+            selectors.add(factory.newInstance(options));
         return selectors;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java b/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java
index e4040fa..e14cd5c 100644
--- a/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java
@@ -21,6 +21,7 @@ import java.nio.ByteBuffer;
 
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -55,7 +56,7 @@ public final class SimpleSelector extends Selector
             }
 
             @Override
-            public Selector newInstance()
+            public Selector newInstance(QueryOptions options)
             {
                 return new SimpleSelector(def.name.toString(), idx, def.type);
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/selection/TermSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/TermSelector.java b/src/java/org/apache/cassandra/cql3/selection/TermSelector.java
new file mode 100644
index 0000000..5aa4522
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/TermSelector.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.AssignmentTestable;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * Selector representing a simple term (literals or bound variables).
+ * <p>
+ * Note that we know the term does not include function calls for instance (this is actually enforced by the parser), those
+ * being dealt with by their own Selector.
+ */
+public class TermSelector extends Selector
+{
+    private final ByteBuffer value;
+    private final AbstractType<?> type;
+
+    public static Factory newFactory(final String name, final Term term, final AbstractType<?> type)
+    {
+        return new Factory()
+        {
+            protected String getColumnName()
+            {
+                return name;
+            }
+
+            protected AbstractType<?> getReturnType()
+            {
+                return type;
+            }
+
+            protected void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultColumn)
+            {
+               mapping.addMapping(resultColumn, (ColumnDefinition)null);
+            }
+
+            public Selector newInstance(QueryOptions options)
+            {
+                return new TermSelector(term.bindAndGet(options), type);
+            }
+        };
+    }
+
+    private TermSelector(ByteBuffer value, AbstractType<?> type)
+    {
+        this.value = value;
+        this.type = type;
+    }
+
+    public void addInput(int protocolVersion, Selection.ResultSetBuilder rs) throws InvalidRequestException
+    {
+    }
+
+    public ByteBuffer getOutput(int protocolVersion) throws InvalidRequestException
+    {
+        return value;
+    }
+
+    public AbstractType<?> getType()
+    {
+        return type;
+    }
+
+    public void reset()
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java b/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java
index 131827f..78380d7 100644
--- a/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.cql3.selection;
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -54,7 +55,7 @@ final class WritetimeOrTTLSelector extends Selector
                mapping.addMapping(resultsColumn, def);
             }
 
-            public Selector newInstance()
+            public Selector newInstance(QueryOptions options)
             {
                 return new WritetimeOrTTLSelector(def.name.toString(), idx, isWritetime);
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/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 ee2b623..afe2776 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
@@ -53,14 +53,14 @@ public class AlterTableStatement extends SchemaAlteringStatement
 
     public final Type oType;
     private final TableAttributes attrs;
-    private final Map<ColumnIdentifier.Raw, ColumnIdentifier.Raw> renames;
+    private final Map<ColumnDefinition.Raw, ColumnDefinition.Raw> renames;
     private final List<AlterTableStatementColumn> colNameList;
 
     public AlterTableStatement(CFName name,
                                Type type,
                                List<AlterTableStatementColumn> colDataList,
                                TableAttributes attrs,
-                               Map<ColumnIdentifier.Raw, ColumnIdentifier.Raw> renames)
+                               Map<ColumnDefinition.Raw, ColumnDefinition.Raw> renames)
     {
         super(name);
         this.oType = type;
@@ -91,7 +91,7 @@ public class AlterTableStatement extends SchemaAlteringStatement
         CQL3Type.Raw dataType = null;
         boolean isStatic = false;
         CQL3Type validator = null;
-        ColumnIdentifier.Raw rawColumnName = null;
+        ColumnDefinition.Raw rawColumnName = null;
 
         List<ViewDefinition> viewUpdates = null;
         Iterable<ViewDefinition> views = View.findAll(keyspace(), columnFamily());
@@ -101,11 +101,10 @@ public class AlterTableStatement extends SchemaAlteringStatement
             case ADD:
                 for (AlterTableStatementColumn colData : colNameList)
                 {
-                    columnName = null;
                     rawColumnName = colData.getColumnName();
                     if (rawColumnName != null)
                     {
-                        columnName = rawColumnName.prepare(cfm);
+                        columnName = rawColumnName.getIdentifier(cfm);
                         def =  cfm.getColumnDefinition(columnName);
                         dataType = colData.getColumnType();
                         isStatic = colData.getStaticType();
@@ -191,7 +190,7 @@ public class AlterTableStatement extends SchemaAlteringStatement
                 rawColumnName = colNameList.get(0).getColumnName();
                 if (rawColumnName != null)
                 {
-                    columnName = rawColumnName.prepare(cfm);
+                    columnName = rawColumnName.getIdentifier(cfm);
                     def = cfm.getColumnDefinition(columnName);
                     dataType = colNameList.get(0).getColumnType();
                     validator = dataType == null ? null : dataType.prepare(keyspace());
@@ -234,7 +233,7 @@ public class AlterTableStatement extends SchemaAlteringStatement
                     rawColumnName = colData.getColumnName();
                     if (rawColumnName != null)
                     {
-                        columnName = rawColumnName.prepare(cfm);
+                        columnName = rawColumnName.getIdentifier(cfm);
                         def = cfm.getColumnDefinition(columnName);
                     }
                     assert columnName != null;
@@ -322,10 +321,10 @@ public class AlterTableStatement extends SchemaAlteringStatement
 
                 break;
             case RENAME:
-                for (Map.Entry<ColumnIdentifier.Raw, ColumnIdentifier.Raw> entry : renames.entrySet())
+                for (Map.Entry<ColumnDefinition.Raw, ColumnDefinition.Raw> entry : renames.entrySet())
                 {
-                    ColumnIdentifier from = entry.getKey().prepare(cfm);
-                    ColumnIdentifier to = entry.getValue().prepare(cfm);
+                    ColumnIdentifier from = entry.getKey().getIdentifier(cfm);
+                    ColumnIdentifier to = entry.getValue().getIdentifier(cfm);
                     cfm.renameColumn(from, to);
 
                     // If the view includes a renamed column, it must be renamed in the view table and the definition.
@@ -334,8 +333,8 @@ public class AlterTableStatement extends SchemaAlteringStatement
                         if (!view.includes(from)) continue;
 
                         ViewDefinition viewCopy = view.copy();
-                        ColumnIdentifier viewFrom = entry.getKey().prepare(viewCopy.metadata);
-                        ColumnIdentifier viewTo = entry.getValue().prepare(viewCopy.metadata);
+                        ColumnIdentifier viewFrom = entry.getKey().getIdentifier(viewCopy.metadata);
+                        ColumnIdentifier viewTo = entry.getValue().getIdentifier(viewCopy.metadata);
                         viewCopy.renameColumn(viewFrom, viewTo);
 
                         if (viewUpdates == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/statements/AlterTableStatementColumn.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatementColumn.java b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatementColumn.java
index a5d7de7..813effe 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatementColumn.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatementColumn.java
@@ -17,37 +17,45 @@
  */
 package org.apache.cassandra.cql3.statements;
 
+import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.CQL3Type;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 
-public class AlterTableStatementColumn {
+public class AlterTableStatementColumn
+{
     private final CQL3Type.Raw dataType;
-    private final ColumnIdentifier.Raw colName;
+    private final ColumnDefinition.Raw colName;
     private final Boolean isStatic;
 
-    public AlterTableStatementColumn(ColumnIdentifier.Raw colName, CQL3Type.Raw dataType, boolean isStatic) {
+    public AlterTableStatementColumn(ColumnDefinition.Raw colName, CQL3Type.Raw dataType, boolean isStatic)
+    {
         this.dataType = dataType;
         this.colName = colName;
         this.isStatic = isStatic;
     }
 
-    public AlterTableStatementColumn(ColumnIdentifier.Raw colName, CQL3Type.Raw dataType) {
+    public AlterTableStatementColumn(ColumnDefinition.Raw colName, CQL3Type.Raw dataType)
+    {
         this(colName, dataType,false );
     }
 
-    public AlterTableStatementColumn(ColumnIdentifier.Raw colName) {
+    public AlterTableStatementColumn(ColumnDefinition.Raw colName)
+    {
         this(colName, null, false);
     }
 
-    public CQL3Type.Raw getColumnType() {
+    public CQL3Type.Raw getColumnType()
+    {
         return dataType;
     }
 
-    public ColumnIdentifier.Raw getColumnName() {
+    public ColumnDefinition.Raw getColumnName()
+    {
         return colName;
     }
 
-    public Boolean getStaticType() {
+    public Boolean getStaticType()
+    {
         return isStatic;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java
index 4ed726a..64bccf5 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java
@@ -51,17 +51,17 @@ public abstract class AlterTypeStatement extends SchemaAlteringStatement
 
     protected abstract UserType makeUpdatedType(UserType toUpdate, KeyspaceMetadata ksm) throws InvalidRequestException;
 
-    public static AlterTypeStatement addition(UTName name, ColumnIdentifier fieldName, CQL3Type.Raw type)
+    public static AlterTypeStatement addition(UTName name, FieldIdentifier fieldName, CQL3Type.Raw type)
     {
         return new AddOrAlter(name, true, fieldName, type);
     }
 
-    public static AlterTypeStatement alter(UTName name, ColumnIdentifier fieldName, CQL3Type.Raw type)
+    public static AlterTypeStatement alter(UTName name, FieldIdentifier fieldName, CQL3Type.Raw type)
     {
         return new AddOrAlter(name, false, fieldName, type);
     }
 
-    public static AlterTypeStatement renames(UTName name, Map<ColumnIdentifier, ColumnIdentifier> renames)
+    public static AlterTypeStatement renames(UTName name, Map<FieldIdentifier, FieldIdentifier> renames)
     {
         return new Renames(name, renames);
     }
@@ -137,14 +137,6 @@ public abstract class AlterTypeStatement extends SchemaAlteringStatement
         return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TYPE, keyspace(), name.getStringTypeName());
     }
 
-    private static int getIdxOfField(UserType type, ColumnIdentifier field)
-    {
-        for (int i = 0; i < type.size(); i++)
-            if (field.bytes.equals(type.fieldName(i)))
-                return i;
-        return -1;
-    }
-
     private boolean updateDefinition(CFMetaData cfm, ColumnDefinition def, String keyspace, ByteBuffer toReplace, UserType updated)
     {
         AbstractType<?> t = updateWith(def.type, keyspace, toReplace, updated);
@@ -247,10 +239,10 @@ public abstract class AlterTypeStatement extends SchemaAlteringStatement
     private static class AddOrAlter extends AlterTypeStatement
     {
         private final boolean isAdd;
-        private final ColumnIdentifier fieldName;
+        private final FieldIdentifier fieldName;
         private final CQL3Type.Raw type;
 
-        public AddOrAlter(UTName name, boolean isAdd, ColumnIdentifier fieldName, CQL3Type.Raw type)
+        public AddOrAlter(UTName name, boolean isAdd, FieldIdentifier fieldName, CQL3Type.Raw type)
         {
             super(name);
             this.isAdd = isAdd;
@@ -260,12 +252,12 @@ public abstract class AlterTypeStatement extends SchemaAlteringStatement
 
         private UserType doAdd(UserType toUpdate) throws InvalidRequestException
         {
-            if (getIdxOfField(toUpdate, fieldName) >= 0)
+            if (toUpdate.fieldPosition(fieldName) >= 0)
                 throw new InvalidRequestException(String.format("Cannot add new field %s to type %s: a field of the same name already exists", fieldName, name));
 
-            List<ByteBuffer> newNames = new ArrayList<>(toUpdate.size() + 1);
+            List<FieldIdentifier> newNames = new ArrayList<>(toUpdate.size() + 1);
             newNames.addAll(toUpdate.fieldNames());
-            newNames.add(fieldName.bytes);
+            newNames.add(fieldName);
 
             AbstractType<?> addType = type.prepare(keyspace()).getType();
             if (addType.referencesUserType(toUpdate.getNameAsString()))
@@ -282,7 +274,7 @@ public abstract class AlterTypeStatement extends SchemaAlteringStatement
         {
             checkTypeNotUsedByAggregate(ksm);
 
-            int idx = getIdxOfField(toUpdate, fieldName);
+            int idx = toUpdate.fieldPosition(fieldName);
             if (idx < 0)
                 throw new InvalidRequestException(String.format("Unknown field %s in type %s", fieldName, name));
 
@@ -290,7 +282,7 @@ public abstract class AlterTypeStatement extends SchemaAlteringStatement
             if (!type.prepare(keyspace()).getType().isCompatibleWith(previous))
                 throw new InvalidRequestException(String.format("Type %s is incompatible with previous type %s of field %s in user type %s", type, previous.asCQL3Type(), fieldName, name));
 
-            List<ByteBuffer> newNames = new ArrayList<>(toUpdate.fieldNames());
+            List<FieldIdentifier> newNames = new ArrayList<>(toUpdate.fieldNames());
             List<AbstractType<?>> newTypes = new ArrayList<>(toUpdate.fieldTypes());
             newTypes.set(idx, type.prepare(keyspace()).getType());
 
@@ -305,9 +297,9 @@ public abstract class AlterTypeStatement extends SchemaAlteringStatement
 
     private static class Renames extends AlterTypeStatement
     {
-        private final Map<ColumnIdentifier, ColumnIdentifier> renames;
+        private final Map<FieldIdentifier, FieldIdentifier> renames;
 
-        public Renames(UTName name, Map<ColumnIdentifier, ColumnIdentifier> renames)
+        public Renames(UTName name, Map<FieldIdentifier, FieldIdentifier> renames)
         {
             super(name);
             this.renames = renames;
@@ -317,17 +309,17 @@ public abstract class AlterTypeStatement extends SchemaAlteringStatement
         {
             checkTypeNotUsedByAggregate(ksm);
 
-            List<ByteBuffer> newNames = new ArrayList<>(toUpdate.fieldNames());
+            List<FieldIdentifier> newNames = new ArrayList<>(toUpdate.fieldNames());
             List<AbstractType<?>> newTypes = new ArrayList<>(toUpdate.fieldTypes());
 
-            for (Map.Entry<ColumnIdentifier, ColumnIdentifier> entry : renames.entrySet())
+            for (Map.Entry<FieldIdentifier, FieldIdentifier> entry : renames.entrySet())
             {
-                ColumnIdentifier from = entry.getKey();
-                ColumnIdentifier to = entry.getValue();
-                int idx = getIdxOfField(toUpdate, from);
+                FieldIdentifier from = entry.getKey();
+                FieldIdentifier to = entry.getValue();
+                int idx = toUpdate.fieldPosition(from);
                 if (idx < 0)
                     throw new InvalidRequestException(String.format("Unknown field %s in type %s", from, name));
-                newNames.set(idx, to.bytes);
+                newNames.set(idx, to);
             }
 
             UserType updated = new UserType(toUpdate.keyspace, toUpdate.name, newNames, newTypes, toUpdate.isMultiCell());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/statements/CreateTypeStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateTypeStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateTypeStatement.java
index 3268296..6f4331b 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateTypeStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateTypeStatement.java
@@ -37,7 +37,7 @@ import org.apache.cassandra.transport.Event;
 public class CreateTypeStatement extends SchemaAlteringStatement
 {
     private final UTName name;
-    private final List<ColumnIdentifier> columnNames = new ArrayList<>();
+    private final List<FieldIdentifier> columnNames = new ArrayList<>();
     private final List<CQL3Type.Raw> columnTypes = new ArrayList<>();
     private final boolean ifNotExists;
 
@@ -55,7 +55,7 @@ public class CreateTypeStatement extends SchemaAlteringStatement
             name.setKeyspace(state.getKeyspace());
     }
 
-    public void addDefinition(ColumnIdentifier name, CQL3Type.Raw type)
+    public void addDefinition(FieldIdentifier name, CQL3Type.Raw type)
     {
         columnNames.add(name);
         columnTypes.add(type);
@@ -88,13 +88,11 @@ public class CreateTypeStatement extends SchemaAlteringStatement
     {
         for (int i = 0; i < type.size() - 1; i++)
         {
-            ByteBuffer fieldName = type.fieldName(i);
+            FieldIdentifier fieldName = type.fieldName(i);
             for (int j = i+1; j < type.size(); j++)
             {
                 if (fieldName.equals(type.fieldName(j)))
-                    throw new InvalidRequestException(String.format("Duplicate field name %s in type %s",
-                                                                    UTF8Type.instance.getString(fieldName),
-                                                                    UTF8Type.instance.getString(type.name)));
+                    throw new InvalidRequestException(String.format("Duplicate field name %s in type %s", fieldName, type.name));
             }
         }
     }
@@ -102,7 +100,7 @@ public class CreateTypeStatement extends SchemaAlteringStatement
     public void addToRawBuilder(Types.RawBuilder builder) throws InvalidRequestException
     {
         builder.add(name.getStringTypeName(),
-                    columnNames.stream().map(ColumnIdentifier::toString).collect(Collectors.toList()),
+                    columnNames.stream().map(FieldIdentifier::toString).collect(Collectors.toList()),
                     columnTypes.stream().map(CQL3Type.Raw::toString).collect(Collectors.toList()));
     }
 
@@ -114,15 +112,11 @@ public class CreateTypeStatement extends SchemaAlteringStatement
 
     public UserType createType() throws InvalidRequestException
     {
-        List<ByteBuffer> names = new ArrayList<>(columnNames.size());
-        for (ColumnIdentifier name : columnNames)
-            names.add(name.bytes);
-
         List<AbstractType<?>> types = new ArrayList<>(columnTypes.size());
         for (CQL3Type.Raw type : columnTypes)
             types.add(type.prepare(keyspace()).getType());
 
-        return new UserType(name.getKeyspace(), name.getUserTypeName(), names, types, true);
+        return new UserType(name.getKeyspace(), name.getUserTypeName(), columnNames, types, true);
     }
 
     public Event.SchemaChange announceMigration(boolean isLocalOnly) throws InvalidRequestException, ConfigurationException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ed00607/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
index dbb2b9b..71e248a 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
@@ -51,8 +51,8 @@ public class CreateViewStatement extends SchemaAlteringStatement
     private final CFName baseName;
     private final List<RawSelector> selectClause;
     private final WhereClause whereClause;
-    private final List<ColumnIdentifier.Raw> partitionKeys;
-    private final List<ColumnIdentifier.Raw> clusteringKeys;
+    private final List<ColumnDefinition.Raw> partitionKeys;
+    private final List<ColumnDefinition.Raw> clusteringKeys;
     public final CFProperties properties = new CFProperties();
     private final boolean ifNotExists;
 
@@ -60,8 +60,8 @@ public class CreateViewStatement extends SchemaAlteringStatement
                                CFName baseName,
                                List<RawSelector> selectClause,
                                WhereClause whereClause,
-                               List<ColumnIdentifier.Raw> partitionKeys,
-                               List<ColumnIdentifier.Raw> clusteringKeys,
+                               List<ColumnDefinition.Raw> partitionKeys,
+                               List<ColumnDefinition.Raw> clusteringKeys,
                                boolean ifNotExists)
     {
         super(viewName);
@@ -159,30 +159,26 @@ public class CreateViewStatement extends SchemaAlteringStatement
                 throw new InvalidRequestException("Cannot use function when defining a materialized view");
             if (selectable instanceof Selectable.WritetimeOrTTL.Raw)
                 throw new InvalidRequestException("Cannot use function when defining a materialized view");
-            ColumnIdentifier identifier = (ColumnIdentifier) selectable.prepare(cfm);
             if (selector.alias != null)
-                throw new InvalidRequestException(String.format("Cannot alias column '%s' as '%s' when defining a materialized view", identifier.toString(), selector.alias.toString()));
+                throw new InvalidRequestException("Cannot use alias when defining a materialized view");
 
-            ColumnDefinition cdef = cfm.getColumnDefinition(identifier);
+            Selectable s = selectable.prepare(cfm);
+            if (s instanceof Term.Raw)
+                throw new InvalidRequestException("Cannot use terms in selection when defining a materialized view");
 
-            if (cdef == null)
-                throw new InvalidRequestException("Unknown column name detected in CREATE MATERIALIZED VIEW statement : "+identifier);
-
-            included.add(identifier);
+            ColumnDefinition cdef = (ColumnDefinition)s;
+            included.add(cdef.name);
         }
 
-        Set<ColumnIdentifier.Raw> targetPrimaryKeys = new HashSet<>();
-        for (ColumnIdentifier.Raw identifier : Iterables.concat(partitionKeys, clusteringKeys))
+        Set<ColumnDefinition.Raw> targetPrimaryKeys = new HashSet<>();
+        for (ColumnDefinition.Raw identifier : Iterables.concat(partitionKeys, clusteringKeys))
         {
             if (!targetPrimaryKeys.add(identifier))
                 throw new InvalidRequestException("Duplicate entry found in PRIMARY KEY: "+identifier);
 
-            ColumnDefinition cdef = cfm.getColumnDefinition(identifier.prepare(cfm));
-
-            if (cdef == null)
-                throw new InvalidRequestException("Unknown column name detected in CREATE MATERIALIZED VIEW statement : "+identifier);
+            ColumnDefinition cdef = identifier.prepare(cfm);
 
-            if (cfm.getColumnDefinition(identifier.prepare(cfm)).type.isMultiCell())
+            if (cdef.type.isMultiCell())
                 throw new InvalidRequestException(String.format("Cannot use MultiCell column '%s' in PRIMARY KEY of materialized view", identifier));
 
             if (cdef.isStatic())
@@ -190,7 +186,7 @@ public class CreateViewStatement extends SchemaAlteringStatement
         }
 
         // build the select statement
-        Map<ColumnIdentifier.Raw, Boolean> orderings = Collections.emptyMap();
+        Map<ColumnDefinition.Raw, Boolean> orderings = Collections.emptyMap();
         SelectStatement.Parameters parameters = new SelectStatement.Parameters(orderings, false, true, false);
         SelectStatement.RawStatement rawSelect = new SelectStatement.RawStatement(baseName, parameters, selectClause, whereClause, null, null);
 
@@ -226,10 +222,10 @@ public class CreateViewStatement extends SchemaAlteringStatement
 
         // This is only used as an intermediate state; this is to catch whether multiple non-PK columns are used
         boolean hasNonPKColumn = false;
-        for (ColumnIdentifier.Raw raw : partitionKeys)
+        for (ColumnDefinition.Raw raw : partitionKeys)
             hasNonPKColumn |= getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetPartitionKeys, restrictions);
 
-        for (ColumnIdentifier.Raw raw : clusteringKeys)
+        for (ColumnDefinition.Raw raw : clusteringKeys)
             hasNonPKColumn |= getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetClusteringColumns, restrictions);
 
         // We need to include all of the primary key columns from the base table in order to make sure that we do not
@@ -307,25 +303,24 @@ public class CreateViewStatement extends SchemaAlteringStatement
     private static boolean getColumnIdentifier(CFMetaData cfm,
                                                Set<ColumnIdentifier> basePK,
                                                boolean hasNonPKColumn,
-                                               ColumnIdentifier.Raw raw,
+                                               ColumnDefinition.Raw raw,
                                                List<ColumnIdentifier> columns,
                                                StatementRestrictions restrictions)
     {
-        ColumnIdentifier identifier = raw.prepare(cfm);
-        ColumnDefinition def = cfm.getColumnDefinition(identifier);
+        ColumnDefinition def = raw.prepare(cfm);
 
-        boolean isPk = basePK.contains(identifier);
+        boolean isPk = basePK.contains(def.name);
         if (!isPk && hasNonPKColumn)
-            throw new InvalidRequestException(String.format("Cannot include more than one non-primary key column '%s' in materialized view partition key", identifier));
+            throw new InvalidRequestException(String.format("Cannot include more than one non-primary key column '%s' in materialized view partition key", def.name));
 
         // We don't need to include the "IS NOT NULL" filter on a non-composite partition key
         // because we will never allow a single partition key to be NULL
-        boolean isSinglePartitionKey = cfm.getColumnDefinition(identifier).isPartitionKey()
+        boolean isSinglePartitionKey = def.isPartitionKey()
                                        && cfm.partitionKeyColumns().size() == 1;
         if (!isSinglePartitionKey && !restrictions.isRestricted(def))
-            throw new InvalidRequestException(String.format("Primary key column '%s' is required to be filtered by 'IS NOT NULL'", identifier));
+            throw new InvalidRequestException(String.format("Primary key column '%s' is required to be filtered by 'IS NOT NULL'", def.name));
 
-        columns.add(identifier);
+        columns.add(def.name);
         return !isPk;
     }
 }