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 2013/01/31 12:20:09 UTC

[3/3] git commit: Improve CQL3 type validation

Updated Branches:
  refs/heads/cassandra-1.2 be36736d3 -> a67f77922


Improve CQL3 type validation

patch by slebresne; reviewed by iamaleksey for CASSANDRA-5198


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

Branch: refs/heads/cassandra-1.2
Commit: a67f77922a5aba36540abd4526adf4ded1d6e8c7
Parents: 4df6136
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Jan 31 11:40:28 2013 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Jan 31 12:19:51 2013 +0100

----------------------------------------------------------------------
 NEWS.txt                                           |   11 +
 doc/cql3/CQL.textile                               |   57 +++--
 src/java/org/apache/cassandra/cql3/CQL3Type.java   |  185 +++++++++++++++
 src/java/org/apache/cassandra/cql3/Cql.g           |   82 ++++---
 src/java/org/apache/cassandra/cql3/ParsedType.java |  152 ------------
 src/java/org/apache/cassandra/cql3/Term.java       |   55 ++++-
 .../cassandra/cql3/operations/ColumnOperation.java |    7 +-
 .../cassandra/cql3/operations/ListOperation.java   |   15 +-
 .../cassandra/cql3/operations/MapOperation.java    |    9 +-
 .../cassandra/cql3/operations/Operation.java       |    2 +-
 .../cql3/operations/PreparedOperation.java         |   46 +++-
 .../cassandra/cql3/operations/SetOperation.java    |   49 +++--
 .../cql3/statements/AlterTableStatement.java       |    4 +-
 .../statements/CreateColumnFamilyStatement.java    |    8 +-
 .../cassandra/cql3/statements/DeleteStatement.java |    6 +-
 .../cassandra/cql3/statements/SelectStatement.java |   15 +-
 .../cassandra/cql3/statements/UpdateStatement.java |   27 +--
 src/java/org/apache/cassandra/db/SystemTable.java  |    6 +-
 .../apache/cassandra/db/marshal/AbstractType.java  |   15 ++
 .../org/apache/cassandra/db/marshal/AsciiType.java |   16 ++
 .../apache/cassandra/db/marshal/BooleanType.java   |   16 ++
 .../org/apache/cassandra/db/marshal/BytesType.java |   16 ++
 .../cassandra/db/marshal/CollectionType.java       |    7 +
 .../cassandra/db/marshal/CounterColumnType.java    |   16 ++
 .../org/apache/cassandra/db/marshal/DateType.java  |   17 ++-
 .../apache/cassandra/db/marshal/DecimalType.java   |   16 ++
 .../apache/cassandra/db/marshal/DoubleType.java    |   16 ++
 .../org/apache/cassandra/db/marshal/FloatType.java |   16 ++
 .../cassandra/db/marshal/InetAddressType.java      |   16 ++
 .../org/apache/cassandra/db/marshal/Int32Type.java |   15 ++
 .../apache/cassandra/db/marshal/IntegerType.java   |   16 ++
 .../cassandra/db/marshal/LexicalUUIDType.java      |   10 +
 .../org/apache/cassandra/db/marshal/LongType.java  |   16 ++
 .../apache/cassandra/db/marshal/TimeUUIDType.java  |   16 ++
 .../org/apache/cassandra/db/marshal/UTF8Type.java  |   16 ++
 .../org/apache/cassandra/db/marshal/UUIDType.java  |   16 ++
 .../dht/AbstractByteOrderedPartitioner.java        |    7 +
 .../org/apache/cassandra/dht/IPartitioner.java     |    3 +
 .../org/apache/cassandra/dht/LocalPartitioner.java |    5 +
 .../apache/cassandra/dht/Murmur3Partitioner.java   |    7 +
 .../cassandra/dht/OrderPreservingPartitioner.java  |    7 +
 .../apache/cassandra/dht/RandomPartitioner.java    |    7 +
 .../apache/cassandra/service/StorageService.java   |   13 +-
 43 files changed, 759 insertions(+), 298 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index c6757aa..5a6e4f5 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -8,6 +8,17 @@ upgrade, just in case you need to roll back to the previous version.
 (Cassandra version X + 1 will always be able to read data files created
 by version X, but the inverse is not necessarily the case.)
 
