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 2012/01/25 13:35:05 UTC

[2/5] git commit: CQL 3.0 beta

CQL 3.0 beta

patch by slebresne; reviewed by jbellis and urandom for CASSANDRA-3761


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

Branch: refs/heads/trunk
Commit: 655ccc3abfc0f3fbe974d3f52cd38bf0d6afe2a9
Parents: 86637d4
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Mon Jan 16 18:50:53 2012 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Jan 25 13:34:18 2012 +0100

----------------------------------------------------------------------
 CHANGES.txt                                        |    1 +
 build.xml                                          |   15 +-
 interface/cassandra.thrift                         |    6 +-
 src/avro/internode.genavro                         |    2 +
 .../org/apache/cassandra/config/CFMetaData.java    |   91 ++-
 .../org/apache/cassandra/cql/QueryProcessor.java   |    3 +-
 src/java/org/apache/cassandra/cql3/Attributes.java |   32 +
 .../org/apache/cassandra/cql3/CFDefinition.java    |  303 +++++
 src/java/org/apache/cassandra/cql3/CFName.java     |   58 +
 src/java/org/apache/cassandra/cql3/CFPropDefs.java |  260 ++++
 .../org/apache/cassandra/cql3/CQLStatement.java    |   70 +
 .../apache/cassandra/cql3/ColumnIdentifier.java    |   79 ++
 .../apache/cassandra/cql3/ColumnNameBuilder.java   |   73 ++
 src/java/org/apache/cassandra/cql3/Cql.g           |  633 ++++++++++
 src/java/org/apache/cassandra/cql3/Operation.java  |   54 +
 .../org/apache/cassandra/cql3/QueryProcessor.java  |  205 +++
 src/java/org/apache/cassandra/cql3/Relation.java   |  117 ++
 src/java/org/apache/cassandra/cql3/Term.java       |  205 +++
 .../cql3/statements/AlterTableStatement.java       |  180 +++
 .../cassandra/cql3/statements/BatchStatement.java  |  166 +++
 .../cassandra/cql3/statements/CFStatement.java     |   56 +
 .../statements/CreateColumnFamilyStatement.java    |  294 +++++
 .../cql3/statements/CreateIndexStatement.java      |  108 ++
 .../cql3/statements/CreateKeyspaceStatement.java   |  114 ++
 .../cassandra/cql3/statements/DeleteStatement.java |  164 +++
 .../cql3/statements/DropColumnFamilyStatement.java |   39 +
 .../cql3/statements/DropIndexStatement.java        |   74 ++
 .../cql3/statements/DropKeyspaceStatement.java     |   52 +
 .../cql3/statements/ModificationStatement.java     |  129 ++
 .../cassandra/cql3/statements/Preprocessable.java  |   27 +
 .../cql3/statements/SchemaAlteringStatement.java   |  184 +++
 .../cassandra/cql3/statements/SelectStatement.java |  972 +++++++++++++++
 .../cql3/statements/TruncateStatement.java         |   68 +
 .../cassandra/cql3/statements/UpdateStatement.java |  359 ++++++
 .../cassandra/cql3/statements/UseStatement.java    |   48 +
 .../org/apache/cassandra/db/CounterMutation.java   |    9 +
 src/java/org/apache/cassandra/db/IMutation.java    |    1 +
 src/java/org/apache/cassandra/db/RowMutation.java  |   19 +
 src/java/org/apache/cassandra/db/SystemTable.java  |    2 +-
 .../db/marshal/AbstractCompositeType.java          |   17 +
 .../apache/cassandra/db/marshal/CompositeType.java |  128 ++-
 .../org/apache/cassandra/service/ClientState.java  |   57 +-
 .../apache/cassandra/service/StorageService.java   |    1 +
 .../apache/cassandra/thrift/CassandraServer.java   |   42 +-
 .../apache/cassandra/thrift/ThriftValidation.java  |    8 +-
 .../apache/cassandra/utils/SemanticVersion.java    |  231 ++++
 .../cassandra/utils/SemanticVersionTest.java       |  105 ++
 47 files changed, 5829 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/655ccc3a/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 66ab562..03d2e69 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -55,6 +55,7 @@
  * Add SnapshotCommand to trigger snapshot on remote node (CASSANDRA-3721)
  * Make CFMetaData conversions to/from thrift/native schema inverses
    (CASSANDRA_3559)
+ * Add initial code for CQL 3.0-beta (CASSANDRA-3781)
 
 
 1.0.8

http://git-wip-us.apache.org/repos/asf/cassandra/blob/655ccc3a/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index d382e34..2a46f37 100644
--- a/build.xml
+++ b/build.xml
@@ -202,6 +202,9 @@
       <uptodate property="cqlcurrent" 
                 srcfile="${build.src.java}/org/apache/cassandra/cql/Cql.g" 
                 targetfile="${build.src.gen-java}/org/apache/cassandra/cql/Cql.tokens"/>
+      <uptodate property="cqlcurrent"
+                srcfile="${build.src.java}/org/apache/cassandra/cql3/Cql.g"
+                targetfile="${build.src.gen-java}/org/apache/cassandra/cql3/Cql.tokens"/>
     </target>
  
     <target name="gen-cql-grammar" depends="check-gen-cql-grammar" unless="cqlcurrent">
@@ -213,9 +216,17 @@
          <arg value="${build.src.java}/org/apache/cassandra/cql/Cql.g" />
          <arg value="-fo" />
          <arg value="${build.src.gen-java}/org/apache/cassandra/cql/" />
-      </java> 
+      </java>
+      <java classname="org.antlr.Tool"
+            classpath="${build.lib}/antlr-3.2.jar"
+            fork="true"
+            failonerror="true">
+         <arg value="${build.src.java}/org/apache/cassandra/cql3/Cql.g" />
+         <arg value="-fo" />
+         <arg value="${build.src.gen-java}/org/apache/cassandra/cql3/" />
+      </java>
     </target>
-	
+
 	<target name="generate-cql-html" depends="maven-ant-tasks-init" description="Generate HTML from textile source">
 	  <artifact:dependencies pathId="wikitext.classpath">
 	    <dependency groupId="com.datastax.wikitext" artifactId="wikitext-core-ant" version="1.3"/>	    	  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/655ccc3a/interface/cassandra.thrift
----------------------------------------------------------------------
diff --git a/interface/cassandra.thrift b/interface/cassandra.thrift
index e28b236..08b776e 100644
--- a/interface/cassandra.thrift
+++ b/interface/cassandra.thrift
@@ -46,7 +46,7 @@ namespace rb CassandraThrift
 #           for every edit that doesn't result in a change to major/minor.
 #
 # See the Semantic Versioning Specification (SemVer) http://semver.org.
-const string VERSION = "19.25.0"
+const string VERSION = "19.26.0"
 
 
 #
@@ -415,6 +415,8 @@ struct CfDef {
     32: optional map<string,string> compression_options,
     33: optional double bloom_filter_fp_chance,
     34: optional string caching="keys_only",
+    35: optional list<binary> column_aliases,
+    36: optional binary value_alias,
 }
 
 /* describes a keyspace. */
@@ -714,6 +716,6 @@ service Cassandra {
             2:UnavailableException ue,
             3:TimedOutException te,
             4:SchemaDisagreementException sde)
-           
 
+  void set_cql_version(1: required string version) throws (1:InvalidRequestException ire)
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/655ccc3a/src/avro/internode.genavro
----------------------------------------------------------------------
diff --git a/src/avro/internode.genavro b/src/avro/internode.genavro
index c36a83f..36c2cba 100644
--- a/src/avro/internode.genavro
+++ b/src/avro/internode.genavro
@@ -67,6 +67,8 @@ protocol InterNode {
         union { null, map<string> } compression_options = null;
         union { null, double } bloom_filter_fp_chance = null;
         union { null, string } caching = null;
+        union { null, array<bytes> } column_aliases = null;
+        union { null, bytes } value_alias = null;
     }
 
     @aliases(["org.apache.cassandra.config.avro.KsDef"])

http://git-wip-us.apache.org/repos/asf/cassandra/blob/655ccc3a/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java
index 9bd731a..a82639d 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -32,6 +32,7 @@ import org.apache.commons.lang.builder.EqualsBuilder;
 import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.commons.lang.builder.ToStringBuilder;
 
+import org.apache.cassandra.cql3.CFDefinition;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.QueryPath;
 import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
@@ -173,6 +174,8 @@ public final class CFMetaData
     // thrift compatibility
     private double mergeShardsChance;                 // default 0.1, chance [0.0, 1.0] of merging old shards during replication
     private ByteBuffer keyAlias;                      // default NULL
+    private List<ByteBuffer> columnAliases = new ArrayList<ByteBuffer>();
+    private ByteBuffer valueAlias;                    // default NULL
     private Double bloomFilterFpChance;               // default NULL
     private Caching caching;                          // default KEYS_ONLY (possible: all, key_only, row_only, none)
 
@@ -182,17 +185,24 @@ public final class CFMetaData
 
     private CompressionParameters compressionParameters;
 
+    // Processed infos used by CQL. This can be fully reconstructed from the CFMedata,
+    // so it's not saved on disk. It is however costlyish to recreate for each query
+    // so we cache it here (and update on each relevant CFMetadata change)
+    private CFDefinition cqlCfDef;
+
     public CFMetaData comment(String prop) { comment = enforceCommentNotNull(prop); return this;}
     public CFMetaData readRepairChance(double prop) {readRepairChance = prop; return this;}
     public CFMetaData replicateOnWrite(boolean prop) {replicateOnWrite = prop; return this;}
     public CFMetaData gcGraceSeconds(int prop) {gcGraceSeconds = prop; return this;}
-    public CFMetaData defaultValidator(AbstractType<?> prop) {defaultValidator = prop; return this;}
-    public CFMetaData keyValidator(AbstractType<?> prop) {keyValidator = prop; return this;}
+    public CFMetaData defaultValidator(AbstractType<?> prop) {defaultValidator = prop; updateCfDef(); return this;}
+    public CFMetaData keyValidator(AbstractType<?> prop) {keyValidator = prop; updateCfDef(); return this;}
     public CFMetaData minCompactionThreshold(int prop) {minCompactionThreshold = prop; return this;}
     public CFMetaData maxCompactionThreshold(int prop) {maxCompactionThreshold = prop; return this;}
     public CFMetaData mergeShardsChance(double prop) {mergeShardsChance = prop; return this;}
-    public CFMetaData keyAlias(ByteBuffer prop) {keyAlias = prop; return this;}
-    public CFMetaData columnMetadata(Map<ByteBuffer,ColumnDefinition> prop) {column_metadata = prop; return this;}
+    public CFMetaData keyAlias(ByteBuffer prop) {keyAlias = prop; updateCfDef(); return this;}
+    public CFMetaData columnAliases(List<ByteBuffer> prop) {columnAliases = prop; updateCfDef(); return this;}
+    public CFMetaData valueAlias(ByteBuffer prop) {valueAlias = prop; updateCfDef(); return this;}
+    public CFMetaData columnMetadata(Map<ByteBuffer,ColumnDefinition> prop) {column_metadata = prop; updateCfDef(); return this;}
     public CFMetaData compactionStrategyClass(Class<? extends AbstractCompactionStrategy> prop) {compactionStrategyClass = prop; return this;}
     public CFMetaData compactionStrategyOptions(Map<String, String> prop) {compactionStrategyOptions = prop; return this;}
     public CFMetaData compressionParameters(CompressionParameters prop) {compressionParameters = prop; return this;}
@@ -246,6 +256,7 @@ public final class CFMetaData
         keyValidator = BytesType.instance;
         comment = "";
         keyAlias = null; // This qualifies as a 'strange default'.
+        valueAlias = null;
         column_metadata = new HashMap<ByteBuffer,ColumnDefinition>();
 
         try
@@ -259,6 +270,7 @@ public final class CFMetaData
         compactionStrategyOptions = new HashMap<String, String>();
 
         compressionParameters = new CompressionParameters(null);
+        updateCfDef(); // init cqlCfDef
     }
 
     private static CFMetaData newSystemMetadata(String cfName, int cfId, String comment, AbstractType<?> comparator, AbstractType<?> subcc)
@@ -369,6 +381,8 @@ public final class CFMetaData
         if (cf.max_compaction_threshold != null) { newCFMD.maxCompactionThreshold(cf.max_compaction_threshold); }
         if (cf.merge_shards_chance != null) { newCFMD.mergeShardsChance(cf.merge_shards_chance); }
         if (cf.key_alias != null) { newCFMD.keyAlias(cf.key_alias); }
+        if (cf.column_aliases != null) { newCFMD.columnAliases(fixAvroRetardation(cf.column_aliases)); }
+        if (cf.value_alias != null) { newCFMD.valueAlias(cf.value_alias); }
         if (cf.compaction_strategy != null)
         {
             try
@@ -418,6 +432,18 @@ public final class CFMetaData
                       .bloomFilterFpChance(cf.bloom_filter_fp_chance)
                       .caching(caching);
     }
+
+    /*
+     * Avro handles array with it's own class, GenericArray, that extends
+     * AbstractList but redefine equals() in a way that violate List.equals()
+     * specification (basically only a GenericArray can ever be equal to a
+     * GenericArray).
+     * (Concretely, keeping the list returned by avro breaks DefsTest.saveAndRestore())
+     */
+    private static <T> List<T> fixAvroRetardation(List<T> array)
+    {
+        return new ArrayList<T>(array);
+    }
     
     public String getComment()
     {
@@ -469,6 +495,21 @@ public final class CFMetaData
         return keyAlias == null ? DEFAULT_KEY_NAME : keyAlias;
     }
 
+    public ByteBuffer getKeyAlias()
+    {
+        return keyAlias;
+    }
+
+    public List<ByteBuffer> getColumnAliases()
+    {
+        return columnAliases;
+    }
+
+    public ByteBuffer getValueAlias()
+    {
+        return valueAlias;
+    }
+
     public CompressionParameters compressionParameters()
     {
         return compressionParameters;
@@ -524,6 +565,8 @@ public final class CFMetaData
             .append(column_metadata, rhs.column_metadata)
             .append(mergeShardsChance, rhs.mergeShardsChance)
             .append(keyAlias, rhs.keyAlias)
+            .append(columnAliases, rhs.columnAliases)
+            .append(valueAlias, rhs.valueAlias)
             .append(compactionStrategyClass, rhs.compactionStrategyClass)
             .append(compactionStrategyOptions, rhs.compactionStrategyOptions)
             .append(compressionParameters, rhs.compressionParameters)
@@ -552,6 +595,8 @@ public final class CFMetaData
             .append(column_metadata)
             .append(mergeShardsChance)
             .append(keyAlias)
+            .append(columnAliases)
+            .append(valueAlias)
             .append(compactionStrategyClass)
             .append(compactionStrategyOptions)
             .append(compressionParameters)
@@ -613,6 +658,8 @@ public final class CFMetaData
         if (cf_def.isSetMax_compaction_threshold()) { newCFMD.maxCompactionThreshold(cf_def.max_compaction_threshold); }
         if (cf_def.isSetMerge_shards_chance()) { newCFMD.mergeShardsChance(cf_def.merge_shards_chance); }
         if (cf_def.isSetKey_alias()) { newCFMD.keyAlias(cf_def.key_alias); }
+        if (cf_def.isSetColumn_aliases() && cf_def.column_aliases != null) { newCFMD.columnAliases(cf_def.column_aliases); }
+        if (cf_def.isSetValue_alias()) { newCFMD.valueAlias(cf_def.value_alias); }
         if (cf_def.isSetKey_validation_class()) { newCFMD.keyValidator(TypeParser.parse(cf_def.key_validation_class)); }
         if (cf_def.isSetCompaction_strategy())
             newCFMD.compactionStrategyClass = createCompactionStrategy(cf_def.compaction_strategy);
@@ -698,6 +745,8 @@ public final class CFMetaData
         maxCompactionThreshold = cf_def.max_compaction_threshold;
         mergeShardsChance = cf_def.merge_shards_chance;
         keyAlias = cf_def.key_alias;
+        columnAliases = cf_def.column_aliases;
+        valueAlias = cf_def.value_alias;
         if (cf_def.isSetBloom_filter_fp_chance())
             bloomFilterFpChance = cf_def.bloom_filter_fp_chance;
         caching = Caching.fromString(cf_def.caching);
@@ -1073,14 +1122,29 @@ public final class CFMetaData
         if (!cfDef.isSetColumn_metadata())
             return;
 
-        AbstractType comparator = TypeParser.parse(cfDef.column_type.equals("Super")
-                                           ? cfDef.subcomparator_type
-                                           : cfDef.comparator_type);
+        AbstractType comparator = getColumnDefinitionComparator(cfDef);
 
         for (ColumnDef columnDef : cfDef.column_metadata)
             ColumnDefinition.addToSchema(mutation, cfDef.name, comparator, columnDef, timestamp);
     }
 