+1.2.2
+=====
+
+Upgrading
+---------
+    - CQL3 type validation for constants has been fixed, which may require
+      fixing queries that were relying on the previous loose validation. Please
+      refer to the CQL3 documentation (http://cassandra.apache.org/doc/cql3/CQL.html)
+      and in particular the changelog section for more details.
+
+
 1.2.1
 =====
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/doc/cql3/CQL.textile
----------------------------------------------------------------------
diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile
index dd4090b..35b75bf 100644
--- a/doc/cql3/CQL.textile
+++ b/doc/cql3/CQL.textile
@@ -1,6 +1,6 @@
 <link rel="StyleSheet" href="CQL.css" type="text/css" media="screen">
 
-h1. Cassandra Query Language (CQL) v3.0.1
+h1. Cassandra Query Language (CQL) v3.0.2
 
 
  <span id="tableOfContents">
@@ -47,12 +47,15 @@ p. There is a second kind of identifiers called _quoted identifiers_ defined by
 
 h3(#constants). Constants
 
-CQL defines 4 kinds of _implicitly-typed constants_: strings, numbers, uuids and booleans:
+CQL defines the following kind of _constants_: strings, integers, floats, booleans, uuids and blobs:
 * A string constant is an arbitrary sequence of characters characters enclosed by single-quote(@'@). One can include a single-quote in a string by repeating it, e.g. @'It''s raining today'@. Those are not to be confused with quoted identifiers that use double-quotes.
-* Numeric constants are either integer constant defined by @'-'?[0-9]+@ or a float constant defined by @'-'?[0-9]+('.'[0-9]*)?([eE][+-]?[0-9+])?@.
-* A "UUID":http://en.wikipedia.org/wiki/Universally_unique_identifier constant is defined by @hex{8}-hex{4}-hex{4}-hex{4}-hex{12}@ where @hex@ is an hexadecimal character, e.g. @[0-9a-fA-F]@ and @{4}@ is the number of such characters.
+* An integer constant is defined by @'-'?[0-9]+@.
+* A float constant is defined by @'-'?[0-9]+('.'[0-9]*)?([eE][+-]?[0-9+])?@.
 * A boolean constant is either @true@ or @false@ up to case-insensitivity (i.e. @True@ is a valid boolean constant).
+* A "UUID":http://en.wikipedia.org/wiki/Universally_unique_identifier constant is defined by @hex{8}-hex{4}-hex{4}-hex{4}-hex{12}@ where @hex@ is an hexadecimal character, e.g. @[0-9a-fA-F]@ and @{4}@ is the number of such characters.
+* A blob constant is an hexadecimal number defined by @0[xX](hex)+@ where @hex@ is an hexadecimal character, e.g. @[0-9a-fA-F]@.
 
+For how these constants are typed, see the "data types section":#types.
 
 h3. Comments
 
@@ -692,7 +695,7 @@ SELECT firstname, lastname FROM users WHERE birth_year = 1981 AND country = 'FR'
 
 h2(#types). Data Types
 
-CQL supports a rich set of native data types for columns defined in a table.  On top of those native types, users can also provide custom types (through a JAVA class extending @AbstractType@ loadable by Cassandra). The syntax of types is thus:
+CQL supports a rich set of data types for columns defined in a table, including collection types. On top of those native and collection types, users can also provide custom types (through a JAVA class extending @AbstractType@ loadable by Cassandra). The syntax of types is thus:
 
 bc(syntax).. 
 <type> ::= <native-type>
@@ -721,25 +724,27 @@ bc(syntax)..
                     | map  '<' <native-type> ',' <native-type> '>'
 p. Note that the native types are keywords and as such are case-insensitive. They are however not reserved ones.
 
-p. The following table gives additional informations on the native data types:
-
-|_. type    |_. description|
-|@ascii@    |ASCII character string|
-|@bigint@   |64-bit signed long|
-|@blob@     |Arbitrary bytes (no validation)|
-|@boolean@  |true or false|
-|@counter@  |Counter column (64-bit signed value). See "Counters":#counters for details|
-|@decimal@  |Variable-precision decimal|
-|@double@   |64-bit IEEE-754 floating point|
-|@float@    |32-bit IEEE-754 floating point|
-|@inet@     |An IP address. It can be either 4 bytes long (IPv4) or 16 bytes long (IPv6)|
-|@int@      |32-bit signed int|
-|@text@     |UTF8 encoded string|
-|@timestamp@|A timestamp. See "Working with dates":#usingdates below for more information.|
-|@timeuuid@ |Type 1 UUID. This is generally used as a "conflict-free" timestamp. See "Working with @timeuuid@":#usingtimeuuid below.|
-|@uuid@     |Type 1 or type 4 UUID|
-|@varchar@  |UTF8 encoded string|
-|@varint@   |Arbitrary-precision integer|
+p. The following table gives additional informations on the native data types, and on which kind of "constants":#constants each type supports:
+
+|_. type    |_. constants supported|_. description|
+|@ascii@    |   strings            |ASCII character string|
+|@bigint@   |   integers           |64-bit signed long|
+|@blob@     |   blobs              |Arbitrary bytes (no validation)|
+|@boolean@  |   booleans           |true or false|
+|@counter@  |   integers           |Counter column (64-bit signed value). See "Counters":#counters for details|
+|@decimal@  |   integers, floats   |Variable-precision decimal|
+|@double@   |   integers           |64-bit IEEE-754 floating point|
+|@float@    |   integers, floats   |32-bit IEEE-754 floating point|
+|@inet@     |   strings            |An IP address. It can be either 4 bytes long (IPv4) or 16 bytes long (IPv6). There is no @inet@ constant, IP address should be inputed as strings|
+|@int@      |   integers           |32-bit signed int|
+|@text@     |   strings            |UTF8 encoded string|
+|@timestamp@|   integers, strings  |A timestamp. Strings constant are allow to input timestamps as dates, see "Working with dates":#usingdates below for more information.|
+|@timeuuid@ |   uuids              |Type 1 UUID. This is generally used as a "conflict-free" timestamp. See "Working with @timeuuid@":#usingtimeuuid below.|
+|@uuid@     |   uuids              |Type 1 or type 4 UUID|
+|@varchar@  |   strings            |UTF8 encoded string|
+|@varint@   |   integers           |Arbitrary-precision integer|
+
+For more information on how to use the collection types, see the "Working with collections":#collections section below.
 
 h3(#usingdates). Working with dates
 
@@ -1000,6 +1005,10 @@ h2(#changes). Changes
 
 The following describes the addition/changes brought for each version of CQL.
 
+h3. 3.0.2
+
+- Type validation for the "constants":#constants has been fixed. For instance, the implementation used to allow @'2'@ as a valid value for an @int@ column (interpreting it has the equivalent of @2@), or @42@ as a valid @blob@ value (in which case @42@ was interpreted as an hexadecimal representation of the blob). This is no longer the case, type validation of constants is now more strict. See the "data types":#dataTypes section for details on which constant is allowed for which type, but note that this let to the introduction of "blobs constants":#constants.
+
 h3. 3.0.1
 
 - "Date strings":#usingdates (and timestamps) are no longer accepted as valid @timeuuid@ values. Doing so was a bug in the sense that date string are not valid @timeuuid@, and it was thus resulting in "confusing behaviors":https://issues.apache.org/jira/browse/CASSANDRA-4936.  However, the following new methods have been added to help working with @timeuuid@: @now@, @minTimeuuid@, @maxTimeuuid@ , @dateOf@ and @unixTimestampOf@. See the "section dedicated to these methods":#usingtimeuuid for more detail.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/cql3/CQL3Type.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/CQL3Type.java b/src/java/org/apache/cassandra/cql3/CQL3Type.java
new file mode 100644
index 0000000..58a2fa3
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/CQL3Type.java
@@ -0,0 +1,185 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3;
+
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
+
+public interface CQL3Type
+{
+    public boolean isCollection();
+    public boolean isCounter();
+    public AbstractType<?> getType();
+
+    public enum Native implements CQL3Type
+    {
+        ASCII    (AsciiType.instance),
+        BIGINT   (LongType.instance),
+        BLOB     (BytesType.instance),
+        BOOLEAN  (BooleanType.instance),
+        COUNTER  (CounterColumnType.instance),
+        DECIMAL  (DecimalType.instance),
+        DOUBLE   (DoubleType.instance),
+        FLOAT    (FloatType.instance),
+        INET     (InetAddressType.instance),
+        INT      (Int32Type.instance),
+        TEXT     (UTF8Type.instance),
+        TIMESTAMP(DateType.instance),
+        UUID     (UUIDType.instance),
+        VARCHAR  (UTF8Type.instance),
+        VARINT   (IntegerType.instance),
+        TIMEUUID (TimeUUIDType.instance);
+
+        private final AbstractType<?> type;
+
+        private Native(AbstractType<?> type)
+        {
+            this.type = type;
+        }
+
+        public boolean isCollection()
+        {
+            return false;
+        }
+
+        public AbstractType<?> getType()
+        {
+            return type;
+        }
+
+        public boolean isCounter()
+        {
+            return this == COUNTER;
+        }
+
+        @Override
+        public String toString()
+        {
+            return super.toString().toLowerCase();
+        }
+    }
+
+    public static class Custom implements CQL3Type
+    {
+        private final AbstractType<?> type;
+
+        public Custom(AbstractType<?> type)
+        {
+            this.type = type;
+        }
+
+        public Custom(String className) throws SyntaxException, ConfigurationException
+        {
+            this(TypeParser.parse(className));
+        }
+
+        public boolean isCollection()
+        {
+            return false;
+        }
+
+        public AbstractType<?> getType()
+        {
+            return type;
+        }
+
+        public boolean isCounter()
+        {
+            return false;
+        }
+
+        @Override
+        public String toString()
+        {
+            return "'" + type + "'";
+        }
+    }
+
+    public static class Collection implements CQL3Type
+    {
+        CollectionType type;
+
+        public Collection(CollectionType type)
+        {
+            this.type = type;
+        }
+
+        public static Collection map(CQL3Type t1, CQL3Type t2) throws InvalidRequestException
+        {
+            if (t1.isCollection() || t2.isCollection())
+                throw new InvalidRequestException("map type cannot contain another collection");
+            if (t1.isCounter() || t2.isCounter())
+                throw new InvalidRequestException("counters are not allowed inside a collection");
+
+            return new Collection(MapType.getInstance(t1.getType(), t2.getType()));
+        }
+
+        public static Collection list(CQL3Type t) throws InvalidRequestException
+        {
+            if (t.isCollection())
+                throw new InvalidRequestException("list type cannot contain another collection");
+            if (t.isCounter())
+                throw new InvalidRequestException("counters are not allowed inside a collection");
+
+            return new Collection(ListType.getInstance(t.getType()));
+        }
+
+        public static Collection set(CQL3Type t) throws InvalidRequestException
+        {
+            if (t.isCollection())
+                throw new InvalidRequestException("set type cannot contain another collection");
+            if (t.isCounter())
+                throw new InvalidRequestException("counters are not allowed inside a collection");
+
+            return new Collection(SetType.getInstance(t.getType()));
+        }
+
+        public boolean isCollection()
+        {
+            return true;
+        }
+
+        public AbstractType<?> getType()
+        {
+            return type;
+        }
+
+        public boolean isCounter()
+        {
+            return false;
+        }
+
+        @Override
+        public String toString()
+        {
+            switch (type.kind)
+            {
+                case LIST:
+                    return "list<" + ((ListType)type).elements.asCQL3Type() + ">";
+                case SET:
+                    return "set<" + ((SetType)type).elements.asCQL3Type() + ">";
+                case MAP:
+                    MapType mt = (MapType)type;
+                    return "set<" + mt.keys.asCQL3Type() + ", " + mt.values.asCQL3Type() + ">";
+            }
+            throw new AssertionError();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Cql.g b/src/java/org/apache/cassandra/cql3/Cql.g
index 7f587f1..e1abd36 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -662,8 +662,8 @@ map_literal returns [Map<Term, Term> value]
     ;
 
 finalTerm returns [Term term]
-    : t=(STRING_LITERAL | UUID | INTEGER | FLOAT | K_TRUE | K_FALSE ) { $term = new Term($t.text, $t.type); }
-    | f=(K_MIN_TIMEUUID | K_MAX_TIMEUUID | K_NOW) '(' (v=(STRING_LITERAL | INTEGER))? ')' { $term = new Term($f.text + "(" + ($v == null ? "" : $v.text) + ")", UUID); }
+    : t=(STRING_LITERAL | UUID | INTEGER | FLOAT | BOOLEAN | HEXNUMBER ) { $term = new Term($t.text, $t.type); }
+    | f=(K_MIN_TIMEUUID | K_MAX_TIMEUUID | K_NOW) '(' (v=(STRING_LITERAL | INTEGER))? ')' { $term = new Term($f.text + "(" + ($v == null ? "" : $v.text) + ")", Term.Type.UUID, true); }
     ;
 
 term returns [Term term]
@@ -741,15 +741,15 @@ property[PropertyDefinitions props]
     ;
 
 propertyValue returns [String str]
-    : v=(STRING_LITERAL | IDENT | INTEGER | FLOAT | K_TRUE | K_FALSE) { $str = $v.text; }
+    : v=(STRING_LITERAL | IDENT | INTEGER | FLOAT | BOOLEAN | HEXNUMBER) { $str = $v.text; }
     | u=unreserved_keyword                         { $str = u; }
     ;
 
-// Either a string or a list of terms
-tokenDefinition returns [Pair<String, List<Term>> tkdef]
+// Either a term or a list of terms
+tokenDefinition returns [Pair<Term, List<Term>> tkdef]
     : K_TOKEN { List<Term> l = new ArrayList<Term>(); }
-         '(' t1=term { l.add(t1); } ( ',' tn=term { l.add(tn); } )*  ')' { $tkdef = Pair.<String, List<Term>>create(null, l); }
-    | t=STRING_LITERAL { $tkdef = Pair.<String, List<Term>>create($t.text, null); }
+         '(' t1=term { l.add(t1); } ( ',' tn=term { l.add(tn); } )*  ')' { $tkdef = Pair.<Term, List<Term>>create(null, l); }
+    | t=term  { $tkdef = Pair.<Term, List<Term>>create(t, null); }
     ;
 
 relation[List<Relation> clauses]
@@ -764,10 +764,10 @@ relation[List<Relation> clauses]
            }
            else
            {
-               Term str = tkd.left == null ? null : new Term(tkd.left, Term.Type.STRING);
+               Term tokenLitteral = tkd.left;
                for (int i = 0; i < l.size(); i++)
                {
-                   Term tt = str == null ? Term.tokenOf(tkd.right.get(i)) : str;
+                   Term tt = tokenLitteral == null ? Term.tokenOf(tkd.right.get(i)) : tokenLitteral;
                    $clauses.add(new Relation(l.get(i), $type.text, tt, true));
                }
            }
@@ -776,13 +776,13 @@ relation[List<Relation> clauses]
        '(' f1=term { rel.addInValue(f1); } (',' fN=term { rel.addInValue(fN); } )* ')' { $clauses.add(rel); }
     ;
 
-comparatorType returns [ParsedType t]
+comparatorType returns [CQL3Type t]
     : c=native_type     { $t = c; }
     | c=collection_type { $t = c; }
     | s=STRING_LITERAL
       {
         try {
-            $t = new ParsedType.Custom($s.text);
+            $t = new CQL3Type.Custom($s.text);
         } catch (SyntaxException e) {
             addRecognitionError("Cannot parse type " + $s.text + ": " + e.getMessage());
         } catch (ConfigurationException e) {
@@ -791,36 +791,36 @@ comparatorType returns [ParsedType t]
       }
     ;
 
-native_type returns [ParsedType t]
-    : K_ASCII     { $t = ParsedType.Native.ASCII; }
-    | K_BIGINT    { $t = ParsedType.Native.BIGINT; }
-    | K_BLOB      { $t = ParsedType.Native.BLOB; }
-    | K_BOOLEAN   { $t = ParsedType.Native.BOOLEAN; }
-    | K_COUNTER   { $t = ParsedType.Native.COUNTER; }
-    | K_DECIMAL   { $t = ParsedType.Native.DECIMAL; }
-    | K_DOUBLE    { $t = ParsedType.Native.DOUBLE; }
-    | K_FLOAT     { $t = ParsedType.Native.FLOAT; }
-    | K_INET      { $t = ParsedType.Native.INET;}
-    | K_INT       { $t = ParsedType.Native.INT; }
-    | K_TEXT      { $t = ParsedType.Native.TEXT; }
-    | K_TIMESTAMP { $t = ParsedType.Native.TIMESTAMP; }
-    | K_UUID      { $t = ParsedType.Native.UUID; }
-    | K_VARCHAR   { $t = ParsedType.Native.VARCHAR; }
-    | K_VARINT    { $t = ParsedType.Native.VARINT; }
-    | K_TIMEUUID  { $t = ParsedType.Native.TIMEUUID; }
-    ;
-
-collection_type returns [ParsedType pt]
+native_type returns [CQL3Type t]
+    : K_ASCII     { $t = CQL3Type.Native.ASCII; }
+    | K_BIGINT    { $t = CQL3Type.Native.BIGINT; }
+    | K_BLOB      { $t = CQL3Type.Native.BLOB; }
+    | K_BOOLEAN   { $t = CQL3Type.Native.BOOLEAN; }
+    | K_COUNTER   { $t = CQL3Type.Native.COUNTER; }
+    | K_DECIMAL   { $t = CQL3Type.Native.DECIMAL; }
+    | K_DOUBLE    { $t = CQL3Type.Native.DOUBLE; }
+    | K_FLOAT     { $t = CQL3Type.Native.FLOAT; }
+    | K_INET      { $t = CQL3Type.Native.INET;}
+    | K_INT       { $t = CQL3Type.Native.INT; }
+    | K_TEXT      { $t = CQL3Type.Native.TEXT; }
+    | K_TIMESTAMP { $t = CQL3Type.Native.TIMESTAMP; }
+    | K_UUID      { $t = CQL3Type.Native.UUID; }
+    | K_VARCHAR   { $t = CQL3Type.Native.VARCHAR; }
+    | K_VARINT    { $t = CQL3Type.Native.VARINT; }
+    | K_TIMEUUID  { $t = CQL3Type.Native.TIMEUUID; }
+    ;
+
+collection_type returns [CQL3Type pt]
     : K_MAP  '<' t1=comparatorType ',' t2=comparatorType '>'
         { try {
             // if we can't parse either t1 or t2, antlr will "recover" and we may have t1 or t2 null.
             if (t1 != null && t2 != null)
-                $pt = ParsedType.Collection.map(t1, t2);
+                $pt = CQL3Type.Collection.map(t1, t2);
           } catch (InvalidRequestException e) { addRecognitionError(e.getMessage()); } }
     | K_LIST '<' t=comparatorType '>'
-        { try { if (t != null) $pt = ParsedType.Collection.list(t); } catch (InvalidRequestException e) { addRecognitionError(e.getMessage()); } }
+        { try { if (t != null) $pt = CQL3Type.Collection.list(t); } catch (InvalidRequestException e) { addRecognitionError(e.getMessage()); } }
     | K_SET  '<' t=comparatorType '>'
-        { try { if (t != null) $pt = ParsedType.Collection.set(t); } catch (InvalidRequestException e) { addRecognitionError(e.getMessage()); } }
+        { try { if (t != null) $pt = CQL3Type.Collection.set(t); } catch (InvalidRequestException e) { addRecognitionError(e.getMessage()); } }
     ;
 
 username
@@ -945,9 +945,6 @@ K_WRITETIME:   W R I T E T I M E;
 K_MAP:         M A P;
 K_LIST:        L I S T;
 
-K_TRUE:        T R U E;
-K_FALSE:       F A L S E;
-
 K_MIN_TIMEUUID:     M I N T I M E U U I D;
 K_MAX_TIMEUUID:     M A X T I M E U U I D;
 K_NOW:              N O W;
@@ -1027,10 +1024,21 @@ FLOAT
     | INTEGER '.' DIGIT* EXPONENT?
     ;
 
+/*
+ * This has to be before IDENT so it takes precendence over it.
+ */
+BOOLEAN
+    : T R U E | F A L S E
+    ;
+
 IDENT
     : LETTER (LETTER | DIGIT | '_')*
     ;
 
+HEXNUMBER
+    : '0' X HEX+
+    ;
+
 UUID
     : HEX HEX HEX HEX HEX HEX HEX HEX '-'
       HEX HEX HEX HEX '-'

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/cql3/ParsedType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ParsedType.java b/src/java/org/apache/cassandra/cql3/ParsedType.java
deleted file mode 100644
index d8d6e12..0000000
--- a/src/java/org/apache/cassandra/cql3/ParsedType.java
+++ /dev/null
@@ -1,152 +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.cql3;
-
-import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.SyntaxException;
-
-public interface ParsedType
-{
-    public boolean isCollection();
-    public boolean isCounter();
-    public AbstractType<?> getType();
-
-    public enum Native implements ParsedType
-    {
-        ASCII    (AsciiType.instance),
-        BIGINT   (LongType.instance),
-        BLOB     (BytesType.instance),
-        BOOLEAN  (BooleanType.instance),
-        COUNTER  (CounterColumnType.instance),
-        DECIMAL  (DecimalType.instance),
-        DOUBLE   (DoubleType.instance),
-        FLOAT    (FloatType.instance),
-        INET     (InetAddressType.instance),
-        INT      (Int32Type.instance),
-        TEXT     (UTF8Type.instance),
-        TIMESTAMP(DateType.instance),
-        UUID     (UUIDType.instance),
-        VARCHAR  (UTF8Type.instance),
-        VARINT   (IntegerType.instance),
-        TIMEUUID (TimeUUIDType.instance);
-
-        private final AbstractType<?> type;
-
-        private Native(AbstractType<?> type)
-        {
-            this.type = type;
-        }
-
-        public boolean isCollection()
-        {
-            return false;
-        }
-
-        public AbstractType<?> getType()
-        {
-            return type;
-        }
-
-        public boolean isCounter()
-        {
-            return this == COUNTER;
-        }
-    }
-
-    public static class Custom implements ParsedType
-    {
-        private final AbstractType<?> type;
-
-        public Custom(String className) throws SyntaxException, ConfigurationException
-        {
-            this.type = TypeParser.parse(className);
-        }
-
-        public boolean isCollection()
-        {
-            return false;
-        }
-
-        public AbstractType<?> getType()
-        {
-            return type;
-        }
-
-        public boolean isCounter()
-        {
-            return false;
-        }
-    }
-
-    public static class Collection implements ParsedType
-    {
-        CollectionType type;
-
-        private Collection(CollectionType type)
-        {
-            this.type = type;
-        }
-
-        public static Collection map(ParsedType t1, ParsedType t2) throws InvalidRequestException
-        {
-            if (t1.isCollection() || t2.isCollection())
-                throw new InvalidRequestException("map type cannot contain another collection");
-            if (t1.isCounter() || t2.isCounter())
-                throw new InvalidRequestException("counters are not allowed inside a collection");
-
-            return new Collection(MapType.getInstance(t1.getType(), t2.getType()));
-        }
-
-        public static Collection list(ParsedType t) throws InvalidRequestException
-        {
-            if (t.isCollection())
-                throw new InvalidRequestException("list type cannot contain another collection");
-            if (t.isCounter())
-                throw new InvalidRequestException("counters are not allowed inside a collection");
-
-            return new Collection(ListType.getInstance(t.getType()));
-        }
-
-        public static Collection set(ParsedType t) throws InvalidRequestException
-        {
-            if (t.isCollection())
-                throw new InvalidRequestException("set type cannot contain another collection");
-            if (t.isCounter())
-                throw new InvalidRequestException("counters are not allowed inside a collection");
-
-            return new Collection(SetType.getInstance(t.getType()));
-        }
-
-        public boolean isCollection()
-        {
-            return true;
-        }
-
-        public AbstractType<?> getType()
-        {
-            return type;
-        }
-
-        public boolean isCounter()
-        {
-            return false;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/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 cfea259..b3c312e 100644
--- a/src/java/org/apache/cassandra/cql3/Term.java
+++ b/src/java/org/apache/cassandra/cql3/Term.java
@@ -20,8 +20,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.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.db.marshal.MarshalException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 
@@ -30,11 +32,11 @@ public class Term
 {
     public enum Type
     {
-        STRING, INTEGER, UUID, FLOAT, BOOLEAN, QMARK;
+        STRING, INTEGER, UUID, FLOAT, BOOLEAN, HEX, QMARK;
 
         static Type forInt(int type)
         {
-            if ((type == CqlParser.STRING_LITERAL) || (type == CqlParser.IDENT))
+            if (type == CqlParser.STRING_LITERAL)
                 return STRING;
             else if (type == CqlParser.INTEGER)
                 return INTEGER;
@@ -42,13 +44,14 @@ public class Term
                 return UUID;
             else if (type == CqlParser.FLOAT)
                 return FLOAT;
-            else if (type == CqlParser.K_TRUE || type == CqlParser.K_FALSE)
+            else if (type == CqlParser.BOOLEAN)
                 return BOOLEAN;
+            else if (type == CqlParser.HEXNUMBER)
+                return HEX;
             else if (type == CqlParser.QMARK)
                 return QMARK;
 
-            // FIXME: handled scenario that should never occur.
-            return null;
+            throw new AssertionError();
         }
     }
 
@@ -57,17 +60,27 @@ public class Term
     public final int bindIndex;
     public final boolean isToken;
 
-    private Term(String text, Type type, int bindIndex, boolean isToken)
+    // This is a hack for the timeuuid functions (minTimeuuid, maxTimeuuid, now) because instead of handling them as
+    // true function we let the TimeUUID.fromString() method handle it. We should probably clean that up someday
+    private final boolean skipTypeValidation;
+
+    private Term(String text, Type type, int bindIndex, boolean isToken, boolean skipTypeValidation)
     {
-        this.text = text == null ? "" : text;
+        this.text = text;
         this.type = type;
         this.bindIndex = bindIndex;
         this.isToken = isToken;
+        this.skipTypeValidation = skipTypeValidation;
+    }
+
+    public Term(String text, Type type, boolean skipTypeValidation)
+    {
+        this(text, type, -1, false, skipTypeValidation);
     }
 
     public Term(String text, Type type)
     {
-        this(text, type, -1, false);
+        this(text, type, -1, false, false);
     }
 
     /**
@@ -89,12 +102,12 @@ public class Term
 
     public Term(String text, int type, int index)
     {
-        this(text, Type.forInt(type), index, false);
+        this(text, Type.forInt(type), index, false, false);
     }
 
     public static Term tokenOf(Term t)
     {
-        return new Term(t.text, t.type, t.bindIndex, true);
+        return new Term(t.text, t.type, t.bindIndex, true, false);
     }
 
     /**
@@ -119,7 +132,12 @@ public class Term
         try
         {
             if (!isBindMarker())
+            {
+                // BytesType doesn't want it's input prefixed by '0x'.
+                if (type == Type.HEX && validator instanceof BytesType)
+                    return validator.fromString(text.substring(2));
                 return validator.fromString(text);
+            }
 
             // must be a marker term so check for a CqlBindValue stored in the term
             if (bindIndex == -1)
@@ -138,6 +156,23 @@ public class Term
         }
     }
 
+    public void validateType(String identifier, AbstractType<?> validator) throws InvalidRequestException
+    {
+        if (skipTypeValidation)
+            return;
+
+        Set<Type> supported = validator.supportedCQL3Constants();
+        // Treat null specially as this mean "I don't have a supportedCQL3Type method"
+        if (supported == null)
+            return;
+
+        if (!supported.contains(type))
+        {
+            // TODO: Ideallly we'd keep the declared CQL3 type of columns and use that in the following message, instead of the AbstracType class name.
+            throw new InvalidRequestException(String.format("Invalid %s constant for %s of type %s", type, identifier, validator.asCQL3Type()));
+        }
+    }
+
     /**
      * Obtain the term's type.
      *

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/cql3/operations/ColumnOperation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/operations/ColumnOperation.java b/src/java/org/apache/cassandra/cql3/operations/ColumnOperation.java
index 224829f..d53a6b0 100644
--- a/src/java/org/apache/cassandra/cql3/operations/ColumnOperation.java
+++ b/src/java/org/apache/cassandra/cql3/operations/ColumnOperation.java
@@ -112,10 +112,14 @@ public class ColumnOperation implements Operation
         cf.addCounter(new QueryPath(cf.metadata().cfName, null, builder.build()), val);
     }
 
-    public void addBoundNames(ColumnSpecification column, ColumnSpecification[] boundNames) throws InvalidRequestException
+    public Operation validateAndAddBoundNames(ColumnSpecification column, ColumnSpecification[] boundNames) throws InvalidRequestException
     {
+        value.validateType(column.name.toString(), column.type);
+
         if (value.isBindMarker())
             boundNames[value.bindIndex] = column;
+
+        return this;
     }
 
     public List<Term> getValues()
@@ -123,7 +127,6 @@ public class ColumnOperation implements Operation
         return Collections.singletonList(value);
     }
 
-
     public boolean requiresRead(AbstractType<?> validator)
     {
         return false;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/cql3/operations/ListOperation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/operations/ListOperation.java b/src/java/org/apache/cassandra/cql3/operations/ListOperation.java
index 1e09195..e5dd953 100644
--- a/src/java/org/apache/cassandra/cql3/operations/ListOperation.java
+++ b/src/java/org/apache/cassandra/cql3/operations/ListOperation.java
@@ -285,26 +285,29 @@ public class ListOperation implements Operation
         cf.addColumn(params.makeTombstone(list.get(idx).right.name()));
     }
 
-    public void addBoundNames(ColumnSpecification column, ColumnSpecification[] boundNames) throws InvalidRequestException
+    public Operation validateAndAddBoundNames(ColumnSpecification column, ColumnSpecification[] boundNames) throws InvalidRequestException
     {
         // Since the parser couldn't disambiguate between a 'list set by idx'
         // and a 'map put by key', we have to do it now.
         if (kind == Kind.SET_IDX && (column.type instanceof MapType))
         {
             assert values.size() == 2;
-            MapOperation.Put(values.get(0), values.get(1)).addBoundNames(column, boundNames);
-            return;
+            return MapOperation.Put(values.get(0), values.get(1)).validateAndAddBoundNames(column, boundNames);
         }
 
         if (!(column.type instanceof ListType))
-            throw new InvalidRequestException(String.format("Invalid operation, %s is not of list type", column.name));
+            throw new InvalidRequestException(String.format("Cannot apply list operation on column %s of type %s", column, column.type));
 
         ListType lt = (ListType)column.type;
         if (kind == Kind.SET_IDX)
         {
             assert values.size() == 2;
             Term idx = values.get(0);
+            idx.validateType("list index", Int32Type.instance);
+
             Term value = values.get(1);
+            value.validateType(column + " element", lt.elements);
+
             if (idx.isBindMarker())
                 boundNames[idx.bindIndex] = indexSpecOf(column);
             if (value.isBindMarker())
@@ -313,9 +316,13 @@ public class ListOperation implements Operation
         else
         {
             for (Term t : values)
+            {
+                t.validateType(column + " element", lt.elements);
                 if (t.isBindMarker())
                     boundNames[t.bindIndex] = column;
+            }
         }
+        return this;
     }
 
     public static ColumnSpecification indexSpecOf(ColumnSpecification column)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/cql3/operations/MapOperation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/operations/MapOperation.java b/src/java/org/apache/cassandra/cql3/operations/MapOperation.java
index ddded47..fcd0ec7 100644
--- a/src/java/org/apache/cassandra/cql3/operations/MapOperation.java
+++ b/src/java/org/apache/cassandra/cql3/operations/MapOperation.java
@@ -129,21 +129,26 @@ public class MapOperation implements Operation
         cf.addColumn(params.makeTombstone(name));
     }
 
-    public void addBoundNames(ColumnSpecification column, ColumnSpecification[] boundNames) throws InvalidRequestException
+    public Operation validateAndAddBoundNames(ColumnSpecification column, ColumnSpecification[] boundNames) throws InvalidRequestException
     {
         if (!(column.type instanceof MapType))
-            throw new InvalidRequestException(String.format("Invalid operation, %s is not of map type", column.name));
+            throw new InvalidRequestException(String.format("Cannot apply map operation on column %s of type %s", column, column.type));
 
         MapType mt = (MapType)column.type;
         for (Map.Entry<Term, Term> entry : values.entrySet())
         {
             Term key = entry.getKey();
+            key.validateType(column + " key", mt.keys);
+
             Term value = entry.getValue();
+            value.validateType(column + " value", mt.values);
+
             if (key.isBindMarker())
                 boundNames[key.bindIndex] = keySpecOf(column, mt);
             if (value.isBindMarker())
                 boundNames[value.bindIndex] = valueSpecOf(column, mt);
         }
+        return this;
     }
 
     public static ColumnSpecification keySpecOf(ColumnSpecification column, MapType type)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/cql3/operations/Operation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/operations/Operation.java b/src/java/org/apache/cassandra/cql3/operations/Operation.java
index 6c30f7c..98ec503 100644
--- a/src/java/org/apache/cassandra/cql3/operations/Operation.java
+++ b/src/java/org/apache/cassandra/cql3/operations/Operation.java
@@ -41,7 +41,7 @@ public interface Operation
                         UpdateParameters params,
                         List<Pair<ByteBuffer, IColumn>> list) throws InvalidRequestException;
 
-    public void addBoundNames(ColumnSpecification column, ColumnSpecification[] boundNames) throws InvalidRequestException;
+    public Operation validateAndAddBoundNames(ColumnSpecification column, ColumnSpecification[] boundNames) throws InvalidRequestException;
 
     public List<Term> getValues();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/cql3/operations/PreparedOperation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/operations/PreparedOperation.java b/src/java/org/apache/cassandra/cql3/operations/PreparedOperation.java
index 969e63c..77cd183 100644
--- a/src/java/org/apache/cassandra/cql3/operations/PreparedOperation.java
+++ b/src/java/org/apache/cassandra/cql3/operations/PreparedOperation.java
@@ -26,6 +26,7 @@ import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.IColumn;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.CounterColumnType;
 import org.apache.cassandra.db.marshal.ListType;
 import org.apache.cassandra.db.marshal.MapType;
 import org.apache.cassandra.db.marshal.SetType;
@@ -79,8 +80,6 @@ public class PreparedOperation implements Operation
                         case SET:
                             SetOperation.doSetFromPrepared(cf, builder, (SetType)validator, preparedValue, params);
                             break;
-                        case PREPARED_PLUS:
-                            throw new InvalidRequestException("Unsupported syntax, cannot add to a prepared set");
                         case PLUS_PREPARED:
                             SetOperation.doAddFromPrepared(cf, builder, (SetType)validator, preparedValue, params);
                             break;
@@ -95,13 +94,9 @@ public class PreparedOperation implements Operation
                         case SET:
                             MapOperation.doSetFromPrepared(cf, builder, (MapType)validator, preparedValue, params);
                             break;
-                        case PREPARED_PLUS:
-                            throw new InvalidRequestException("Unsupported syntax, cannot put to a prepared map");
                         case PLUS_PREPARED:
                             MapOperation.doPutFromPrepared(cf, builder, (MapType)validator, preparedValue, params);
                             break;
-                        case MINUS_PREPARED:
-                            throw new InvalidRequestException("Unsuppoted syntax, discard syntax for map not supported");
                     }
                     break;
             }
@@ -113,8 +108,6 @@ public class PreparedOperation implements Operation
                 case SET:
                     ColumnOperation.Set(preparedValue).execute(cf, builder, validator, params, null);
                     break;
-                case PREPARED_PLUS:
-                    throw new InvalidRequestException("Unsupported syntax for increment, must be of the form X = X + <value>");
                 case PLUS_PREPARED:
                     ColumnOperation.CounterInc(preparedValue).execute(cf, builder, validator, params, null);
                     break;
@@ -125,10 +118,42 @@ public class PreparedOperation implements Operation
         }
     }
 
-    public void addBoundNames(ColumnSpecification column, ColumnSpecification[] boundNames) throws InvalidRequestException
+    public Operation validateAndAddBoundNames(ColumnSpecification column, ColumnSpecification[] boundNames) throws InvalidRequestException
     {
+        if (column.type instanceof CollectionType)
+        {
+            switch (kind)
+            {
+                case PREPARED_PLUS:
+                    if (column.type instanceof MapType)
+                        throw new InvalidRequestException("Unsupported syntax, cannot put to a prepared map");
+                    if (column.type instanceof SetType)
+                        throw new InvalidRequestException("Unsupported syntax, cannot add to a prepared set");
+                    break;
+                case MINUS_PREPARED:
+                    if (column.type instanceof MapType)
+                        throw new InvalidRequestException("Unsuppoted syntax, discard syntax for map not supported");
+                    break;
+            }
+            switch (((CollectionType)column.type).kind)
+            {
+            }
+        }
+        else if (column.type instanceof CounterColumnType)
+        {
+            if (kind == Kind.PREPARED_PLUS)
+                throw new InvalidRequestException("Unsupported syntax for increment, must be of the form X = X + <value>");
+        }
+        else
+        {
+            // Any other operation than a set is invalid
+            if (kind != Kind.SET)
+                throw new InvalidRequestException(String.format("Invalid operation for %s of type %s", column, column.type));
+        }
+
         if (preparedValue.isBindMarker())
             boundNames[preparedValue.bindIndex] = column;
+        return this;
     }
 
     public List<Term> getValues()
@@ -142,7 +167,8 @@ public class PreparedOperation implements Operation
         return (validator instanceof ListType) && kind == Kind.MINUS_PREPARED;
     }
 
-    public boolean isPotentialCounterOperation() {
+    public boolean isPotentialCounterOperation()
+    {
         return kind == Kind.PLUS_PREPARED || kind == Kind.MINUS_PREPARED;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/cql3/operations/SetOperation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/operations/SetOperation.java b/src/java/org/apache/cassandra/cql3/operations/SetOperation.java
index bec0e1a..30a96ae 100644
--- a/src/java/org/apache/cassandra/cql3/operations/SetOperation.java
+++ b/src/java/org/apache/cassandra/cql3/operations/SetOperation.java
@@ -33,6 +33,7 @@ import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.CollectionType;
 import org.apache.cassandra.db.marshal.MarshalException;
 import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.marshal.MapType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
@@ -74,25 +75,6 @@ public class SetOperation implements Operation
         }
     }
 
-    public Operation maybeConvertToEmptyMapOperation()
-    {
-        // If it's not empty or a DISCARD, it's a proper invalid query, not
-        // just the parser that hasn't been able to distinguish empty set from
-        // empty map. However, we just this as it will be rejected later and
-        // there is no point in duplicating validation
-        if (!values.isEmpty())
-            return this;
-
-        switch (kind)
-        {
-            case SET:
-                return MapOperation.Set(Collections.<Term, Term>emptyMap());
-            case ADD:
-                return MapOperation.Put(Collections.<Term, Term>emptyMap());
-        }
-        return this;
-    }
-
     public static void doSetFromPrepared(ColumnFamily cf, ColumnNameBuilder builder, SetType validator, Term values, UpdateParameters params) throws InvalidRequestException
     {
         if (!values.isBindMarker())
@@ -166,11 +148,38 @@ public class SetOperation implements Operation
         }
     }
 
-    public void addBoundNames(ColumnSpecification column, ColumnSpecification[] boundNames) throws InvalidRequestException
+    public Operation validateAndAddBoundNames(ColumnSpecification column, ColumnSpecification[] boundNames) throws InvalidRequestException
     {
+        // On the parser side, we're unable to differentiate an empty map from an empty set for add and set operations.
+        // Fix it now that we have the actual type.
+        if (column.type instanceof MapType && values.isEmpty())
+            return toEmptyMapOperation().validateAndAddBoundNames(column, boundNames);
+
+        if (!(column.type instanceof SetType))
+            throw new InvalidRequestException(String.format("Cannot apply set operation on column %s of type %s", column, column.type));
+
+        AbstractType<?> valuesType = ((SetType)column.type).elements;
+
         for (Term t : values)
+        {
+            t.validateType(column + " element", valuesType);
+
             if (t.isBindMarker())
                 boundNames[t.bindIndex] = column;
+        }
+        return this;
+    }
+
+    private Operation toEmptyMapOperation()
+    {
+        switch (kind)
+        {
+            case SET:
+                return MapOperation.Set(Collections.<Term, Term>emptyMap());
+            case ADD:
+                return MapOperation.Put(Collections.<Term, Term>emptyMap());
+        }
+        return this;
     }
 
     public List<Term> getValues()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/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 7381895..b07a8a8 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
@@ -43,12 +43,12 @@ public class AlterTableStatement extends SchemaAlteringStatement
     }
 
     public final Type oType;
-    public final ParsedType validator;
+    public final CQL3Type validator;
     public final ColumnIdentifier columnName;
     private final CFPropDefs cfProps;
     private final Map<ColumnIdentifier, ColumnIdentifier> renames;
 
-    public AlterTableStatement(CFName name, Type type, ColumnIdentifier columnName, ParsedType validator, CFPropDefs cfProps, Map<ColumnIdentifier, ColumnIdentifier> renames)
+    public AlterTableStatement(CFName name, Type type, ColumnIdentifier columnName, CQL3Type validator, CFPropDefs cfProps, Map<ColumnIdentifier, ColumnIdentifier> renames)
     {
         super(name);
         this.oType = type;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/cql3/statements/CreateColumnFamilyStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateColumnFamilyStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateColumnFamilyStatement.java
index 483e083..f2c3d6a 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateColumnFamilyStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateColumnFamilyStatement.java
@@ -142,7 +142,7 @@ public class CreateColumnFamilyStatement extends SchemaAlteringStatement
 
     public static class RawStatement extends CFStatement
     {
-        private final Map<ColumnIdentifier, ParsedType> definitions = new HashMap<ColumnIdentifier, ParsedType>();
+        private final Map<ColumnIdentifier, CQL3Type> definitions = new HashMap<ColumnIdentifier, CQL3Type>();
         public final CFPropDefs properties = new CFPropDefs();
 
         private final List<List<ColumnIdentifier>> keyAliases = new ArrayList<List<ColumnIdentifier>>();
@@ -178,10 +178,10 @@ public class CreateColumnFamilyStatement extends SchemaAlteringStatement
             stmt.setBoundTerms(getBoundsTerms());
 
             Map<ByteBuffer, CollectionType> definedCollections = null;
-            for (Map.Entry<ColumnIdentifier, ParsedType> entry : definitions.entrySet())
+            for (Map.Entry<ColumnIdentifier, CQL3Type> entry : definitions.entrySet())
             {
                 ColumnIdentifier id = entry.getKey();
-                ParsedType pt = entry.getValue();
+                CQL3Type pt = entry.getValue();
                 if (pt.isCollection())
                 {
                     if (definedCollections == null)
@@ -350,7 +350,7 @@ public class CreateColumnFamilyStatement extends SchemaAlteringStatement
             return isReversed != null && isReversed ? ReversedType.getInstance(type) : type;
         }
 
-        public void addDefinition(ColumnIdentifier def, ParsedType type)
+        public void addDefinition(ColumnIdentifier def, CQL3Type type)
         {
             definedNames.add(def);
             definitions.put(def, type);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/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 d8369b7..ae47a40 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
@@ -217,15 +217,19 @@ public class DeleteStatement extends ModificationStatement
             if (name.kind != CFDefinition.Name.Kind.COLUMN_METADATA && name.kind != CFDefinition.Name.Kind.VALUE_ALIAS)
                 throw new InvalidRequestException(String.format("Invalid identifier %s for deletion (should not be a PRIMARY KEY part)", column));
 
-            if (column.key() != null)
+            if (column.hasKey())
             {
                 if (name.type instanceof ListType)
                 {
+                    column.key().validateType("list index", Int32Type.instance);
+
                     if (column.key().isBindMarker())
                         boundNames[column.key().bindIndex] = ListOperation.indexSpecOf(name);
                 }
                 else if (name.type instanceof MapType)
                 {
+                    column.key().validateType("map key", ((MapType)name.type).keys);
+
                     if (column.key().isBindMarker())
                         boundNames[column.key().bindIndex] = MapOperation.keySpecOf(name, (MapType)name.type);
                 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/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 1cf6d01..8f74b72 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -371,7 +371,7 @@ public class SelectStatement implements CQLStatement
         if (t == null)
             return p.getMinimumToken();
 
-        if (t.getType() == Term.Type.STRING && !t.isToken)
+        if (!t.isToken)
         {
             try
             {
@@ -1026,6 +1026,9 @@ public class SelectStatement implements CQLStatement
                         }
                     }
 
+                    if (t.hasKey())
+                        throw new InvalidRequestException("Selecting a list/map element by index/key is not yet supported");
+
                     stmt.selectedNames.add(Pair.create(name, t));
                 }
             }
@@ -1047,12 +1050,22 @@ public class SelectStatement implements CQLStatement
                 if (rel.operator() == Relation.Type.IN)
                 {
                     for (Term value : rel.getInValues())
+                    {
+                        if (!rel.onToken || value.isToken)
+                            value.validateType(name.toString(), name.type);
+                        else
+                            value.validateType("token of " + name.toString(), StorageService.getPartitioner().getTokenValidator());
                         if (value.isBindMarker())
                             names[value.bindIndex] = name;
+                    }
                 }
                 else
                 {
                     Term value = rel.getValue();
+                    if (!rel.onToken || value.isToken)
+                        value.validateType(name.toString(), name.type);
+                    else
+                        value.validateType("token of " + name.toString(), StorageService.getPartitioner().getTokenValidator());
                     if (value.isBindMarker())
                         names[value.bindIndex] = name;
                 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/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 7db2bdb..aac4f04 100644
--- a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
@@ -285,8 +285,7 @@ public class UpdateStatement extends ModificationStatement
                 if (name == null)
                     throw new InvalidRequestException(String.format("Unknown identifier %s", columnNames.get(i)));
 
-                Operation operation = columnOperations.get(i);
-                operation.addBoundNames(name, boundNames);
+                Operation operation = columnOperations.get(i).validateAndAddBoundNames(name, boundNames);
 
                 switch (name.kind)
                 {
@@ -303,7 +302,7 @@ public class UpdateStatement extends ModificationStatement
                     case COLUMN_METADATA:
                         if (processedColumns.containsKey(name))
                             throw new InvalidRequestException(String.format("Multiple definitions found for column %s", name));
-                        addNewOperation(name, operation);
+                        processedColumns.put(name, operation);
                         break;
                 }
             }
@@ -317,7 +316,7 @@ public class UpdateStatement extends ModificationStatement
                 if (name == null)
                     throw new InvalidRequestException(String.format("Unknown identifier %s", entry.left));
 
-                Operation operation = entry.right;
+                Operation operation = entry.right.validateAndAddBoundNames(name, boundNames);
 
                 switch (operation.getType())
                 {
@@ -328,9 +327,6 @@ public class UpdateStatement extends ModificationStatement
                     case LIST:
                     case SET:
                     case MAP:
-                        if (!name.type.isCollection())
-                            throw new InvalidRequestException("Cannot apply collection operation on column " + name + " with " + name.type + " type.");
-                    // Fallthrough on purpose
                     case COLUMN:
                         if (type == Type.COUNTER)
                             throw new InvalidRequestException("Invalid non-counter operation on counter table.");
@@ -351,9 +347,7 @@ public class UpdateStatement extends ModificationStatement
                         for (Operation otherOp : processedColumns.get(name))
                             if (otherOp.getType() == Operation.Type.COLUMN)
                                 throw new InvalidRequestException(String.format("Multiple definitions found for column %s", name));
-
-                        operation.addBoundNames(name, boundNames);
-                        addNewOperation(name, operation);
+                        processedColumns.put(name, operation);
                         break;
                 }
             }
@@ -363,16 +357,6 @@ public class UpdateStatement extends ModificationStatement
         return new ParsedStatement.Prepared(this, Arrays.<ColumnSpecification>asList(boundNames));
     }
 
-    private void addNewOperation(CFDefinition.Name name, Operation operation)
-    {
-        // On the parser side, we're unable to differentiate an empty map from an empty set for add and set operations.
-        // Fix it now that we have the actual type.
-        if (operation.getType() == Operation.Type.SET && (name.type instanceof MapType))
-            operation = ((SetOperation)operation).maybeConvertToEmptyMapOperation();
-
-        processedColumns.put(name, operation);
-    }
-
     public ParsedStatement.Prepared prepare() throws InvalidRequestException
     {
         ColumnSpecification[] names = new ColumnSpecification[getBoundsTerms()];
@@ -403,8 +387,11 @@ public class UpdateStatement extends ModificationStatement
                     if (processed.containsKey(name.name))
                         throw new InvalidRequestException(String.format("Multiple definitions found for PRIMARY KEY part %s", name));
                     for (Term value : values)
+                    {
+                        value.validateType(name.toString(), name.type);
                         if (value.isBindMarker())
                             names[value.bindIndex] = name;
+                    }
                     processed.put(name.name, values);
                     break;
                 case VALUE_ALIAS:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/db/SystemTable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemTable.java b/src/java/org/apache/cassandra/db/SystemTable.java
index 8d4bef2..9dad12e 100644
--- a/src/java/org/apache/cassandra/db/SystemTable.java
+++ b/src/java/org/apache/cassandra/db/SystemTable.java
@@ -205,7 +205,7 @@ public class SystemTable
         {
             throw new RuntimeException(e);
         }
-        return String.format("{'%s': '%s'}",
+        return String.format("{%s: 0x%s}",
                              cfs.metadata.cfId,
                              ByteBufferUtil.bytesToHex(ByteBuffer.wrap(out.getData(), 0, out.getLength())));
     }
@@ -263,7 +263,7 @@ public class SystemTable
         if (ep.equals(FBUtilities.getBroadcastAddress()))
             return;
 
-        String req = "INSERT INTO system.%s (peer, %s) VALUES ('%s', '%s')";
+        String req = "INSERT INTO system.%s (peer, %s) VALUES ('%s', %s)";
         processInternal(String.format(req, PEERS_CF, columnName, ep.getHostAddress(), value));
     }
 
@@ -571,7 +571,7 @@ public class SystemTable
         hostId = UUID.randomUUID();
         logger.warn("No host ID found, created {} (Note: This should happen exactly once per node).", hostId);
 
-        req = "INSERT INTO system.%s (key, host_id) VALUES ('%s', '%s')";
+        req = "INSERT INTO system.%s (key, host_id) VALUES ('%s', %s)";
         processInternal(String.format(req, LOCAL_CF, LOCAL_KEY, hostId));
         return hostId;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/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 828926d..5c4c03e 100644
--- a/src/java/org/apache/cassandra/db/marshal/AbstractType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
@@ -21,7 +21,10 @@ import java.nio.ByteBuffer;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.Map;
+import java.util.Set;
 
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.db.IColumn;
 import org.apache.cassandra.db.OnDiskAtom;
@@ -147,6 +150,18 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>
     /* validate that the byte array is a valid sequence for the type we are supposed to be comparing */
     public abstract void validate(ByteBuffer bytes) throws MarshalException;
 
+    /* CQL3 types will actually override this, but we use a default for compatibility sake */
+    public Set<Term.Type> supportedCQL3Constants()
+    {
+        return null;
+    }
+
+    /* Most of our internal type should override that. */
+    public CQL3Type asCQL3Type()
+    {
+        return new CQL3Type.Custom(this);
+    }
+
     /** @deprecated use reverseComparator field instead */
     public Comparator<ByteBuffer> getReverseComparator()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/db/marshal/AsciiType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/AsciiType.java b/src/java/org/apache/cassandra/db/marshal/AsciiType.java
index 460d882..c1762cc 100644
--- a/src/java/org/apache/cassandra/db/marshal/AsciiType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AsciiType.java
@@ -18,13 +18,19 @@
 package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
+import java.util.EnumSet;
+import java.util.Set;
 
 import org.apache.cassandra.cql.jdbc.JdbcAscii;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Term;
 
 public class AsciiType extends AbstractType<String>
 {
     public static final AsciiType instance = new AsciiType();
 
+    private final Set<Term.Type> supportedCQL3Constants = EnumSet.of(Term.Type.STRING);
+
     AsciiType() {} // singleton
 
     public String getString(ByteBuffer bytes)
@@ -69,4 +75,14 @@ public class AsciiType extends AbstractType<String>
                 throw new MarshalException("Invalid byte for ascii: " + Byte.toString(b));
         }
     }
+
+    public Set<Term.Type> supportedCQL3Constants()
+    {
+        return supportedCQL3Constants;
+    }
+
+    public CQL3Type asCQL3Type()
+    {
+        return CQL3Type.Native.ASCII;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/db/marshal/BooleanType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/BooleanType.java b/src/java/org/apache/cassandra/db/marshal/BooleanType.java
index aeaede4..87b1183 100644
--- a/src/java/org/apache/cassandra/db/marshal/BooleanType.java
+++ b/src/java/org/apache/cassandra/db/marshal/BooleanType.java
@@ -18,13 +18,19 @@
 package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
+import java.util.EnumSet;
+import java.util.Set;
 
 import org.apache.cassandra.cql.jdbc.JdbcBoolean;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Term;
 
 public class BooleanType extends AbstractType<Boolean>
 {
   public static final BooleanType instance = new BooleanType();
 
+  public final Set<Term.Type> supportedCQL3Constants = EnumSet.of(Term.Type.BOOLEAN);
+
   BooleanType() {} // singleton
 
   public Boolean compose(ByteBuffer bytes)
@@ -77,4 +83,14 @@ public class BooleanType extends AbstractType<Boolean>
       if (bytes.remaining() != 1 && bytes.remaining() != 0)
           throw new MarshalException(String.format("Expected 1 or 0 byte value (%d)", bytes.remaining()));
   }
+
+  public Set<Term.Type> supportedCQL3Constants()
+  {
+      return supportedCQL3Constants;
+  }
+
+  public CQL3Type asCQL3Type()
+  {
+      return CQL3Type.Native.BOOLEAN;
+  }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/db/marshal/BytesType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/BytesType.java b/src/java/org/apache/cassandra/db/marshal/BytesType.java
index b429b7d..a2ddbf8 100644
--- a/src/java/org/apache/cassandra/db/marshal/BytesType.java
+++ b/src/java/org/apache/cassandra/db/marshal/BytesType.java
@@ -18,8 +18,12 @@
 package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
+import java.util.EnumSet;
+import java.util.Set;
 
 import org.apache.cassandra.cql.jdbc.JdbcBytes;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Hex;
 
@@ -27,6 +31,8 @@ public class BytesType extends AbstractType<ByteBuffer>
 {
     public static final BytesType instance = new BytesType();
 
+    private final Set<Term.Type> supportedCQL3Constants = EnumSet.of(Term.Type.HEX);
+
     BytesType() {} // singleton
 
     public ByteBuffer compose(ByteBuffer bytes)
@@ -81,4 +87,14 @@ public class BytesType extends AbstractType<ByteBuffer>
         // bytesType validate everything, so it is compatible with the former.
         return this == previous || previous == AsciiType.instance || previous == UTF8Type.instance;
     }
+
+    public Set<Term.Type> supportedCQL3Constants()
+    {
+        return supportedCQL3Constants;
+    }
+
+    public CQL3Type asCQL3Type()
+    {
+        return CQL3Type.Native.BLOB;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/db/marshal/CollectionType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/CollectionType.java b/src/java/org/apache/cassandra/db/marshal/CollectionType.java
index a19912b..b448f1b 100644
--- a/src/java/org/apache/cassandra/db/marshal/CollectionType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CollectionType.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.db.marshal;
 import java.nio.ByteBuffer;
 import java.util.List;
 
+import org.apache.cassandra.cql3.CQL3Type;
 import org.apache.cassandra.db.IColumn;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
@@ -103,4 +104,10 @@ public abstract class CollectionType<T> extends AbstractType<T>
         }
         return (ByteBuffer)result.flip();
     }
+
+
+    public CQL3Type asCQL3Type()
+    {
+        return new CQL3Type.Collection(this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java b/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java
index d4b0384..8cd2457 100644
--- a/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java
@@ -18,7 +18,11 @@
 package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
+import java.util.EnumSet;
+import java.util.Set;
 
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -26,6 +30,8 @@ public class CounterColumnType extends AbstractCommutativeType
 {
     public static final CounterColumnType instance = new CounterColumnType();
 
+    private final Set<Term.Type> supportedCQL3Constants = EnumSet.of(Term.Type.INTEGER);
+
     CounterColumnType() {} // singleton
 
     public int compare(ByteBuffer o1, ByteBuffer o2)
@@ -59,4 +65,14 @@ public class CounterColumnType extends AbstractCommutativeType
         if (bytes.remaining() != 8 && bytes.remaining() != 0)
             throw new MarshalException(String.format("Expected 8 or 0 byte long (%d)", bytes.remaining()));
     }
+
+    public Set<Term.Type> supportedCQL3Constants()
+    {
+        return supportedCQL3Constants;
+    }
+
+    public CQL3Type asCQL3Type()
+    {
+        return CQL3Type.Native.COUNTER;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/db/marshal/DateType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/DateType.java b/src/java/org/apache/cassandra/db/marshal/DateType.java
index 4270511..0da1af5 100644
--- a/src/java/org/apache/cassandra/db/marshal/DateType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DateType.java
@@ -23,8 +23,12 @@ import java.nio.ByteBuffer;
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.util.Date;
+import java.util.EnumSet;
+import java.util.Set;
 
 import org.apache.cassandra.cql.jdbc.JdbcDate;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.commons.lang.time.DateUtils;
 
@@ -33,9 +37,10 @@ public class DateType extends AbstractType<Date>
     public static final DateType instance = new DateType();
 
     static final String DEFAULT_FORMAT = iso8601Patterns[3];
-
     static final SimpleDateFormat FORMATTER = new SimpleDateFormat(DEFAULT_FORMAT);
 
+    private final Set<Term.Type> supportedCQL3Constants = EnumSet.of(Term.Type.STRING, Term.Type.INTEGER);
+
     DateType() {} // singleton
 
     public Date compose(ByteBuffer bytes)
@@ -124,4 +129,14 @@ public class DateType extends AbstractType<Date>
         if (bytes.remaining() != 8 && bytes.remaining() != 0)
             throw new MarshalException(String.format("Expected 8 or 0 byte long for date (%d)", bytes.remaining()));
     }
+
+    public Set<Term.Type> supportedCQL3Constants()
+    {
+        return supportedCQL3Constants;
+    }
+
+    public CQL3Type asCQL3Type()
+    {
+        return CQL3Type.Native.TIMESTAMP;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/db/marshal/DecimalType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/DecimalType.java b/src/java/org/apache/cassandra/db/marshal/DecimalType.java
index 4639a60..75e179e 100644
--- a/src/java/org/apache/cassandra/db/marshal/DecimalType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DecimalType.java
@@ -19,14 +19,20 @@ package org.apache.cassandra.db.marshal;
 
 import java.math.BigDecimal;
 import java.nio.ByteBuffer;
+import java.util.EnumSet;
+import java.util.Set;
 
 import org.apache.cassandra.cql.jdbc.JdbcDecimal;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class DecimalType extends AbstractType<BigDecimal>
 {
     public static final DecimalType instance = new DecimalType();
 
+    private final Set<Term.Type> supportedCQL3Constants = EnumSet.of(Term.Type.INTEGER, Term.Type.FLOAT);
+
     DecimalType() {} // singleton
 
     public int compare(ByteBuffer bb0, ByteBuffer bb1)
@@ -85,4 +91,14 @@ public class DecimalType extends AbstractType<BigDecimal>
     {
         // no useful check for invalid decimals.
     }
+
+    public Set<Term.Type> supportedCQL3Constants()
+    {
+        return supportedCQL3Constants;
+    }
+
+    public CQL3Type asCQL3Type()
+    {
+        return CQL3Type.Native.DECIMAL;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/db/marshal/DoubleType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/DoubleType.java b/src/java/org/apache/cassandra/db/marshal/DoubleType.java
index 2bed8ea..23854bd 100644
--- a/src/java/org/apache/cassandra/db/marshal/DoubleType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DoubleType.java
@@ -18,14 +18,20 @@
 package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
+import java.util.EnumSet;
+import java.util.Set;
 
 import org.apache.cassandra.cql.jdbc.JdbcDouble;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class DoubleType extends AbstractType<Double>
 {
     public static final DoubleType instance = new DoubleType();
 
+    private final Set<Term.Type> supportedCQL3Constants = EnumSet.of(Term.Type.INTEGER, Term.Type.FLOAT);
+
     DoubleType() {} // singleton
 
     public Double compose(ByteBuffer bytes)
@@ -88,4 +94,14 @@ public class DoubleType extends AbstractType<Double>
         if (bytes.remaining() != 8 && bytes.remaining() != 0)
             throw new MarshalException(String.format("Expected 8 or 0 byte value for a double (%d)", bytes.remaining()));
     }
+
+    public Set<Term.Type> supportedCQL3Constants()
+    {
+        return supportedCQL3Constants;
+    }
+
+    public CQL3Type asCQL3Type()
+    {
+        return CQL3Type.Native.DOUBLE;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/db/marshal/FloatType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/FloatType.java b/src/java/org/apache/cassandra/db/marshal/FloatType.java
index 43a6eaa..95854e1 100644
--- a/src/java/org/apache/cassandra/db/marshal/FloatType.java
+++ b/src/java/org/apache/cassandra/db/marshal/FloatType.java
@@ -18,8 +18,12 @@
 package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
+import java.util.EnumSet;
+import java.util.Set;
 
 import org.apache.cassandra.cql.jdbc.JdbcFloat;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 
@@ -27,6 +31,8 @@ public class FloatType extends AbstractType<Float>
 {
     public static final FloatType instance = new FloatType();
 
+    private final Set<Term.Type> supportedCQL3Constants = EnumSet.of(Term.Type.INTEGER, Term.Type.FLOAT);
+
     FloatType() {} // singleton
 
     public Float compose(ByteBuffer bytes)
@@ -87,4 +93,14 @@ public class FloatType extends AbstractType<Float>
         if (bytes.remaining() != 4 && bytes.remaining() != 0)
             throw new MarshalException(String.format("Expected 4 or 0 byte value for a float (%d)", bytes.remaining()));
     }
+
+    public Set<Term.Type> supportedCQL3Constants()
+    {
+        return supportedCQL3Constants;
+    }
+
+    public CQL3Type asCQL3Type()
+    {
+        return CQL3Type.Native.FLOAT;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/db/marshal/InetAddressType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/InetAddressType.java b/src/java/org/apache/cassandra/db/marshal/InetAddressType.java
index f621f64..e90bdf3 100644
--- a/src/java/org/apache/cassandra/db/marshal/InetAddressType.java
+++ b/src/java/org/apache/cassandra/db/marshal/InetAddressType.java
@@ -20,14 +20,20 @@ package org.apache.cassandra.db.marshal;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
+import java.util.EnumSet;
+import java.util.Set;
 
 import org.apache.cassandra.cql.jdbc.JdbcInetAddress;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class InetAddressType extends AbstractType<InetAddress>
 {
     public static final InetAddressType instance = new InetAddressType();
 
+    private final Set<Term.Type> supportedCQL3Constants = EnumSet.of(Term.Type.STRING);
+
     InetAddressType() {} // singleton
 
     public InetAddress compose(ByteBuffer bytes)
@@ -81,4 +87,14 @@ public class InetAddressType extends AbstractType<InetAddress>
             throw new MarshalException(String.format("Expected 4 or 16 byte inetaddress; got %s", ByteBufferUtil.bytesToHex(bytes)));
         }
     }
+
+    public Set<Term.Type> supportedCQL3Constants()
+    {
+        return supportedCQL3Constants;
+    }
+
+    public CQL3Type asCQL3Type()
+    {
+        return CQL3Type.Native.INET;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/db/marshal/Int32Type.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/Int32Type.java b/src/java/org/apache/cassandra/db/marshal/Int32Type.java
index 720fa98..a945074 100644
--- a/src/java/org/apache/cassandra/db/marshal/Int32Type.java
+++ b/src/java/org/apache/cassandra/db/marshal/Int32Type.java
@@ -18,14 +18,20 @@
 package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
+import java.util.EnumSet;
+import java.util.Set;
 
 import org.apache.cassandra.cql.jdbc.JdbcInt32;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class Int32Type extends AbstractType<Integer>
 {
     public static final Int32Type instance = new Int32Type();
 
+    private final Set<Term.Type> supportedCQL3Constants = EnumSet.of(Term.Type.INTEGER);
+
     Int32Type() {} // singleton
 
     public Integer compose(ByteBuffer bytes)
@@ -95,4 +101,13 @@ public class Int32Type extends AbstractType<Integer>
             throw new MarshalException(String.format("Expected 4 or 0 byte int (%d)", bytes.remaining()));
     }
 
+    public Set<Term.Type> supportedCQL3Constants()
+    {
+        return supportedCQL3Constants;
+    }
+
+    public CQL3Type asCQL3Type()
+    {
+        return CQL3Type.Native.INT;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/db/marshal/IntegerType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/IntegerType.java b/src/java/org/apache/cassandra/db/marshal/IntegerType.java
index 0a7846a..d492ba6 100644
--- a/src/java/org/apache/cassandra/db/marshal/IntegerType.java
+++ b/src/java/org/apache/cassandra/db/marshal/IntegerType.java
@@ -19,14 +19,20 @@ package org.apache.cassandra.db.marshal;
 
 import java.math.BigInteger;
 import java.nio.ByteBuffer;
+import java.util.EnumSet;
+import java.util.Set;
 
 import org.apache.cassandra.cql.jdbc.JdbcInteger;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 public final class IntegerType extends AbstractType<BigInteger>
 {
     public static final IntegerType instance = new IntegerType();
 
+    private final Set<Term.Type> supportedCQL3Constants = EnumSet.of(Term.Type.INTEGER);
+
     private static int findMostSignificantByte(ByteBuffer bytes)
     {
         int len = bytes.remaining() - 1;
@@ -152,4 +158,14 @@ public final class IntegerType extends AbstractType<BigInteger>
     {
         // no invalid integers.
     }
+
+    public Set<Term.Type> supportedCQL3Constants()
+    {
+        return supportedCQL3Constants;
+    }
+
+    public CQL3Type asCQL3Type()
+    {
+        return CQL3Type.Native.VARINT;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java b/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java
index c1f2049..3805ac3 100644
--- a/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java
+++ b/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java
@@ -18,9 +18,12 @@
 package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
+import java.util.EnumSet;
+import java.util.Set;
 import java.util.UUID;
 
 import org.apache.cassandra.cql.jdbc.JdbcLexicalUUID;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.UUIDGen;
 
@@ -28,6 +31,8 @@ public class LexicalUUIDType extends AbstractType<UUID>
 {
     public static final LexicalUUIDType instance = new LexicalUUIDType();
 
+    private final Set<Term.Type> supportedCQL3Constants = EnumSet.of(Term.Type.UUID);
+
     LexicalUUIDType() {} // singleton
 
     public UUID compose(ByteBuffer bytes)
@@ -88,4 +93,9 @@ public class LexicalUUIDType extends AbstractType<UUID>
             throw new MarshalException(String.format("LexicalUUID should be 16 or 0 bytes (%d)", bytes.remaining()));
         // not sure what the version should be for this.
     }
+
+    public Set<Term.Type> supportedCQL3Constants()
+    {
+        return supportedCQL3Constants;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/db/marshal/LongType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/LongType.java b/src/java/org/apache/cassandra/db/marshal/LongType.java
index 2f251ca..4bf95c6 100644
--- a/src/java/org/apache/cassandra/db/marshal/LongType.java
+++ b/src/java/org/apache/cassandra/db/marshal/LongType.java
@@ -18,14 +18,20 @@
 package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
+import java.util.EnumSet;
+import java.util.Set;
 
 import org.apache.cassandra.cql.jdbc.JdbcLong;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class LongType extends AbstractType<Long>
 {
     public static final LongType instance = new LongType();
 
+    private final Set<Term.Type> supportedCQL3Constants = EnumSet.of(Term.Type.INTEGER);
+
     LongType() {} // singleton
 
     public Long compose(ByteBuffer bytes)
@@ -94,4 +100,14 @@ public class LongType extends AbstractType<Long>
         if (bytes.remaining() != 8 && bytes.remaining() != 0)
             throw new MarshalException(String.format("Expected 8 or 0 byte long (%d)", bytes.remaining()));
     }
+
+    public Set<Term.Type> supportedCQL3Constants()
+    {
+        return supportedCQL3Constants;
+    }
+
+    public CQL3Type asCQL3Type()
+    {
+        return CQL3Type.Native.BIGINT;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a67f7792/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java b/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
index c114775..e37e04e 100644
--- a/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
@@ -18,11 +18,15 @@
 package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
+import java.util.EnumSet;
+import java.util.Set;
 import java.util.UUID;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.cassandra.cql.jdbc.JdbcTimeUUID;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.UUIDGen;
 
@@ -33,6 +37,8 @@ public class TimeUUIDType extends AbstractType<UUID>
     static final Pattern regexPattern = Pattern.compile("[A-Fa-f0-9]{8}\\-[A-Fa-f0-9]{4}\\-[A-Fa-f0-9]{4}\\-[A-Fa-f0-9]{4}\\-[A-Fa-f0-9]{12}");
     static final Pattern functionPattern = Pattern.compile("(\\w+)\\((.*)\\)");
 
+    private final Set<Term.Type> supportedCQL3Constants = EnumSet.of(Term.Type.UUID);
+
     TimeUUIDType() {} // singleton
 
     public UUID compose(ByteBuffer bytes)
@@ -209,4 +215,14 @@ public class TimeUUIDType extends AbstractType<UUID>
                 throw new MarshalException("Invalid version for TimeUUID type.");
         }
     }
+
+    public Set<Term.Type> supportedCQL3Constants()
+    {
+        return supportedCQL3Constants;
+    }
+
+    public CQL3Type asCQL3Type()
+    {
+        return CQL3Type.Native.TIMEUUID;
+    }
 }