+    public static AbstractType<?> getColumnDefinitionComparator(CfDef cfDef) throws ConfigurationException
+    {
+        AbstractType<?> cfComparator = TypeParser.parse(cfDef.column_type.equals("Super")
+                                     ? cfDef.subcomparator_type
+                                     : cfDef.comparator_type);
+
+        if (cfComparator instanceof CompositeType)
+        {
+            List<AbstractType<?>> types = ((CompositeType)cfComparator).types;
+            return types.get(types.size() - 1);
+        }
+        else
+        {
+            return cfComparator;
+        }
+    }
+
     /**
      * Deserialize CF metadata from low-level representation
      *
@@ -1130,6 +1194,17 @@ public final class CFMetaData
         return cfDef;
     }
 
+    private void updateCfDef()
+    {
+        cqlCfDef = new CFDefinition(this);
+    }
+
+    public CFDefinition getCfDef()
+    {
+        assert cqlCfDef != null;
+        return cqlCfDef;
+    }
+
     @Override
     public String toString()
     {
@@ -1150,6 +1225,8 @@ public final class CFMetaData
             .append("maxCompactionThreshold", maxCompactionThreshold)
             .append("mergeShardsChance", mergeShardsChance)
             .append("keyAlias", keyAlias)
+            .append("columnAliases", columnAliases)
+            .append("valueAlias", keyAlias)
             .append("column_metadata", column_metadata)
             .append("compactionStrategyClass", compactionStrategyClass)
             .append("compactionStrategyOptions", compactionStrategyOptions)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/655ccc3a/src/java/org/apache/cassandra/cql/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/QueryProcessor.java b/src/java/org/apache/cassandra/cql/QueryProcessor.java
index 917fb92..c539979 100644
--- a/src/java/org/apache/cassandra/cql/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql/QueryProcessor.java
@@ -53,6 +53,7 @@ import org.apache.cassandra.thrift.Column;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.SemanticVersion;
 
 import com.google.common.base.Predicates;
 import com.google.common.collect.Maps;
@@ -65,7 +66,7 @@ import static org.apache.cassandra.thrift.ThriftValidation.validateColumnFamily;
 
 public class QueryProcessor
 {
-    public static final String CQL_VERSION = "2.0.0";
+    public static final SemanticVersion CQL_VERSION = new SemanticVersion("2.0.0");
 
     private static final Logger logger = LoggerFactory.getLogger(QueryProcessor.class);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/655ccc3a/src/java/org/apache/cassandra/cql3/Attributes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Attributes.java b/src/java/org/apache/cassandra/cql3/Attributes.java
new file mode 100644
index 0000000..41f78fb
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/Attributes.java
@@ -0,0 +1,32 @@
+/*
+ * 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.thrift.ConsistencyLevel;
+
+/**
+ * Utility class for the Parser to gather attributes for modification
+ * statements.
+ */
+public class Attributes
+{
+    public ConsistencyLevel cLevel;
+    public Long timestamp;
+    public int timeToLive;
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/655ccc3a/src/java/org/apache/cassandra/cql3/CFDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/CFDefinition.java b/src/java/org/apache/cassandra/cql3/CFDefinition.java
new file mode 100644
index 0000000..b3725db
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/CFDefinition.java
@@ -0,0 +1,303 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.cassandra.cql3;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.collect.AbstractIterator;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.thrift.InvalidRequestException;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+/**
+ * Holds metadata on a CF preprocessed for use by CQL queries.
+ */
+public class CFDefinition implements Iterable<CFDefinition.Name>
+{
+    public static final AbstractType<?> definitionType = UTF8Type.instance;
+
+    private static final String DEFAULT_KEY_ALIAS = "key";
+    private static final String DEFAULT_COLUMN_ALIAS = "column";
+    private static final String DEFAULT_VALUE_ALIAS = "value";
+
+    public final CFMetaData cfm;
+    public final Name key;
+    // LinkedHashMap because the order does matter (it is the order in the composite type)
+    public final LinkedHashMap<ColumnIdentifier, Name> columns = new LinkedHashMap<ColumnIdentifier, Name>();
+    public final Name value;
+    // Keep metadata lexicographically ordered so that wildcard expansion have a deterministic order
+    public final Map<ColumnIdentifier, Name> metadata = new TreeMap<ColumnIdentifier, Name>();
+
+    public final boolean isComposite;
+    public final boolean isCompact;
+
+    public CFDefinition(CFMetaData cfm)
+    {
+        this.cfm = cfm;
+        this.key = new Name(getKeyId(cfm), Name.Kind.KEY_ALIAS, cfm.getKeyValidator());
+        if (cfm.comparator instanceof CompositeType)
+        {
+            this.isComposite = true;
+            CompositeType composite = (CompositeType)cfm.comparator;
+            if (!cfm.getColumn_metadata().isEmpty())
+            {
+                // "sparse" composite
+                this.isCompact = false;
+                this.value = null;
+                assert cfm.getValueAlias() == null;
+                for (int i = 0; i < composite.types.size() - 1; i++)
+                {
+                    ColumnIdentifier id = getColumnId(cfm, i);
+                    this.columns.put(id, new Name(id, Name.Kind.COLUMN_ALIAS, i, composite.types.get(i)));
+                }
+
+                for (Map.Entry<ByteBuffer, ColumnDefinition> def : cfm.getColumn_metadata().entrySet())
+                {
+                    ColumnIdentifier id = new ColumnIdentifier(def.getKey());
+                    this.metadata.put(id, new Name(id, Name.Kind.COLUMN_METADATA, def.getValue().getValidator()));
+                }
+            }
+            else
+            {
+                // "dense" composite
+                this.isCompact = true;
+                for (int i = 0; i < composite.types.size(); i++)
+                {
+                    ColumnIdentifier id = getColumnId(cfm, i);
+                    this.columns.put(id, new Name(id, Name.Kind.COLUMN_ALIAS, i, composite.types.get(i)));
+                }
+                this.value = new Name(getValueId(cfm), Name.Kind.VALUE_ALIAS, cfm.getDefaultValidator());
+            }
+        }
+        else
+        {
+            this.isComposite = false;
+            if (!cfm.getColumn_metadata().isEmpty())
+            {
+                // static CF
+                this.isCompact = false;
+                this.value = null;
+                assert cfm.getValueAlias() == null;
+                assert cfm.getColumnAliases() == null || cfm.getColumnAliases().isEmpty();
+                for (Map.Entry<ByteBuffer, ColumnDefinition> def : cfm.getColumn_metadata().entrySet())
+                {
+                    ColumnIdentifier id = new ColumnIdentifier(def.getKey());
+                    this.metadata.put(id, new Name(id, Name.Kind.COLUMN_METADATA, def.getValue().getValidator()));
+                }
+            }
+            else
+            {
+                // dynamic CF
+                this.isCompact = true;
+                ColumnIdentifier id = getColumnId(cfm, 0);
+                Name name = new Name(id, Name.Kind.COLUMN_ALIAS, 0, cfm.comparator);
+                this.columns.put(id, name);
+                this.value = new Name(getValueId(cfm), Name.Kind.VALUE_ALIAS, cfm.getDefaultValidator());
+            }
+        }
+        assert value == null || metadata.isEmpty();
+    }
+
+    private static ColumnIdentifier getKeyId(CFMetaData cfm)
+    {
+        return cfm.getKeyAlias() == null
+             ? new ColumnIdentifier(DEFAULT_KEY_ALIAS, false)
+             : new ColumnIdentifier(cfm.getKeyAlias());
+    }
+
+    private static ColumnIdentifier getColumnId(CFMetaData cfm, int i)
+    {
+        List<ByteBuffer> definedNames = cfm.getColumnAliases();
+        return definedNames == null || i >= definedNames.size()
+             ? new ColumnIdentifier(DEFAULT_COLUMN_ALIAS + (i + 1), false)
+             : new ColumnIdentifier(cfm.getColumnAliases().get(i));
+    }
+
+    private static ColumnIdentifier getValueId(CFMetaData cfm)
+    {
+        return cfm.getValueAlias() == null
+             ? new ColumnIdentifier(DEFAULT_VALUE_ALIAS, false)
+             : new ColumnIdentifier(cfm.getValueAlias());
+    }
+
+    public Name get(ColumnIdentifier name)
+    {
+        if (name.equals(key.name))
+            return key;
+        if (value != null && name.equals(value.name))
+            return value;
+        CFDefinition.Name def = columns.get(name);
+        if (def != null)
+            return def;
+        return metadata.get(name);
+    }
+
+    public Iterator<Name> iterator()
+    {
+        return new AbstractIterator<Name>()
+        {
+            private boolean keyDone;
+            private final Iterator<Name> columnIter = columns.values().iterator();
+            private boolean valueDone;
+            private final Iterator<Name> metadataIter = metadata.values().iterator();
+
+            protected Name computeNext()
+            {
+                if (!keyDone)
+                {
+                    keyDone = true;
+                    return key;
+                }
+
+                if (columnIter.hasNext())
+                    return columnIter.next();
+
+                if (value != null && !valueDone)
+                {
+                    valueDone = true;
+                    return value;
+                }
+
+                if (metadataIter.hasNext())
+                    return metadataIter.next();
+
+                return endOfData();
+            }
+        };
+    }
+
+    public ColumnNameBuilder getColumnNameBuilder()
+    {
+        return isComposite
+             ? new CompositeType.Builder((CompositeType)cfm.comparator)
+             : new NonCompositeBuilder(cfm.comparator);
+    }
+
+    public static class Name
+    {
+        public static enum Kind
+        {
+            KEY_ALIAS, COLUMN_ALIAS, VALUE_ALIAS, COLUMN_METADATA
+        }
+
+        private Name(ColumnIdentifier name, Kind kind, AbstractType<?> type)
+        {
+            this(name, kind, -1, type);
+        }
+
+        private Name(ColumnIdentifier name, Kind kind, int position, AbstractType<?> type)
+        {
+            this.kind = kind;
+            this.name = name;
+            this.compositePosition = position;
+            this.type = type;
+        }
+
+        public final Kind kind;
+        public final ColumnIdentifier name;
+        public final int compositePosition; // only make sense for COLUMN_ALIAS if CFDefinition.isComposite()
+        public final AbstractType<?> type;
+
+        @Override
+        public String toString()
+        {
+            // It is not fully conventional, but it is convenient for error messages to the user
+            return name.toString();
+        }
+    }
+
+    @Override
+    public String toString()
+    {
+        StringBuilder sb = new StringBuilder();
+        sb.append(key.name);
+        for (Name name : columns.values())
+            sb.append(", ").append(name.name);
+        sb.append(" => ");
+        if (value != null)
+            sb.append(value.name);
+        if (!metadata.isEmpty())
+        {
+            sb.append("{");
+            for (Name name : metadata.values())
+                sb.append(" ").append(name.name);
+            sb.append(" }");
+        }
+        sb.append("]");
+        return sb.toString();
+    }
+
+    private static class NonCompositeBuilder implements ColumnNameBuilder
+    {
+        private final AbstractType<?> type;
+        private ByteBuffer columnName;
+
+        private NonCompositeBuilder(AbstractType<?> type)
+        {
+            this.type = type;
+        }
+
+        public NonCompositeBuilder add(ByteBuffer bb)
+        {
+            if (columnName != null)
+                throw new IllegalStateException("Column name is already constructed");
+
+            columnName = bb;
+            return this;
+        }
+
+        public NonCompositeBuilder add(Term t, Relation.Type op, List<ByteBuffer> variables) throws InvalidRequestException
+        {
+            if (columnName != null)
+                throw new IllegalStateException("Column name is already constructed");
+
+            // We don't support the relation type yet, i.e., there is no distinction between x > 3 and x >= 3.
+            columnName = t.getByteBuffer(type, variables);
+            return this;
+        }
+
+        public int componentCount()
+        {
+            return columnName == null ? 0 : 1;
+        }
+
+        public ByteBuffer build()
+        {
+            return columnName == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : columnName;
+        }
+
+        public ByteBuffer buildAsEndOfRange()
+        {
+            throw new IllegalStateException();
+        }
+
+        public NonCompositeBuilder copy()
+        {
+            NonCompositeBuilder newBuilder = new NonCompositeBuilder(type);
+            newBuilder.columnName = columnName;
+            return newBuilder;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/655ccc3a/src/java/org/apache/cassandra/cql3/CFName.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/CFName.java b/src/java/org/apache/cassandra/cql3/CFName.java
new file mode 100644
index 0000000..cd6e6d3
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/CFName.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.cassandra.cql3;
+
+import java.util.Locale;
+
+public class CFName
+{
+    private String ksName;
+    private String cfName;
+
+    public void setKeyspace(String ks, boolean keepCase)
+    {
+        ksName = keepCase ? ks : ks.toLowerCase(Locale.US);
+    }
+
+    public void setColumnFamily(String cf, boolean keepCase)
+    {
+        cfName = keepCase ? cf : cf.toLowerCase(Locale.US);
+    }
+
+    public boolean hasKeyspace()
+    {
+        return ksName != null;
+    }
+
+    public String getKeyspace()
+    {
+        return ksName;
+    }
+
+    public String getColumnFamily()
+    {
+        return cfName;
+    }
+
+    @Override
+    public String toString()
+    {
+        return (hasKeyspace() ? (ksName + ".") : "") + cfName;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/655ccc3a/src/java/org/apache/cassandra/cql3/CFPropDefs.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/CFPropDefs.java b/src/java/org/apache/cassandra/cql3/CFPropDefs.java
new file mode 100644
index 0000000..e878e63
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/CFPropDefs.java
@@ -0,0 +1,260 @@
+/*
+ * 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 com.google.common.collect.Sets;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.TypeParser;
+import org.apache.cassandra.thrift.InvalidRequestException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class CFPropDefs
+{
+    private static Logger logger = LoggerFactory.getLogger(CFPropDefs.class);
+
+    public static final String KW_COMMENT = "comment";
+    public static final String KW_READREPAIRCHANCE = "read_repair_chance";
+    public static final String KW_GCGRACESECONDS = "gc_grace_seconds";
+    public static final String KW_MINCOMPACTIONTHRESHOLD = "min_compaction_threshold";
+    public static final String KW_MAXCOMPACTIONTHRESHOLD = "max_compaction_threshold";
+    public static final String KW_REPLICATEONWRITE = "replicate_on_write";
+
+    public static final String KW_COMPACTION_STRATEGY_CLASS = "compaction_strategy_class";
+
+    // Maps CQL short names to the respective Cassandra comparator/validator class names
+    public static final Map<String, String> comparators = new HashMap<String, String>();
+    public static final Set<String> keywords = new HashSet<String>();
+    public static final Set<String> obsoleteKeywords = new HashSet<String>();
+    public static final Set<String> allowedKeywords = new HashSet<String>();
+
+    public static final String COMPACTION_OPTIONS_PREFIX = "compaction_strategy_options";
+    public static final String COMPRESSION_PARAMETERS_PREFIX = "compression_parameters";
+
+    static
+    {
+        comparators.put("ascii", "AsciiType");
+        comparators.put("bigint", "LongType");
+        comparators.put("blob", "BytesType");
+        comparators.put("boolean", "BooleanType");
+        comparators.put("counter", "CounterColumnType");
+        comparators.put("decimal", "DecimalType");
+        comparators.put("double", "DoubleType");
+        comparators.put("float", "FloatType");
+        comparators.put("int", "Int32Type");
+        comparators.put("text", "UTF8Type");
+        comparators.put("timestamp", "DateType");
+        comparators.put("uuid", "UUIDType");
+        comparators.put("varchar", "UTF8Type");
+        comparators.put("varint", "IntegerType");
+
+        keywords.add(KW_COMMENT);
+        keywords.add(KW_READREPAIRCHANCE);
+        keywords.add(KW_GCGRACESECONDS);
+        keywords.add(KW_MINCOMPACTIONTHRESHOLD);
+        keywords.add(KW_MAXCOMPACTIONTHRESHOLD);
+        keywords.add(KW_REPLICATEONWRITE);
+        keywords.add(KW_COMPACTION_STRATEGY_CLASS);
+
+        obsoleteKeywords.add("row_cache_size");
+        obsoleteKeywords.add("key_cache_size");
+        obsoleteKeywords.add("row_cache_save_period_in_seconds");
+        obsoleteKeywords.add("key_cache_save_period_in_seconds");
+        obsoleteKeywords.add("memtable_throughput_in_mb");
+        obsoleteKeywords.add("memtable_operations_in_millions");
+        obsoleteKeywords.add("memtable_flush_after_mins");
+        obsoleteKeywords.add("row_cache_provider");
+        obsoleteKeywords.add("comparator");
+        obsoleteKeywords.add("default_validation");
+
+        allowedKeywords.addAll(keywords);
+        allowedKeywords.addAll(obsoleteKeywords);
+    }
+
+    public final Map<String, String> properties = new HashMap<String, String>();
+    public final Map<String, String> compactionStrategyOptions = new HashMap<String, String>();
+    public final Map<String, String> compressionParameters = new HashMap<String, String>();
+
+    public static AbstractType<?> parseType(String type) throws InvalidRequestException
+    {
+        try
+        {
+            String className = comparators.get(type);
+            if (className == null)
+                className = type;
+            return TypeParser.parse(className);
+        }
+        catch (ConfigurationException e)
+        {
+            InvalidRequestException ex = new InvalidRequestException(e.toString());
+            ex.initCause(e);
+            throw ex;
+        }
+    }
+
+    /* If not comparator/validator is not specified, default to text (BytesType is the wrong default for CQL
+     * since it uses hex terms).  If the value specified is not found in the comparators map, assume the user
+     * knows what they are doing (a custom comparator/validator for example), and pass it on as-is.
+     */
+
+    public void validate() throws InvalidRequestException
+    {
+        // Catch the case where someone passed a kwarg that is not recognized.
+        for (String bogus : Sets.difference(properties.keySet(), allowedKeywords))
+            throw new InvalidRequestException(bogus + " is not a valid keyword argument for CREATE COLUMNFAMILY");
+        for (String obsolete : Sets.intersection(properties.keySet(), obsoleteKeywords))
+            logger.warn("Ignoring obsolete property {}", obsolete);
+
+        // Validate min/max compaction thresholds
+        Integer minCompaction = getInt(KW_MINCOMPACTIONTHRESHOLD, null);
+        Integer maxCompaction = getInt(KW_MAXCOMPACTIONTHRESHOLD, null);
+
+        if ((minCompaction != null) && (maxCompaction != null))     // Both min and max are set
+        {
+            if ((minCompaction > maxCompaction) && (maxCompaction != 0))
+                throw new InvalidRequestException(String.format("%s cannot be larger than %s",
+                        KW_MINCOMPACTIONTHRESHOLD,
+                        KW_MAXCOMPACTIONTHRESHOLD));
+        }
+        else if (minCompaction != null)     // Only the min threshold is set
+        {
+            if (minCompaction > CFMetaData.DEFAULT_MAX_COMPACTION_THRESHOLD)
+                throw new InvalidRequestException(String.format("%s cannot be larger than %s, (default %s)",
+                        KW_MINCOMPACTIONTHRESHOLD,
+                        KW_MAXCOMPACTIONTHRESHOLD,
+                        CFMetaData.DEFAULT_MAX_COMPACTION_THRESHOLD));
+        }
+        else if (maxCompaction != null)     // Only the max threshold is set
+        {
+            if ((maxCompaction < CFMetaData.DEFAULT_MIN_COMPACTION_THRESHOLD) && (maxCompaction != 0))
+                throw new InvalidRequestException(String.format("%s cannot be smaller than %s, (default %s)",
+                        KW_MAXCOMPACTIONTHRESHOLD,
+                        KW_MINCOMPACTIONTHRESHOLD,
+                        CFMetaData.DEFAULT_MIN_COMPACTION_THRESHOLD));
+        }
+    }
+
+    /** Map a keyword to the corresponding value */
+    public void addProperty(String name, String value)
+    {
+        String[] composite = name.split(":");
+        if (composite.length > 1)
+        {
+            if (composite[0].equals(COMPACTION_OPTIONS_PREFIX))
+            {
+                compactionStrategyOptions.put(composite[1], value);
+                return;
+            }
+            else if (composite[0].equals(COMPRESSION_PARAMETERS_PREFIX))
+            {
+                compressionParameters.put(composite[1], value);
+                return;
+            }
+        }
+        properties.put(name, value);
+    }
+
+    public void addAll(Map<String, String> propertyMap)
+    {
+        for (Map.Entry<String, String> entry : propertyMap.entrySet())
+            addProperty(entry.getKey(), entry.getValue());
+    }
+
+    public Boolean hasProperty(String name)
+    {
+        return properties.containsKey(name);
+    }
+
+    public String get(String name)
+    {
+        return properties.get(name);
+    }
+
+    public String getString(String key, String defaultValue)
+    {
+        String value = properties.get(key);
+        return value != null ? value : defaultValue;
+    }
+
+    // Return a property value, typed as a Boolean
+    public Boolean getBoolean(String key, Boolean defaultValue) throws InvalidRequestException
+    {
+        String value = properties.get(key);
+        return (value == null) ? defaultValue : value.toLowerCase().matches("(1|true|yes)");
+    }
+
+    // Return a property value, typed as a Double
+    public Double getDouble(String key, Double defaultValue) throws InvalidRequestException
+    {
+        Double result;
+        String value = properties.get(key);
+
+        if (value == null)
+            result = defaultValue;
+        else
+        {
+            try
+            {
+                result = Double.parseDouble(value);
+            }
+            catch (NumberFormatException e)
+            {
+                throw new InvalidRequestException(String.format("%s not valid for \"%s\"", value, key));
+            }
+        }
+        return result;
+    }
+
+    // Return a property value, typed as an Integer
+    public Integer getInt(String key, Integer defaultValue) throws InvalidRequestException
+    {
+        Integer result;
+        String value = properties.get(key);
+
+        if (value == null)
+            result = defaultValue;
+        else
+        {
+            try
+            {
+                result = Integer.parseInt(value);
+            }
+            catch (NumberFormatException e)
+            {
+                throw new InvalidRequestException(String.format("%s not valid for \"%s\"", value, key));
+            }
+        }
+        return result;
+    }
+
+    public String toString()
+    {
+        return String.format("CFPropDefs(%s, compaction: %s, compression: %s)",
+                             properties.toString(),
+                             compactionStrategyOptions.toString(),
+                             compressionParameters.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/655ccc3a/src/java/org/apache/cassandra/cql3/CQLStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/CQLStatement.java b/src/java/org/apache/cassandra/cql3/CQLStatement.java
new file mode 100644
index 0000000..90277c0
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/CQLStatement.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.cassandra.cql3;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.thrift.CqlResult;
+import org.apache.cassandra.thrift.InvalidRequestException;
+import org.apache.cassandra.thrift.SchemaDisagreementException;
+import org.apache.cassandra.thrift.TimedOutException;
+import org.apache.cassandra.thrift.UnavailableException;
+
+public abstract class CQLStatement
+{
+    private int boundTerms;
+
+    public int getBoundsTerms()
+    {
+        return boundTerms;
+    }
+
+    // Used by the parser and preparable statement
+    public void setBoundTerms(int boundTerms)
+    {
+        this.boundTerms = boundTerms;
+    }
+
+    /**
+     * Perform any access verification necessary for the statement.
+     *
+     * @param state the current client state
+     */
+    public abstract void checkAccess(ClientState state) throws InvalidRequestException;
+
+    /**
+     * Perform additional validation required by the statment.
+     * To be overriden by subclasses if needed.
+     *
+     * @param state the current client state
+     */
+    public void validate(ClientState state) throws InvalidRequestException, SchemaDisagreementException
+    {}
+
+    /**
+     * Execute the statement and return the resulting result or null if there is no result.
+     *
+     * @param state the current client state
+     * @param variables the values for bounded variables. The implementation
+     * can assume that each bound term have a corresponding value.
+     */
+    public abstract CqlResult execute(ClientState state, List<ByteBuffer> variables) throws InvalidRequestException, UnavailableException, TimedOutException, SchemaDisagreementException;
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/655ccc3a/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
new file mode 100644
index 0000000..60f3ecb
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.cassandra.cql3;
+
+import java.util.Locale;
+import java.nio.charset.CharacterCodingException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+/**
+ * Represents an identifer for a CQL column definition.
+ */
+public class ColumnIdentifier implements Comparable<ColumnIdentifier>
+{
+    public final ByteBuffer key;
+    private final String text;
+
+    public ColumnIdentifier(String rawText, boolean keepCase)
+    {
+        this.text = keepCase ? rawText : rawText.toLowerCase(Locale.US);
+        this.key = ByteBufferUtil.bytes(this.text);
+    }
+
+    public ColumnIdentifier(ByteBuffer key)
+    {
+        try
+        {
+            this.key = key;
+            this.text = ByteBufferUtil.string(key);
+        }
+        catch (CharacterCodingException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public final int hashCode()
+    {
+        return key.hashCode();
+    }
+
+    @Override
+    public final boolean equals(Object o)
+    {
+        if(!(o instanceof ColumnIdentifier))
+            return false;
+        ColumnIdentifier that = (ColumnIdentifier)o;
+        return key.equals(that.key);
+    }
+
+    @Override
+    public String toString()
+    {
+        return text;
+    }
+
+    public int compareTo(ColumnIdentifier other)
+    {
+        return key.compareTo(other.key);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/655ccc3a/src/java/org/apache/cassandra/cql3/ColumnNameBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ColumnNameBuilder.java b/src/java/org/apache/cassandra/cql3/ColumnNameBuilder.java
new file mode 100644
index 0000000..d9ca902
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/ColumnNameBuilder.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.cassandra.cql3;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.cassandra.thrift.InvalidRequestException;
+
+/**
+ * Build a potentially composite column name.
+ */
+public interface ColumnNameBuilder
+{
+    /**
+     * Add a new ByteBuffer as the next component for this name.
+     * @param bb the ByteBuffer to add
+     * @throws IllegalStateException if the builder if full, i.e. if enough component has been added.
+     * @return this builder
+     */
+    public ColumnNameBuilder add(ByteBuffer bb);
+
+    /**
+     * Add a new Term as the next component for this name.
+     * @param t the Term to add
+     * @param op the relationship this component should respect.
+     * @param variables the variables corresponding to prepared markers
+     * @throws IllegalStateException if the builder if full, i.e. if enough component has been added.
+     * @return this builder
+     */
+    public ColumnNameBuilder add(Term t, Relation.Type op, List<ByteBuffer> variables) throws InvalidRequestException;
+
+    /**
+     * Returns the number of component already added to this builder.
+     * @return the number of component in this Builder
+     */
+    public int componentCount();
+
+    /**
+     * Build the column name.
+     * @return the built column name
+     */
+    public ByteBuffer build();
+
+    /**
+     * Build the column name so that the result sorts at the end of the range
+     * represented by this (uncomplete) column name.
+     * @throws IllegalStateException if the builder is empty or full.
+     */
+    public ByteBuffer buildAsEndOfRange();
+
+    /**
+     * Clone this builder.
+     * @return the cloned builder.
+     */
+    public ColumnNameBuilder copy();
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/655ccc3a/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
new file mode 100644
index 0000000..13b2956
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -0,0 +1,633 @@
+/*
+ * 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.
+ */
+
+grammar Cql;
+
+options {
+    language = Java;
+}
+
+@header {
+    package org.apache.cassandra.cql3;
+
+    import java.util.Map;
+    import java.util.HashMap;
+    import java.util.Collections;
+    import java.util.List;
+    import java.util.ArrayList;
+
+    import org.apache.cassandra.cql3.statements.*;
+    import org.apache.cassandra.utils.Pair;
+    import org.apache.cassandra.thrift.ConsistencyLevel;
+    import org.apache.cassandra.thrift.InvalidRequestException;
+}
+
+@members {
+    private List<String> recognitionErrors = new ArrayList<String>();
+    private int currentBindMarkerIdx = -1;
+
+    public void displayRecognitionError(String[] tokenNames, RecognitionException e)
+    {
+        String hdr = getErrorHeader(e);
+        String msg = getErrorMessage(e, tokenNames);
+        recognitionErrors.add(hdr + " " + msg);
+    }
+
+    public List<String> getRecognitionErrors()
+    {
+        return recognitionErrors;
+    }
+
+    public void throwLastRecognitionError() throws InvalidRequestException
+    {
+        if (recognitionErrors.size() > 0)
+            throw new InvalidRequestException(recognitionErrors.get((recognitionErrors.size()-1)));
+    }
+
+    // used by UPDATE of the counter columns to validate if '-' was supplied by user
+    public void validateMinusSupplied(Object op, final Term value, IntStream stream) throws MissingTokenException
+    {
+        if (op == null && (value.isBindMarker() || Long.parseLong(value.getText()) > 0))
+            throw new MissingTokenException(102, stream, value);
+    }
+
+}
+
+@lexer::header {
+    package org.apache.cassandra.cql3;
+
+    import org.apache.cassandra.thrift.InvalidRequestException;
+}
+
+@lexer::members {
+    List<Token> tokens = new ArrayList<Token>();
+
+    public void emit(Token token)
+    {
+        state.token = token;
+        tokens.add(token);
+    }
+
+    public Token nextToken()
+    {
+        super.nextToken();
+        if (tokens.size() == 0)
+            return Token.EOF_TOKEN;
+        return tokens.remove(0);
+    }
+
+    private List<String> recognitionErrors = new ArrayList<String>();
+
+    public void displayRecognitionError(String[] tokenNames, RecognitionException e)
+    {
+        String hdr = getErrorHeader(e);
+        String msg = getErrorMessage(e, tokenNames);
+        recognitionErrors.add(hdr + " " + msg);
+    }
+
+    public List<String> getRecognitionErrors()
+    {
+        return recognitionErrors;
+    }
+
+    public void throwLastRecognitionError() throws InvalidRequestException
+    {
+        if (recognitionErrors.size() > 0)
+            throw new InvalidRequestException(recognitionErrors.get((recognitionErrors.size()-1)));
+    }
+}
+
+/** STATEMENTS **/
+
+query returns [CQLStatement stmnt]
+    : st=cqlStatement (';')* EOF { $stmnt = st; }
+    ;
+
+cqlStatement returns [CQLStatement stmt]
+    @after{ if (stmt != null) stmt.setBoundTerms(currentBindMarkerIdx + 1); }
+    : st1= selectStatement             { $stmt = st1; }
+    | st2= insertStatement             { $stmt = st2; }
+    | st3= updateStatement             { $stmt = st3; }
+    | st4= batchStatement              { $stmt = st4; }
+    | st5= deleteStatement             { $stmt = st5; }
+    | st6= useStatement                { $stmt = st6; }
+    | st7= truncateStatement           { $stmt = st7; }
+    | st8= createKeyspaceStatement     { $stmt = st8; }
+    | st9= createColumnFamilyStatement { $stmt = st9; }
+    | st10=createIndexStatement        { $stmt = st10; }
+    | st11=dropKeyspaceStatement       { $stmt = st11; }
+    | st12=dropColumnFamilyStatement   { $stmt = st12; }
+    | st13=dropIndexStatement          { $stmt = st13; }
+    | st14=alterTableStatement         { $stmt = st14; }
+    ;
+
+/*
+ * USE <KEYSPACE>;
+ */
+useStatement returns [UseStatement stmt]
+    : K_USE ks=keyspaceName { $stmt = new UseStatement(ks); }
+    ;
+
+/**
+ * SELECT <expression>
+ * FROM <CF>
+ * USING CONSISTENCY <LEVEL>
+ * WHERE KEY = "key1" AND COL > 1 AND COL < 100
+ * LIMIT <NUMBER>;
+ */
+selectStatement returns [SelectStatement.RawStatement expr]
+    @init {
+        boolean isCount = false;
+        ConsistencyLevel cLevel = ConsistencyLevel.ONE;
+        int limit = 10000;
+        boolean reversed = false;
+    }
+    : K_SELECT ( sclause=selectClause | (K_COUNT '(' sclause=selectClause ')' { isCount = true; }) )
+      K_FROM cf=columnFamilyName
+      ( K_USING K_CONSISTENCY K_LEVEL { cLevel = ConsistencyLevel.valueOf($K_LEVEL.text.toUpperCase()); } )?
+      ( K_WHERE wclause=whereClause )?
+      ( K_ORDER (K_ASC | K_DESC { reversed = true; }) )?
+      ( K_LIMIT rows=INTEGER { limit = Integer.parseInt($rows.text); } )?
+      {
+          SelectStatement.Parameters params = new SelectStatement.Parameters(cLevel,
+                                                                             limit,
+                                                                             reversed,
+                                                                             isCount);
+          $expr = new SelectStatement.RawStatement(cf, params, sclause, wclause);
+      }
+    ;
+
+selectClause returns [List<ColumnIdentifier> expr]
+    : ids=cidentList { $expr = ids; }
+    | '\*'           { $expr = Collections.<ColumnIdentifier>emptyList();}
+    ;
+
+whereClause returns [List<Relation> clause]
+    @init{ $clause = new ArrayList<Relation>(); }
+    : first=relation { $clause.add(first); } (K_AND next=relation { $clause.add(next); })*
+    ;
+
+/**
+ * INSERT INTO <CF> (<column>, <column>, <column>, ...)
+ * VALUES (<value>, <value>, <value>, ...)
+ * USING CONSISTENCY <level> AND TIMESTAMP <long>;
+ *
+ * Consistency level is set to ONE by default
+ */
+insertStatement returns [UpdateStatement expr]
+    @init {
+        Attributes attrs = new Attributes();
+        List<ColumnIdentifier> columnNames  = new ArrayList<ColumnIdentifier>();
+        List<Term> columnValues = new ArrayList<Term>();
+    }
+    : K_INSERT K_INTO cf=columnFamilyName
+          '(' c1=cident { columnNames.add(c1); }  ( ',' cn=cident { columnNames.add(cn); } )+ ')'
+        K_VALUES
+          '(' v1=term { columnValues.add(v1); } ( ',' vn=term { columnValues.add(vn); } )+ ')'
+        ( usingClause[attrs] )?
+      {
+          $expr = new UpdateStatement(cf, columnNames, columnValues, attrs);
+      }
+    ;
+
+usingClause[Attributes attrs]
+    : K_USING usingClauseObjective[attrs] ( K_AND? usingClauseObjective[attrs] )*
+    ;
+
+usingClauseDelete[Attributes attrs]
+    : K_USING usingClauseDeleteObjective[attrs] ( K_AND? usingClauseDeleteObjective[attrs] )*
+    ;
+
+usingClauseDeleteObjective[Attributes attrs]
+    : K_CONSISTENCY K_LEVEL  { attrs.cLevel = ConsistencyLevel.valueOf($K_LEVEL.text.toUpperCase()); }
+    | K_TIMESTAMP ts=INTEGER { attrs.timestamp = Long.valueOf($ts.text); }
+    ;
+
+usingClauseObjective[Attributes attrs]
+    : usingClauseDeleteObjective[attrs]
+    | K_TTL t=INTEGER { attrs.timeToLive = Integer.valueOf($t.text); }
+    ;
+
+/**
+ * UPDATE <CF>
+ * USING CONSISTENCY <level> AND TIMESTAMP <long>
+ * SET name1 = value1, name2 = value2
+ * WHERE key = value;
+ */
+updateStatement returns [UpdateStatement expr]
+    @init {
+        Attributes attrs = new Attributes();
+        Map<ColumnIdentifier, Operation> columns = new HashMap<ColumnIdentifier, Operation>();
+    }
+    : K_UPDATE cf=columnFamilyName
+      ( usingClause[attrs] )?
+      K_SET termPairWithOperation[columns] (',' termPairWithOperation[columns])*
+      K_WHERE wclause=whereClause
+      {
+          return new UpdateStatement(cf, columns, wclause, attrs);
+      }
+    ;
+
+/**
+ * DELETE name1, name2
+ * FROM <CF>
+ * USING CONSISTENCY <level> AND TIMESTAMP <long>
+ * WHERE KEY = keyname;
+ */
+deleteStatement returns [DeleteStatement expr]
+    @init {
+        Attributes attrs = new Attributes();
+        List<ColumnIdentifier> columnsList = Collections.emptyList();
+    }
+    : K_DELETE ( ids=cidentList { columnsList = ids; } )?
+      K_FROM cf=columnFamilyName
+      ( usingClauseDelete[attrs] )?
+      K_WHERE wclause=whereClause
+      {
+          return new DeleteStatement(cf, columnsList, wclause, attrs);
+      }
+    ;
+
+
+/**
+ * BEGIN BATCH [USING CONSISTENCY <LVL>]
+ *   UPDATE <CF> SET name1 = value1 WHERE KEY = keyname1;
+ *   UPDATE <CF> SET name2 = value2 WHERE KEY = keyname2;
+ *   UPDATE <CF> SET name3 = value3 WHERE KEY = keyname3;
+ *   ...
+ * APPLY BATCH
+ *
+ * OR
+ *
+ * BEGIN BATCH [USING CONSISTENCY <LVL>]
+ *   INSERT INTO <CF> (KEY, <name>) VALUES ('<key>', '<value>');
+ *   INSERT INTO <CF> (KEY, <name>) VALUES ('<key>', '<value>');
+ *   ...
+ * APPLY BATCH
+ *
+ * OR
+ *
+ * BEGIN BATCH [USING CONSISTENCY <LVL>]
+ *   DELETE name1, name2 FROM <CF> WHERE key = <key>
+ *   DELETE name3, name4 FROM <CF> WHERE key = <key>
+ *   ...
+ * APPLY BATCH
+ */
+batchStatement returns [BatchStatement expr]
+    @init {
+        Attributes attrs = new Attributes();
+        List<ModificationStatement> statements = new ArrayList<ModificationStatement>();
+    }
+    : K_BEGIN K_BATCH ( usingClause[attrs] )?
+          s1=batchStatementObjective ';'? { statements.add(s1); } ( sN=batchStatementObjective ';'? { statements.add(sN); } )*
+      K_APPLY K_BATCH
+      {
+          return new BatchStatement(statements, attrs);
+      }
+    ;
+
+batchStatementObjective returns [ModificationStatement statement]
+    : i=insertStatement  { $statement = i; }
+    | u=updateStatement  { $statement = u; }
+    | d=deleteStatement  { $statement = d; }
+    ;
+
+/**
+ * CREATE KEYSPACE <KEYSPACE> WITH attr1 = value1 AND attr2 = value2;
+ */
+createKeyspaceStatement returns [CreateKeyspaceStatement expr]
+    : K_CREATE K_KEYSPACE ks=keyspaceName
+      K_WITH props=properties { $expr = new CreateKeyspaceStatement(ks, props); }
+    ;
+
+/**
+ * CREATE COLUMNFAMILY <CF> (
+ *     <name1> <type>,
+ *     <name2> <type>,
+ *     <name3> <type>
+ * ) WITH <property> = <value> AND ...;
+ */
+createColumnFamilyStatement returns [CreateColumnFamilyStatement.RawStatement expr]
+    : K_CREATE K_COLUMNFAMILY cf=columnFamilyName { $expr = new CreateColumnFamilyStatement.RawStatement(cf); }
+      cfamDefinition[expr]
+    ;
+
+cfamDefinition[CreateColumnFamilyStatement.RawStatement expr]
+    : '(' cfamColumns[expr] ( ',' cfamColumns[expr]? )* ')'
+      ( K_WITH cfamProperty[expr] ( K_AND cfamProperty[expr] )*)?
+    ;
+
+cfamColumns[CreateColumnFamilyStatement.RawStatement expr]
+    : k=cident v=comparatorType { $expr.addDefinition(k, v); } (K_PRIMARY K_KEY { $expr.setKeyAlias(k); })?
+    | K_PRIMARY K_KEY '(' k=cident { $expr.setKeyAlias(k); } (',' c=cident { $expr.addColumnAlias(c); } )* ')'
+    ;
+
+cfamProperty[CreateColumnFamilyStatement.RawStatement expr]
+    : k=property '=' v=propertyValue { $expr.addProperty(k, v); }
+    | K_COMPACT K_STORAGE { $expr.setCompactStorage(); }
+    ;
+
+/**
+ * CREATE INDEX [indexName] ON columnFamily (columnName);
+ */
+createIndexStatement returns [CreateIndexStatement expr]
+    : K_CREATE K_INDEX (idxName=IDENT)? K_ON cf=columnFamilyName '(' id=cident ')'
+      { $expr = new CreateIndexStatement(cf, $idxName.text, id); }
+    ;
+
+/**
+ * ALTER COLUMN FAMILY <CF> ALTER <column> TYPE <newtype>;
+ * ALTER COLUMN FAMILY <CF> ADD <column> <newtype>;
+ * ALTER COLUMN FAMILY <CF> DROP <column>;
+ * ALTER COLUMN FAMILY <CF> WITH <property> = <value>;
+ */
+alterTableStatement returns [AlterTableStatement expr]
+    @init {
+        AlterTableStatement.Type type = null;
+        String validator = null;
+        ColumnIdentifier columnName = null;
+        Map<String, String> propertyMap = null;
+    }
+    : K_ALTER K_COLUMNFAMILY cf=columnFamilyName
+          ( K_ALTER id=cident K_TYPE v=comparatorType { type = AlterTableStatement.Type.ALTER; }
+          | K_ADD   id=cident v=comparatorType        { type = AlterTableStatement.Type.ADD; }
+          | K_DROP  id=cident                         { type = AlterTableStatement.Type.DROP; }
+          | K_WITH  props=properties                  { type = AlterTableStatement.Type.OPTS; }
+          )
+    {
+        $expr = new AlterTableStatement(cf, type, id, v, props);
+    }
+    ;
+
+/**
+ * DROP KEYSPACE <KSP>;
+ */
+dropKeyspaceStatement returns [DropKeyspaceStatement ksp]
+    : K_DROP K_KEYSPACE ks=keyspaceName { $ksp = new DropKeyspaceStatement(ks); }
+    ;
+
+/**
+ * DROP COLUMNFAMILY <CF>;
+ */
+dropColumnFamilyStatement returns [DropColumnFamilyStatement stmt]
+    : K_DROP K_COLUMNFAMILY cf=columnFamilyName { $stmt = new DropColumnFamilyStatement(cf); }
+    ;
+
+/**
+ * DROP INDEX <INDEX_NAME>
+ */
+dropIndexStatement returns [DropIndexStatement expr]
+    :
+      K_DROP K_INDEX index=IDENT
+      { $expr = new DropIndexStatement($index.text); }
+    ;
+
+/**
+  * TRUNCATE <CF>;
+  */
+truncateStatement returns [TruncateStatement stmt]
+    : K_TRUNCATE cf=columnFamilyName { $stmt = new TruncateStatement(cf); }
+    ;
+
+
+/** DEFINITIONS **/
+
+// Column Identifiers
+cident returns [ColumnIdentifier id]
+    : t=( IDENT | UUID | INTEGER ) { $id = new ColumnIdentifier($t.text, false); }
+    | t=QUOTED_NAME                { $id = new ColumnIdentifier($t.text, true); }
+    ;
+
+// Keyspace & Column family names
+keyspaceName returns [String id]
+    @init { CFName name = new CFName(); }
+    : cfOrKsName[name, true] { $id = name.getKeyspace(); }
+    ;
+
+columnFamilyName returns [CFName name]
+    @init { $name = new CFName(); }
+    : (cfOrKsName[name, true] '.')? cfOrKsName[name, false]
+    ;
+
+cfOrKsName[CFName name, boolean isKs]
+    : t=IDENT        { if (isKs) $name.setKeyspace($t.text, false); else $name.setColumnFamily($t.text, false); }
+    | t=QUOTED_NAME  { if (isKs) $name.setKeyspace($t.text, true); else $name.setColumnFamily($t.text, true); }
+    ;
+
+cidentList returns [List<ColumnIdentifier> items]
+    @init{ $items = new ArrayList<ColumnIdentifier>(); }
+    :  t1=cident { $items.add(t1); } (',' tN=cident { $items.add(tN); })*
+    ;
+
+// Values (includes prepared statement markers)
+term returns [Term term]
+    : t=(STRING_LITERAL | UUID | IDENT | INTEGER | FLOAT ) { $term = new Term($t.text, $t.type); }
+    | t=QMARK                                              { $term = new Term($t.text, $t.type, ++currentBindMarkerIdx); }
+    ;
+
+intTerm returns [Term integer]
+    : t=INTEGER { $integer = new Term($t.text, $t.type); }
+    | t=QMARK   { $integer = new Term($t.text, $t.type, ++currentBindMarkerIdx); }
+    ;
+
+termPairWithOperation[Map<ColumnIdentifier, Operation> columns]
+    : key=cident '='
+        ( value=term { columns.put(key, new Operation(value)); }
+        | c=cident ( '+'     v=intTerm { columns.put(key, new Operation(c, Operation.Type.PLUS, v)); }
+                   | op='-'? v=intTerm
+                     {
+                       validateMinusSupplied(op, v, input);
+                       if (op == null)
+                           v = new Term(-(Long.valueOf(v.getText())), v.getType());
+                       columns.put(key, new Operation(c, Operation.Type.MINUS, v));
+                     }
+                    )
+        )
+    ;
+
+property returns [String str]
+    : p=(COMPIDENT | IDENT) { $str = $p.text; }
+    ;
+
+propertyValue returns [String str]
+    : v=(STRING_LITERAL | IDENT | INTEGER | FLOAT) { $str = $v.text; }
+    ;
+
+properties returns [Map<String, String> props]
+    @init{ $props = new HashMap<String, String>(); }
+    : k1=property '=' v1=propertyValue { $props.put(k1, v1); } (K_AND kn=property '=' vn=propertyValue { $props.put(kn, vn); } )*
+    ;
+
+relation returns [Relation rel]
+    : name=cident type=('=' | '<' | '<=' | '>=' | '>') t=term { $rel = new Relation($name.id, $type.text, $t.term); }
+    | name=cident K_IN { $rel = Relation.createInRelation($name.id); }
+      '(' f1=term { $rel.addInValue(f1); } (',' fN=term { $rel.addInValue(fN); } )* ')'
+    ;
+
+comparatorType returns [String str]
+    : c=(IDENT | STRING_LITERAL) { $str = $c.text; }
+    ;
+
+
+// Case-insensitive keywords
+K_SELECT:      S E L E C T;
+K_FROM:        F R O M;
+K_WHERE:       W H E R E;
+K_AND:         A N D;
+K_KEY:         K E Y;
+K_INSERT:      I N S E R T;
+K_UPDATE:      U P D A T E;
+K_WITH:        W I T H;
+K_LIMIT:       L I M I T;
+K_USING:       U S I N G;
+K_CONSISTENCY: C O N S I S T E N C Y;
+K_LEVEL:       ( O N E
+               | Q U O R U M
+               | A L L
+               | A N Y
+               | L O C A L '_' Q U O R U M
+               | E A C H '_' Q U O R U M
+               )
+               ;
+K_USE:         U S E;
+K_COUNT:       C O U N T;
+K_SET:         S E T;
+K_BEGIN:       B E G I N;
+K_APPLY:       A P P L Y;
+K_BATCH:       B A T C H;
+K_TRUNCATE:    T R U N C A T E;
+K_DELETE:      D E L E T E;
+K_IN:          I N;
+K_CREATE:      C R E A T E;
+K_KEYSPACE:    ( K E Y S P A C E
+                 | S C H E M A );
+K_COLUMNFAMILY:( C O L U M N F A M I L Y
+                 | T A B L E );
+K_INDEX:       I N D E X;
+K_ON:          O N;
+K_DROP:        D R O P;
+K_PRIMARY:     P R I M A R Y;
+K_INTO:        I N T O;
+K_VALUES:      V A L U E S;
+K_TIMESTAMP:   T I M E S T A M P;
+K_TTL:         T T L;
+K_ALTER:       A L T E R;
+K_ADD:         A D D;
+K_TYPE:        T Y P E;
+K_COMPACT:     C O M P A C T;
+K_STORAGE:     S T O R A G E;
+K_ORDER:       O R D E R;
+K_ASC:         A S C;
+K_DESC:        D E S C;
+
+// Case-insensitive alpha characters
+fragment A: ('a'|'A');
+fragment B: ('b'|'B');
+fragment C: ('c'|'C');
+fragment D: ('d'|'D');
+fragment E: ('e'|'E');
+fragment F: ('f'|'F');
+fragment G: ('g'|'G');
+fragment H: ('h'|'H');
+fragment I: ('i'|'I');
+fragment J: ('j'|'J');
+fragment K: ('k'|'K');
+fragment L: ('l'|'L');
+fragment M: ('m'|'M');
+fragment N: ('n'|'N');
+fragment O: ('o'|'O');
+fragment P: ('p'|'P');
+fragment Q: ('q'|'Q');
+fragment R: ('r'|'R');
+fragment S: ('s'|'S');
+fragment T: ('t'|'T');
+fragment U: ('u'|'U');
+fragment V: ('v'|'V');
+fragment W: ('w'|'W');
+fragment X: ('x'|'X');
+fragment Y: ('y'|'Y');
+fragment Z: ('z'|'Z');
+
+STRING_LITERAL
+    @init{ StringBuilder b = new StringBuilder(); }
+    @after{ setText(b.toString()); }
+    : '\'' (c=~('\'') { b.appendCodePoint(c);} | '\'' '\'' { b.appendCodePoint('\''); })* '\''
+    ;
+
+QUOTED_NAME
+    @init{ StringBuilder b = new StringBuilder(); }
+    @after{ setText(b.toString()); }
+    : '\"' (c=~('\"') { b.appendCodePoint(c); } | '\"' '\"' { b.appendCodePoint('\"'); })* '\"'
+    ;
+
+fragment DIGIT
+    : '0'..'9'
+    ;
+
+fragment LETTER
+    : ('A'..'Z' | 'a'..'z')
+    ;
+
+fragment HEX
+    : ('A'..'F' | 'a'..'f' | '0'..'9')
+    ;
+
+INTEGER
+    : '-'? DIGIT+
+    ;
+
+QMARK
+    : '?'
+    ;
+
+/*
+ * Normally a lexer only emits one token at a time, but ours is tricked out
+ * to support multiple (see @lexer::members near the top of the grammar).
+ */
+FLOAT
+    : INTEGER '.' INTEGER
+    ;
+
+IDENT
+    : LETTER (LETTER | DIGIT | '_')*
+    ;
+
+COMPIDENT
+    : IDENT ( ':' (IDENT | INTEGER))+
+    ;
+
+UUID
+    : HEX HEX HEX HEX HEX HEX HEX HEX '-'
+      HEX HEX HEX HEX '-'
+      HEX HEX HEX HEX '-'
+      HEX HEX HEX HEX '-'
+      HEX HEX HEX HEX HEX HEX HEX HEX HEX HEX HEX HEX
+    ;
+
+WS
+    : (' ' | '\t' | '\n' | '\r')+ { $channel = HIDDEN; }
+    ;
+
+COMMENT
+    : ('--' | '//') .* ('\n'|'\r') { $channel = HIDDEN; }
+    ;
+
+MULTILINE_COMMENT
+    : '/*' .* '*/' { $channel = HIDDEN; }
+    ;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/655ccc3a/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
new file mode 100644
index 0000000..63097b9
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/Operation.java
@@ -0,0 +1,54 @@
+/*
+ * 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;
+
+public class Operation
+{
+    public static enum Type { PLUS, MINUS }
+
+    public final Type type;
+    public final ColumnIdentifier ident;
+    public final Term value;
+
+    // unary operation
+    public Operation(Term a)
+    {
+        this(null, null, a);
+    }
+
+    // binary operation
+    public Operation(ColumnIdentifier a, Type type, Term b)
+    {
+        this.ident = a;
+        this.type = type;
+        this.value = b;
+    }
+
+    public boolean isUnary()
+    {
+        return type == null && ident == null;
+    }
+
+    public String toString()
+    {
+        return (isUnary())
+                ? String.format("UnaryOperation(%s)", value)
+                : String.format("BinaryOperation(%s, %s, %s)", ident, type, value);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/655ccc3a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
new file mode 100644
index 0000000..7c82616
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.cassandra.cql3;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.base.Predicates;
+import com.google.common.collect.Maps;
+import org.antlr.runtime.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.cql3.statements.*;
+import org.apache.cassandra.config.*;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.thrift.Column;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.SemanticVersion;
+
+public class QueryProcessor
+{
+    public static final SemanticVersion CQL_VERSION = new SemanticVersion("3.0.0-beta1");
+
+    private static final Logger logger = LoggerFactory.getLogger(QueryProcessor.class);
+
+    public static void validateKey(ByteBuffer key) throws InvalidRequestException
+    {
+        if (key == null || key.remaining() == 0)
+        {
+            throw new InvalidRequestException("Key may not be empty");
+        }
+
+        // check that key can be handled by FBUtilities.writeShortByteArray
+        if (key.remaining() > FBUtilities.MAX_UNSIGNED_SHORT)
+        {
+            throw new InvalidRequestException("Key length of " + key.remaining() +
+                                              " is longer than maximum of " + FBUtilities.MAX_UNSIGNED_SHORT);
+        }
+    }
+
+    public static void validateColumnNames(Iterable<ByteBuffer> columns)
+    throws InvalidRequestException
+    {
+        for (ByteBuffer name : columns)
+        {
+            if (name.remaining() > IColumn.MAX_NAME_LENGTH)
+                throw new InvalidRequestException(String.format("column name is too long (%s > %s)",
+                                                                name.remaining(),
+                                                                IColumn.MAX_NAME_LENGTH));
+            if (name.remaining() == 0)
+                throw new InvalidRequestException("zero-length column name");
+        }
+    }
+
+    public static void validateColumnName(ByteBuffer column)
+    throws InvalidRequestException
+    {
+        validateColumnNames(Collections.singletonList(column));
+    }
+
+    public static void validateSlicePredicate(CFMetaData metadata, SlicePredicate predicate)
+    throws InvalidRequestException
+    {
+        if (predicate.slice_range != null)
+            validateSliceRange(metadata, predicate.slice_range);
+        else
+            validateColumnNames(predicate.column_names);
+    }
+
+    public static void validateSliceRange(CFMetaData metadata, SliceRange range)
+    throws InvalidRequestException
+    {
+        validateSliceRange(metadata, range.start, range.finish, range.reversed);
+    }
+
+    public static void validateSliceRange(CFMetaData metadata, ByteBuffer start, ByteBuffer finish, boolean reversed)
+    throws InvalidRequestException
+    {
+        AbstractType<?> comparator = metadata.getComparatorFor(null);
+        Comparator<ByteBuffer> orderedComparator = reversed ? comparator.reverseComparator: comparator;
+        if (start.remaining() > 0 && finish.remaining() > 0 && orderedComparator.compare(start, finish) > 0)
+            throw new InvalidRequestException("Range finish must come after start in traversal order");
+    }
+
+    private static CqlResult processStatement(CQLStatement statement, ClientState clientState, List<ByteBuffer> variables)
+    throws  UnavailableException, InvalidRequestException, TimedOutException, SchemaDisagreementException
+    {
+        statement.checkAccess(clientState);
+        statement.validate(clientState);
+        CqlResult result = statement.execute(clientState, variables);
+        if (result == null)
+        {
+            result = new CqlResult();
+            result.type = CqlResultType.VOID;
+        }
+        return result;
+    }
+
+    public static CqlResult process(String queryString, ClientState clientState)
+    throws RecognitionException, UnavailableException, InvalidRequestException, TimedOutException, SchemaDisagreementException
+    {
+        logger.trace("CQL QUERY: {}", queryString);
+        return processStatement(getStatement(queryString, clientState), clientState, Collections.<ByteBuffer>emptyList());
+    }
+
+    public static CqlPreparedResult prepare(String queryString, ClientState clientState)
+    throws RecognitionException, InvalidRequestException
+    {
+        logger.trace("CQL QUERY: {}", queryString);
+
+        CQLStatement statement = getStatement(queryString, clientState);
+        int statementId = makeStatementId(queryString);
+        logger.trace("Discovered "+ statement.getBoundsTerms() + " bound variables.");
+
+        clientState.getCQL3Prepared().put(statementId, statement);
+        logger.trace(String.format("Stored prepared statement #%d with %d bind markers",
+                                   statementId,
+                                   statement.getBoundsTerms()));
+
+        return new CqlPreparedResult(statementId, statement.getBoundsTerms());
+    }
+
+    public static CqlResult processPrepared(CQLStatement statement, ClientState clientState, List<ByteBuffer> variables)
+    throws UnavailableException, InvalidRequestException, TimedOutException, SchemaDisagreementException
+    {
+        // Check to see if there are any bound variables to verify
+        if (!(variables.isEmpty() && (statement.getBoundsTerms() == 0)))
+        {
+            if (variables.size() != statement.getBoundsTerms())
+                throw new InvalidRequestException(String.format("there were %d markers(?) in CQL but %d bound variables",
+                                                                statement.getBoundsTerms(),
+                                                                variables.size()));
+
+            // at this point there is a match in count between markers and variables that is non-zero
+
+            if (logger.isTraceEnabled())
+                for (int i = 0; i < variables.size(); i++)
+                    logger.trace("[{}] '{}'", i+1, variables.get(i));
+        }
+
+        return processStatement(statement, clientState, variables);
+    }
+
+    private static final int makeStatementId(String cql)
+    {
+        // use the hash of the string till something better is provided
+        return cql.hashCode();
+    }
+
+    private static CQLStatement getStatement(String queryStr, ClientState clientState) throws InvalidRequestException, RecognitionException
+    {
+        CQLStatement statement = parseStatement(queryStr);
+
+        // Set keyspace for statement that require login
+        if (statement instanceof CFStatement)
+            ((CFStatement)statement).prepareKeyspace(clientState);
+
+        if (statement instanceof Preprocessable)
+            statement = ((Preprocessable)statement).preprocess();
+
+        return statement;
+    }
+
+    private static CQLStatement parseStatement(String queryStr) throws InvalidRequestException, RecognitionException
+    {
+        // Lexer and parser
+        CharStream stream = new ANTLRStringStream(queryStr);
+        CqlLexer lexer = new CqlLexer(stream);
+        TokenStream tokenStream = new CommonTokenStream(lexer);
+        CqlParser parser = new CqlParser(tokenStream);
+
+        // Parse the query string to a statement instance
+        CQLStatement statement = parser.query();
+
+        // The lexer and parser queue up any errors they may have encountered
+        // along the way, if necessary, we turn them into exceptions here.
+        lexer.throwLastRecognitionError();
+        parser.throwLastRecognitionError();
+
+        return statement;
+    }
+
+}