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

[1/4] Remove CQL2

Repository: cassandra
Updated Branches:
  refs/heads/trunk 3d7c5fdd3 -> 262170712


http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/cql/hooks/PreExecutionHook.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/hooks/PreExecutionHook.java b/src/java/org/apache/cassandra/cql/hooks/PreExecutionHook.java
deleted file mode 100644
index 29ed38e..0000000
--- a/src/java/org/apache/cassandra/cql/hooks/PreExecutionHook.java
+++ /dev/null
@@ -1,46 +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.cql.hooks;
-
-import org.apache.cassandra.cql.CQLStatement;
-import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-
-/**
- * Run before the CQL Statement is executed in
- * {@link org.apache.cassandra.cql.QueryProcessor}. The CQLStatement
- * returned from the processStatement method is what is actually executed
- * by the QueryProcessor.
- */
-public interface PreExecutionHook
-{
-    /**
-     * Perform pre-processing on a CQL statement prior to it being
-     * executed by the QueryProcessor. If required, implementations
-     * may modify the statement as the returned instance is what
-     * is actually executed.
-     *
-     * @param statement the statement to perform pre-processing on
-     * @param context execution context containing additional info
-     *                about the operation and statement
-     * @return the actual statement that will be executed, possibly
-     *         a modification of the initial statement
-     * @throws RequestExecutionException, RequestValidationException
-     */
-    CQLStatement processStatement(CQLStatement statement, ExecutionContext context) throws RequestExecutionException, RequestValidationException;
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/cql/hooks/PreparationContext.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/hooks/PreparationContext.java b/src/java/org/apache/cassandra/cql/hooks/PreparationContext.java
deleted file mode 100644
index 00cce78..0000000
--- a/src/java/org/apache/cassandra/cql/hooks/PreparationContext.java
+++ /dev/null
@@ -1,39 +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.cql.hooks;
-
-import org.apache.cassandra.cql.CQLStatement;
-import org.apache.cassandra.thrift.ThriftClientState;
-
-/**
- * Contextual information about the preparation of a CQLStatement.
- * Used by {@link org.apache.cassandra.cql.hooks.PostPreparationHook}
- */
-public class PreparationContext
-{
-    public final ThriftClientState clientState;
-    public final String queryString;
-    public final CQLStatement statement;
-
-    public PreparationContext(ThriftClientState clientState, String queryString, CQLStatement statement)
-    {
-        this.clientState = clientState;
-        this.queryString = queryString;
-        this.statement = statement;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/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
index 5acb367..bd88852 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -50,7 +50,6 @@ public class QueryProcessor
     private static final Logger logger = LoggerFactory.getLogger(QueryProcessor.class);
     private static final MemoryMeter meter = new MemoryMeter().withGuessing(MemoryMeter.Guess.FALLBACK_BEST);
     private static final long MAX_CACHE_PREPARED_MEMORY = Runtime.getRuntime().maxMemory() / 256;
-    private static final int MAX_CACHE_PREPARED_COUNT = 10000;
 
     private static EntryWeigher<MD5Digest, CQLStatement> cqlMemoryUsageWeigher = new EntryWeigher<MD5Digest, CQLStatement>()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/db/composites/CType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/CType.java b/src/java/org/apache/cassandra/db/composites/CType.java
index c2516f5..0e26b1f 100644
--- a/src/java/org/apache/cassandra/db/composites/CType.java
+++ b/src/java/org/apache/cassandra/db/composites/CType.java
@@ -97,19 +97,19 @@ public interface CType extends Comparator<Composite>
     /**
      * Deserialize a Composite from a ByteBuffer.
      *
-     * This is meant for thrift/cql2 to convert the fully serialized buffer we
+     * This is meant for thrift to convert the fully serialized buffer we
      * get from the clients to composites.
      */
     public Composite fromByteBuffer(ByteBuffer bb);
 
     /**
-     * Returns a AbstractType corresponding to this CType for thrift/cql2 sake.
+     * Returns a AbstractType corresponding to this CType for thrift sake.
      *
      * If the CType is a "simple" one, this just return the wrapped type, otherwise
      * it returns the corresponding org.apache.cassandra.db.marshal.CompositeType.
      *
      * This is only meant to be use for backward compatibility (particularly for
-     * thrift/cql2) but it's not meant to be used internally.
+     * thrift) but it's not meant to be used internally.
      */
     public AbstractType<?> asAbstractType();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/db/composites/CellNameType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/CellNameType.java b/src/java/org/apache/cassandra/db/composites/CellNameType.java
index 7128c91..e4fa0e2 100644
--- a/src/java/org/apache/cassandra/db/composites/CellNameType.java
+++ b/src/java/org/apache/cassandra/db/composites/CellNameType.java
@@ -144,7 +144,7 @@ public interface CellNameType extends CType
      * Deserialize a Composite from a ByteBuffer.
      *
      * This is equilvalent to CType#fromByteBuffer but assumes the buffer is a full cell
-     * name. This is meant for thrift/cql2 to convert the fully serialized buffer we
+     * name. This is meant for thrift to convert the fully serialized buffer we
      * get from the clients.
      */
     public CellName cellFromByteBuffer(ByteBuffer bb);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/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 ce233de..c148366 100644
--- a/src/java/org/apache/cassandra/db/marshal/AbstractType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
@@ -84,12 +84,6 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>
     /** get a byte representation of the given string. */
     public abstract ByteBuffer fromString(String source) throws MarshalException;
 
-    /** for compatibility with TimeUUID in CQL2. See TimeUUIDType (that overrides it). */
-    public ByteBuffer fromStringCQL2(String source) throws MarshalException
-    {
-        return fromString(source);
-    }
-
     /* validate that the byte array is a valid sequence for the type we are supposed to be comparing */
     public void validate(ByteBuffer bytes) throws MarshalException
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/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 51cf47a..b16b3dd 100644
--- a/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
@@ -83,41 +83,6 @@ public class TimeUUIDType extends AbstractType<UUID>
         return (o1.get(o1Pos + 3) & 0xFF) - (o2.get(o2Pos + 3) & 0xFF);
     }
 
-    // This accepts dates are valid TimeUUID represensation, which is bogus
-    // (see #4936) but kept for CQL2 for compatibility sake.
-    @Override
-    public ByteBuffer fromStringCQL2(String source) throws MarshalException
-    {
-        // Return an empty ByteBuffer for an empty string.
-        if (source.isEmpty())
-            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
-
-        ByteBuffer idBytes = null;
-
-        // ffffffff-ffff-ffff-ffff-ffffffffff
-        if (regexPattern.matcher(source).matches())
-        {
-            UUID uuid = null;
-            try
-            {
-                uuid = UUID.fromString(source);
-                idBytes = decompose(uuid);
-            }
-            catch (IllegalArgumentException e)
-            {
-                throw new MarshalException(String.format("unable to make UUID from '%s'", source), e);
-            }
-
-            if (uuid.version() != 1)
-                throw new MarshalException("TimeUUID supports only version 1 UUIDs");
-        } else
-        {
-            idBytes = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes(DateType.dateStringToTimestamp(source)));
-        }
-
-        return idBytes;
-    }
-
     public ByteBuffer fromString(String source) throws MarshalException
     {
         // Return an empty ByteBuffer for an empty string.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/db/marshal/UUIDType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/UUIDType.java b/src/java/org/apache/cassandra/db/marshal/UUIDType.java
index 4b0751e..cb2dd33 100644
--- a/src/java/org/apache/cassandra/db/marshal/UUIDType.java
+++ b/src/java/org/apache/cassandra/db/marshal/UUIDType.java
@@ -17,9 +17,7 @@
  */
 package org.apache.cassandra.db.marshal;
 
-
 import java.nio.ByteBuffer;
-import java.text.ParseException;
 import java.util.UUID;
 
 import org.apache.cassandra.cql3.CQL3Type;
@@ -28,9 +26,6 @@ import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.UUIDSerializer;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.UUIDGen;
-import org.apache.commons.lang3.time.DateUtils;
-
-import static org.apache.cassandra.serializers.TimestampSerializer.iso8601Patterns;
 
 /**
  * Compares UUIDs using the following criteria:<br>
@@ -165,52 +160,20 @@ public class UUIDType extends AbstractType<UUID>
         if (source.isEmpty())
             return ByteBufferUtil.EMPTY_BYTE_BUFFER;
 
-        ByteBuffer idBytes = null;
-
         // ffffffff-ffff-ffff-ffff-ffffffffff
         if (TimeUUIDType.regexPattern.matcher(source).matches())
         {
-            UUID uuid;
             try
             {
-                uuid = UUID.fromString(source);
-                idBytes = ByteBuffer.wrap(UUIDGen.decompose(uuid));
+                return ByteBuffer.wrap(UUIDGen.decompose(UUID.fromString(source)));
             }
             catch (IllegalArgumentException e)
             {
                 throw new MarshalException(String.format("unable to make UUID from '%s'", source), e);
             }
-        } else if (source.toLowerCase().equals("now"))
-        {
-            idBytes = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes());
-        }
-        // Milliseconds since epoch?
-        else if (source.matches("^\\d+$"))
-        {
-            try
-            {
-                idBytes = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes(Long.parseLong(source)));
-            }
-            catch (NumberFormatException e)
-            {
-                throw new MarshalException(String.format("unable to make version 1 UUID from '%s'", source), e);
-            }
-        }
-        // Last chance, attempt to parse as date-time string
-        else
-        {
-            try
-            {
-                long timestamp = DateUtils.parseDate(source, iso8601Patterns).getTime();
-                idBytes = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes(timestamp));
-            }
-            catch (ParseException e1)
-            {
-                throw new MarshalException(String.format("unable to coerce '%s' to version 1 UUID", source), e1);
-            }
         }
 
-        return idBytes;
+        throw new MarshalException(String.format("unable to coerce '%s' to version 1 UUID", source));
     }
 
     public CQL3Type asCQL3Type()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/service/ClientState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ClientState.java b/src/java/org/apache/cassandra/service/ClientState.java
index 7f312a9..7c3ba22 100644
--- a/src/java/org/apache/cassandra/service/ClientState.java
+++ b/src/java/org/apache/cassandra/service/ClientState.java
@@ -31,6 +31,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.cassandra.auth.*;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.exceptions.AuthenticationException;
@@ -73,8 +74,6 @@ public class ClientState
     private volatile AuthenticatedUser user;
     private volatile String keyspace;
 
-    private SemanticVersion cqlVersion;
-
     // isInternal is used to mark ClientState as used by some internal component
     // that should have an ability to modify system keyspace.
     private final boolean isInternal;
@@ -241,54 +240,14 @@ public class ClientState
             throw new InvalidRequestException("You have not set a keyspace for this session");
     }
 
-    public void setCQLVersion(String str) throws InvalidRequestException
-    {
-        SemanticVersion version;
-        try
-        {
-            version = new SemanticVersion(str);
-        }
-        catch (IllegalArgumentException e)
-        {
-            throw new InvalidRequestException(e.getMessage());
-        }
-
-        SemanticVersion cql = org.apache.cassandra.cql.QueryProcessor.CQL_VERSION;
-        SemanticVersion cql3 = org.apache.cassandra.cql3.QueryProcessor.CQL_VERSION;
-
-        // We've made some backward incompatible changes between CQL3 beta1 and the final.
-        // It's ok because it was a beta, but it still mean we don't support 3.0.0-beta1 so reject it.
-        SemanticVersion cql3Beta = new SemanticVersion("3.0.0-beta1");
-        if (version.equals(cql3Beta))
-            throw new InvalidRequestException(String.format("There has been a few syntax breaking changes between 3.0.0-beta1 and 3.0.0 "
-                                                           + "(mainly the syntax for options of CREATE KEYSPACE and CREATE TABLE). 3.0.0-beta1 "
-                                                           + " is not supported; please upgrade to 3.0.0"));
-        if (version.isSupportedBy(cql))
-            cqlVersion = cql;
-        else if (version.isSupportedBy(cql3))
-            cqlVersion = cql3;
-        else
-            throw new InvalidRequestException(String.format("Provided version %s is not supported by this server (supported: %s)",
-                                                            version,
-                                                            StringUtils.join(getCQLSupportedVersion(), ", ")));
-    }
-
     public AuthenticatedUser getUser()
     {
         return user;
     }
 
-    public SemanticVersion getCQLVersion()
-    {
-        return cqlVersion;
-    }
-
     public static SemanticVersion[] getCQLSupportedVersion()
     {
-        SemanticVersion cql = org.apache.cassandra.cql.QueryProcessor.CQL_VERSION;
-        SemanticVersion cql3 = org.apache.cassandra.cql3.QueryProcessor.CQL_VERSION;
-
-        return new SemanticVersion[]{ cql, cql3 };
+        return new SemanticVersion[]{ QueryProcessor.CQL_VERSION };
     }
 
     private static LoadingCache<Pair<AuthenticatedUser, IResource>, Set<Permission>> initPermissionsCache()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/thrift/CassandraServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CassandraServer.java b/src/java/org/apache/cassandra/thrift/CassandraServer.java
index 650c74e..a5af55d 100644
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@ -44,8 +44,6 @@ import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.cql.CQLStatement;
-import org.apache.cassandra.cql.QueryProcessor;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.*;
@@ -71,7 +69,6 @@ import org.apache.cassandra.service.pager.QueryPagers;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
-import org.apache.cassandra.utils.SemanticVersion;
 import org.apache.cassandra.utils.UUIDGen;
 import org.apache.thrift.TException;
 
@@ -83,8 +80,6 @@ public class CassandraServer implements Cassandra.Iface
 
     private final static List<ColumnOrSuperColumn> EMPTY_COLUMNS = Collections.emptyList();
 
-    private volatile boolean loggedCQL2Warning = false;
-
     /*
      * RequestScheduler to perform the scheduling of incoming requests
      */
@@ -1895,67 +1890,13 @@ public class CassandraServer implements Cassandra.Iface
         return queryString;
     }
 
-    private void validateCQLVersion(int major) throws InvalidRequestException
-    {
-        /*
-         * The rules are:
-         *   - If no version are set, we don't validate anything. The reason is
-         *     that 1) old CQL2 client might not have called set_cql_version
-         *     and 2) some client may have removed the set_cql_version for CQL3
-         *     when updating to 1.2.0. A CQL3 client upgrading from pre-1.2
-         *     shouldn't be in that case however since set_cql_version uses to
-         *     be mandatory (for CQL3).
-         *   - Otherwise, checks the major matches whatever was set.
-         */
-        SemanticVersion versionSet = state().getCQLVersion();
-        if (versionSet == null)
-            return;
-
-        if (versionSet.major != major)
-            throw new InvalidRequestException(
-                "Cannot execute/prepare CQL" + major + " statement since the CQL has been set to CQL" + versionSet.major
-              + "(This might mean your client hasn't been upgraded correctly to use the new CQL3 methods introduced in Cassandra 1.2+).");
-    }
-
-    public CqlResult execute_cql_query(ByteBuffer query, Compression compression)
-    throws InvalidRequestException, UnavailableException, TimedOutException, SchemaDisagreementException, TException
+    public CqlResult execute_cql_query(ByteBuffer query, Compression compression) throws TException
     {
-        validateCQLVersion(2);
-        maybeLogCQL2Warning();
-
-        try
-        {
-            String queryString = uncompress(query, compression);
-            if (startSessionIfRequested())
-            {
-                Tracing.instance.begin("execute_cql_query",
-                                       ImmutableMap.of("query", queryString));
-            }
-            else
-            {
-                logger.debug("execute_cql_query");
-            }
-
-            return QueryProcessor.process(queryString, state());
-        }
-        catch (RequestExecutionException e)
-        {
-            throw ThriftConversion.rethrow(e);
-        }
-        catch (RequestValidationException e)
-        {
-            throw ThriftConversion.toThrift(e);
-        }
-        finally
-        {
-            Tracing.instance.stopSession();
-        }
+        throw new InvalidRequestException("CQL2 has been removed in Cassandra 3.0. Please use CQL3 instead");
     }
 
-    public CqlResult execute_cql3_query(ByteBuffer query, Compression compression, ConsistencyLevel cLevel)
-    throws InvalidRequestException, UnavailableException, TimedOutException, SchemaDisagreementException, TException
+    public CqlResult execute_cql3_query(ByteBuffer query, Compression compression, ConsistencyLevel cLevel) throws TException
     {
-        validateCQLVersion(3);
         try
         {
             String queryString = uncompress(query, compression);
@@ -1986,36 +1927,14 @@ public class CassandraServer implements Cassandra.Iface
         }
     }
 
-    public CqlPreparedResult prepare_cql_query(ByteBuffer query, Compression compression)
-    throws InvalidRequestException, TException
+    public CqlPreparedResult prepare_cql_query(ByteBuffer query, Compression compression) throws TException
     {
-        if (logger.isDebugEnabled())
-            logger.debug("prepare_cql_query");
-
-        validateCQLVersion(2);
-        maybeLogCQL2Warning();
-
-        String queryString = uncompress(query, compression);
-        ThriftClientState cState = state();
-
-        try
-        {
-            cState.validateLogin();
-            return QueryProcessor.prepare(queryString, cState);
-        }
-        catch (RequestValidationException e)
-        {
-            throw ThriftConversion.toThrift(e);
-        }
+        throw new InvalidRequestException("CQL2 has been removed in Cassandra 3.0. Please use CQL3 instead");
     }
 
-    public CqlPreparedResult prepare_cql3_query(ByteBuffer query, Compression compression)
-    throws InvalidRequestException, TException
+    public CqlPreparedResult prepare_cql3_query(ByteBuffer query, Compression compression) throws TException
     {
-        if (logger.isDebugEnabled())
-            logger.debug("prepare_cql3_query");
-
-        validateCQLVersion(3);
+        logger.debug("prepare_cql3_query");
 
         String queryString = uncompress(query, compression);
         ThriftClientState cState = state();
@@ -2031,52 +1950,13 @@ public class CassandraServer implements Cassandra.Iface
         }
     }
 
-    public CqlResult execute_prepared_cql_query(int itemId, List<ByteBuffer> bindVariables)
-    throws InvalidRequestException, UnavailableException, TimedOutException, SchemaDisagreementException, TException
+    public CqlResult execute_prepared_cql_query(int itemId, List<ByteBuffer> bindVariables) throws TException
     {
-        validateCQLVersion(2);
-        maybeLogCQL2Warning();
-
-        if (startSessionIfRequested())
-        {
-            // TODO we don't have [typed] access to CQL bind variables here.  CASSANDRA-4560 is open to add support.
-            Tracing.instance.begin("execute_prepared_cql_query", Collections.<String, String>emptyMap());
-        }
-        else
-        {
-            logger.debug("execute_prepared_cql_query");
-        }
-
-        try
-        {
-            ThriftClientState cState = state();
-            CQLStatement statement = cState.getPrepared().get(itemId);
-
-            if (statement == null)
-                throw new InvalidRequestException(String.format("Prepared query with ID %d not found", itemId));
-            logger.trace("Retrieved prepared statement #{} with {} bind markers", itemId, statement.boundTerms);
-
-            return QueryProcessor.processPrepared(statement, cState, bindVariables);
-        }
-        catch (RequestExecutionException e)
-        {
-            throw ThriftConversion.rethrow(e);
-        }
-        catch (RequestValidationException e)
-        {
-            throw ThriftConversion.toThrift(e);
-        }
-        finally
-        {
-            Tracing.instance.stopSession();
-        }
+        throw new InvalidRequestException("CQL2 has been removed in Cassandra 3.0. Please use CQL3 instead");
     }
 
-    public CqlResult execute_prepared_cql3_query(int itemId, List<ByteBuffer> bindVariables, ConsistencyLevel cLevel)
-    throws InvalidRequestException, UnavailableException, TimedOutException, SchemaDisagreementException, TException
+    public CqlResult execute_prepared_cql3_query(int itemId, List<ByteBuffer> bindVariables, ConsistencyLevel cLevel) throws TException
     {
-        validateCQLVersion(3);
-
         if (startSessionIfRequested())
         {
             // TODO we don't have [typed] access to CQL bind variables here.  CASSANDRA-4560 is open to add support.
@@ -2118,31 +1998,10 @@ public class CassandraServer implements Cassandra.Iface
     }
 
     /*
-     * Deprecated, but if a client sets CQL2, it is a no-op for compatibility sake.
-     * If it sets CQL3 however, we throw an IRE because this mean the client
-     * hasn't been updated for Cassandra 1.2 and should start using the new
-     * execute_cql3_query, etc... and there is no point no warning it early.
+     * No-op since 3.0.
      */
-    public void set_cql_version(String version) throws InvalidRequestException
+    public void set_cql_version(String version)
     {
-        try
-        {
-            state().setCQLVersion(version);
-        }
-        catch (org.apache.cassandra.exceptions.InvalidRequestException e)
-        {
-            throw new InvalidRequestException(e.getMessage());
-        }
-    }
-
-    private void maybeLogCQL2Warning()
-    {
-        if (!loggedCQL2Warning)
-        {
-            logger.warn("CQL2 has been deprecated since Cassandra 2.0, and will be removed entirely in version 2.2."
-                        + " Please switch to CQL3 before then.");
-            loggedCQL2Warning = true;
-        }
     }
 
     public ByteBuffer trace_next_query() throws TException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/thrift/ThriftClientState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/ThriftClientState.java b/src/java/org/apache/cassandra/thrift/ThriftClientState.java
index 56dcf6f..319169f 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftClientState.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftClientState.java
@@ -18,10 +18,8 @@
 package org.apache.cassandra.thrift;
 
 import java.net.SocketAddress;
-import java.util.*;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.cql.CQLStatement;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 
@@ -34,19 +32,8 @@ import org.apache.cassandra.service.QueryState;
  */
 public class ThriftClientState extends ClientState
 {
-    private static final int MAX_CACHE_PREPARED = 10000;    // Enough to keep buggy clients from OOM'ing us
-
     private final QueryState queryState;
 
-    // An LRU map of prepared statements
-    private final Map<Integer, CQLStatement> prepared = new LinkedHashMap<Integer, CQLStatement>(16, 0.75f, true)
-    {
-        protected boolean removeEldestEntry(Map.Entry<Integer, CQLStatement> eldest)
-        {
-            return size() > MAX_CACHE_PREPARED;
-        }
-    };
-
     public ThriftClientState(SocketAddress remoteAddress)
     {
         super(remoteAddress);
@@ -58,11 +45,6 @@ public class ThriftClientState extends ClientState
         return queryState;
     }
 
-    public Map<Integer, CQLStatement> getPrepared()
-    {
-        return prepared;
-    }
-
     public String getSchedulingValue()
     {
         switch(DatabaseDescriptor.getRequestSchedulerId())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/messages/StartupMessage.java b/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
index d2b4350..5ae7a75 100644
--- a/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
@@ -23,8 +23,6 @@ import java.util.Map;
 import org.jboss.netty.buffer.ChannelBuffer;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.*;
 import org.apache.cassandra.utils.SemanticVersion;
@@ -66,23 +64,20 @@ public class StartupMessage extends Message.Request
 
     public Message.Response execute(QueryState state)
     {
-        ClientState cState = state.getClientState();
         String cqlVersion = options.get(CQL_VERSION);
         if (cqlVersion == null)
             throw new ProtocolException("Missing value CQL_VERSION in STARTUP message");
 
         try 
         {
-            cState.setCQLVersion(cqlVersion);
+            if (new SemanticVersion(cqlVersion).compareTo(new SemanticVersion("2.99.0")) < 0)
+                throw new ProtocolException(String.format("CQL version %s is not supported by the binary protocol (supported version are >= 3.0.0)", cqlVersion));
         }
-        catch (InvalidRequestException e)
+        catch (IllegalArgumentException e)
         {
             throw new ProtocolException(e.getMessage());
         }
 
-        if (cState.getCQLVersion().compareTo(new SemanticVersion("2.99.0")) < 0)
-            throw new ProtocolException(String.format("CQL version %s is not supported by the binary protocol (supported version are >= 3.0.0)", cqlVersion));
-
         if (options.containsKey(COMPRESSION))
         {
             String compression = options.get(COMPRESSION).toLowerCase();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/triggers/TriggerExecutor.java b/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
index 91d0ea0..8d7f42b 100644
--- a/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
+++ b/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
@@ -28,9 +28,8 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
 import org.apache.cassandra.config.TriggerDefinition;
-import org.apache.cassandra.cql.QueryProcessor;
+import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -92,8 +91,8 @@ public class TriggerExecutor
         {
             QueryProcessor.validateKey(mutation.key());
             for (ColumnFamily tcf : mutation.getColumnFamilies())
-                for (CellName tName : tcf.getColumnNames())
-                    QueryProcessor.validateColumn(tcf.metadata(), tName, tcf.getColumn(tName).value());
+                for (Cell cell : tcf)
+                    cell.validateFields(tcf.metadata());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/utils/UUIDGen.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/UUIDGen.java b/src/java/org/apache/cassandra/utils/UUIDGen.java
index f385744..e5ab7fe 100644
--- a/src/java/org/apache/cassandra/utils/UUIDGen.java
+++ b/src/java/org/apache/cassandra/utils/UUIDGen.java
@@ -165,27 +165,6 @@ public class UUIDGen
     }
 
     /**
-     * Converts a milliseconds-since-epoch timestamp into the 16 byte representation
-     * of a type 1 UUID (a time-based UUID).
-     *
-     * <p><i><b>Deprecated:</b> This method goes again the principle of a time
-     * UUID and should not be used. For queries based on timestamp, minTimeUUID() and
-     * maxTimeUUID() can be used but this method has questionable usefulness. This is
-     * only kept because CQL2 uses it (see TimeUUID.fromStringCQL2) and we
-     * don't want to break compatibility.</i></p>
-     *
-     * <p><i><b>Warning:</b> This method is not guaranteed to return unique UUIDs; Multiple
-     * invocations using identical timestamps will result in identical UUIDs.</i></p>
-     *
-     * @param timeMillis
-     * @return a type 1 UUID represented as a byte[]
-     */
-    public static byte[] getTimeUUIDBytes(long timeMillis)
-    {
-        return createTimeUUIDBytes(instance.createTimeUnsafe(timeMillis));
-    }
-
-    /**
      * Converts a 100-nanoseconds precision timestamp into the 16 byte representation
      * of a type 1 UUID (a time-based UUID).
      *

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/tools/stress/src/org/apache/cassandra/stress/Operation.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/Operation.java b/tools/stress/src/org/apache/cassandra/stress/Operation.java
index 4519b19..198f7fe 100644
--- a/tools/stress/src/org/apache/cassandra/stress/Operation.java
+++ b/tools/stress/src/org/apache/cassandra/stress/Operation.java
@@ -131,10 +131,6 @@ public abstract class Operation
         {
             return settings.mode.cqlVersion == CqlVersion.CQL3;
         }
-        public boolean isCql2()
-        {
-            return settings.mode.cqlVersion == CqlVersion.CQL2;
-        }
         public Object getCqlCache()
         {
             return cqlCache;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/tools/stress/src/org/apache/cassandra/stress/operations/CqlCounterAdder.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/CqlCounterAdder.java b/tools/stress/src/org/apache/cassandra/stress/operations/CqlCounterAdder.java
index aae99b5..910b6ed 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/CqlCounterAdder.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/CqlCounterAdder.java
@@ -35,12 +35,9 @@ public class CqlCounterAdder extends CqlOperation<Integer>
     @Override
     protected String buildQuery()
     {
-        String counterCF = state.isCql2() ? "Counter1" : "Counter3";
+        String counterCF = "Counter3";
 
-        StringBuilder query = new StringBuilder("UPDATE ").append(wrapInQuotesIfRequired(counterCF));
-
-        if (state.isCql2())
-            query.append(" USING CONSISTENCY ").append(state.settings.command.consistencyLevel);
+        StringBuilder query = new StringBuilder("UPDATE ").append(wrapInQuotes(counterCF));
 
         query.append(" SET ");
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/tools/stress/src/org/apache/cassandra/stress/operations/CqlCounterGetter.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/CqlCounterGetter.java b/tools/stress/src/org/apache/cassandra/stress/operations/CqlCounterGetter.java
index 31fd20d..6186667 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/CqlCounterGetter.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/CqlCounterGetter.java
@@ -42,21 +42,7 @@ public class CqlCounterGetter extends CqlOperation<Integer>
     @Override
     protected String buildQuery()
     {
-        StringBuilder query = new StringBuilder("SELECT ");
-
-        if (state.isCql2())
-            query.append("FIRST ").append(state.settings.columns.maxColumnsPerKey).append(" ''..''");
-        else
-            query.append("*");
-
-        String counterCF = state.isCql2() ? "Counter1" : "Counter3";
-
-        query.append(" FROM ").append(wrapInQuotesIfRequired(counterCF));
-
-        if (state.isCql2())
-            query.append(" USING CONSISTENCY ").append(state.settings.command.consistencyLevel);
-
-        return query.append(" WHERE KEY=?").toString();
+        return "SELECT * FROM \"Counter3\" USING CONSISTENCY " + state.settings.command.consistencyLevel + " WHERE KEY=?";
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/tools/stress/src/org/apache/cassandra/stress/operations/CqlIndexedRangeSlicer.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/CqlIndexedRangeSlicer.java b/tools/stress/src/org/apache/cassandra/stress/operations/CqlIndexedRangeSlicer.java
index ff43322..25af04a 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/CqlIndexedRangeSlicer.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/CqlIndexedRangeSlicer.java
@@ -48,21 +48,10 @@ public class CqlIndexedRangeSlicer extends CqlOperation<byte[][]>
     @Override
     protected String buildQuery()
     {
-        StringBuilder query = new StringBuilder("SELECT ");
-
-        if (state.isCql2())
-            query.append(state.settings.columns.maxColumnsPerKey).append(" ''..''");
-        else
-            query.append("*");
-
-        query.append(" FROM Standard1");
-
-        if (state.isCql2())
-            query.append(" USING CONSISTENCY ").append(state.settings.command.consistencyLevel);
-
+        StringBuilder query = new StringBuilder("SELECT * FROM \"Standard1\"");
         final String columnName = getColumnName(1);
         query.append(" WHERE ").append(columnName).append("=?")
-                .append(" AND KEY > ? LIMIT ").append(((SettingsCommandMulti)state.settings.command).keysAtOnce);
+             .append(" AND KEY > ? LIMIT ").append(((SettingsCommandMulti)state.settings.command).keysAtOnce);
         return query.toString();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/tools/stress/src/org/apache/cassandra/stress/operations/CqlInserter.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/CqlInserter.java b/tools/stress/src/org/apache/cassandra/stress/operations/CqlInserter.java
index 8d964f5..1f8987d 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/CqlInserter.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/CqlInserter.java
@@ -38,10 +38,7 @@ public class CqlInserter extends CqlOperation<Integer>
     @Override
     protected String buildQuery()
     {
-        StringBuilder query = new StringBuilder("UPDATE ").append(wrapInQuotesIfRequired(state.settings.schema.columnFamily));
-
-        if (state.isCql2())
-            query.append(" USING CONSISTENCY ").append(state.settings.command.consistencyLevel);
+        StringBuilder query = new StringBuilder("UPDATE ").append(wrapInQuotes(state.settings.schema.columnFamily));
 
         query.append(" SET ");
 
@@ -55,12 +52,12 @@ public class CqlInserter extends CqlOperation<Integer>
                 if (state.isCql3())
                     throw new UnsupportedOperationException("Cannot use UUIDs in column names with CQL3");
 
-                query.append(wrapInQuotesIfRequired(UUIDGen.getTimeUUID().toString()))
+                query.append(wrapInQuotes(UUIDGen.getTimeUUID().toString()))
                         .append(" = ?");
             }
             else
             {
-                query.append(wrapInQuotesIfRequired("C" + i)).append(" = ?");
+                query.append(wrapInQuotes("C" + i)).append(" = ?");
             }
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/tools/stress/src/org/apache/cassandra/stress/operations/CqlOperation.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/CqlOperation.java b/tools/stress/src/org/apache/cassandra/stress/operations/CqlOperation.java
index b17f520..8674cc0 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/CqlOperation.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/CqlOperation.java
@@ -268,10 +268,7 @@ public abstract class CqlOperation<V> extends Operation
 
     public ClientWrapper wrap(ThriftClient client)
     {
-        return state.isCql3()
-                ? new Cql3CassandraClientWrapper(client)
-                : new Cql2CassandraClientWrapper(client);
-
+        return new Cql3CassandraClientWrapper(client);
     }
 
     public ClientWrapper wrap(JavaDriverClient client)
@@ -302,7 +299,7 @@ public abstract class CqlOperation<V> extends Operation
         @Override
         public <V> V execute(String query, ByteBuffer key, List<ByteBuffer> queryParams, ResultHandler<V> handler)
         {
-            String formattedQuery = formatCqlQuery(query, queryParams, state.isCql3());
+            String formattedQuery = formatCqlQuery(query, queryParams);
             return handler.javaDriverHandler().apply(client.execute(formattedQuery, ThriftConversion.fromThrift(state.settings.command.consistencyLevel)));
         }
 
@@ -334,7 +331,7 @@ public abstract class CqlOperation<V> extends Operation
         @Override
         public <V> V execute(String query, ByteBuffer key, List<ByteBuffer> queryParams, ResultHandler<V> handler)
         {
-            String formattedQuery = formatCqlQuery(query, queryParams, state.isCql3());
+            String formattedQuery = formatCqlQuery(query, queryParams);
             return handler.thriftHandler().apply(client.execute(formattedQuery, ThriftConversion.fromThrift(state.settings.command.consistencyLevel)));
         }
 
@@ -367,7 +364,7 @@ public abstract class CqlOperation<V> extends Operation
         @Override
         public <V> V execute(String query, ByteBuffer key, List<ByteBuffer> queryParams, ResultHandler<V> handler) throws TException
         {
-            String formattedQuery = formatCqlQuery(query, queryParams, true);
+            String formattedQuery = formatCqlQuery(query, queryParams);
             return handler.simpleNativeHandler().apply(
                     client.execute_cql3_query(formattedQuery, key, Compression.NONE, state.settings.command.consistencyLevel)
             );
@@ -389,40 +386,6 @@ public abstract class CqlOperation<V> extends Operation
         }
     }
 
-    // client wrapper for Cql2
-    private final class Cql2CassandraClientWrapper implements ClientWrapper
-    {
-        final ThriftClient client;
-        private Cql2CassandraClientWrapper(ThriftClient client)
-        {
-            this.client = client;
-        }
-
-        @Override
-        public <V> V execute(String query, ByteBuffer key, List<ByteBuffer> queryParams, ResultHandler<V> handler) throws TException
-        {
-            String formattedQuery = formatCqlQuery(query, queryParams, false);
-            return handler.simpleNativeHandler().apply(
-                    client.execute_cql_query(formattedQuery, key, Compression.NONE)
-            );
-        }
-
-        @Override
-        public <V> V execute(Object preparedStatementId, ByteBuffer key, List<ByteBuffer> queryParams, ResultHandler<V> handler) throws TException
-        {
-            Integer id = (Integer) preparedStatementId;
-            return handler.simpleNativeHandler().apply(
-                    client.execute_prepared_cql_query(id, key, queryParams)
-            );
-        }
-
-        @Override
-        public Object createPreparedStatement(String cqlQuery) throws TException
-        {
-            return client.prepare_cql_query(cqlQuery, Compression.NONE);
-        }
-    }
-
     // interface for building functions to standardise results from each client
     protected static interface ResultHandler<V>
     {
@@ -632,11 +595,9 @@ public abstract class CqlOperation<V> extends Operation
 
     }
 
-    private static String getUnQuotedCqlBlob(ByteBuffer term, boolean isCQL3)
+    private static String getUnQuotedCqlBlob(ByteBuffer term)
     {
-        return isCQL3
-                ? "0x" + ByteBufferUtil.bytesToHex(term)
-                : ByteBufferUtil.bytesToHex(term);
+        return "0x" + ByteBufferUtil.bytesToHex(term);
     }
 
     /**
@@ -647,7 +608,7 @@ public abstract class CqlOperation<V> extends Operation
      * @param parms sequence of string query parameters
      * @return formatted CQL query string
      */
-    private static String formatCqlQuery(String query, List<ByteBuffer> parms, boolean isCql3)
+    private static String formatCqlQuery(String query, List<ByteBuffer> parms)
     {
         int marker, position = 0;
         StringBuilder result = new StringBuilder();
@@ -658,7 +619,7 @@ public abstract class CqlOperation<V> extends Operation
         for (ByteBuffer parm : parms)
         {
             result.append(query.substring(position, marker));
-            result.append(getUnQuotedCqlBlob(parm, isCql3));
+            result.append(getUnQuotedCqlBlob(parm));
 
             position = marker + 1;
             if (-1 == (marker = query.indexOf('?', position + 1)))
@@ -671,11 +632,9 @@ public abstract class CqlOperation<V> extends Operation
         return result.toString();
     }
 
-    protected String wrapInQuotesIfRequired(String string)
+    protected String wrapInQuotes(String string)
     {
-        return state.settings.mode.cqlVersion == CqlVersion.CQL3
-                ? "\"" + string + "\""
-                : string;
+        return "\"" + string + "\"";
     }
 
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/tools/stress/src/org/apache/cassandra/stress/operations/CqlRangeSlicer.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/CqlRangeSlicer.java b/tools/stress/src/org/apache/cassandra/stress/operations/CqlRangeSlicer.java
index 76ba966..cce47fc 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/CqlRangeSlicer.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/CqlRangeSlicer.java
@@ -41,13 +41,7 @@ public class CqlRangeSlicer extends CqlOperation<Integer>
     @Override
     protected String buildQuery()
     {
-        StringBuilder query = new StringBuilder("SELECT FIRST ").append(state.settings.columns.maxColumnsPerKey)
-                .append(" ''..'' FROM ").append(state.settings.schema.columnFamily);
-
-        if (state.isCql2())
-            query.append(" USING CONSISTENCY ").append(state.settings.command.consistencyLevel);
-
-        return query.append(" WHERE KEY > ?").toString();
+        return "SELECT FIRST " + state.settings.columns.maxColumnsPerKey + " ''..'' FROM " + state.settings.schema.columnFamily + " WHERE KEY > ?";
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/tools/stress/src/org/apache/cassandra/stress/operations/CqlReader.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/CqlReader.java b/tools/stress/src/org/apache/cassandra/stress/operations/CqlReader.java
index 44da43f..aa949d4 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/CqlReader.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/CqlReader.java
@@ -42,10 +42,7 @@ public class CqlReader extends CqlOperation<ByteBuffer[][]>
 
         if (state.settings.columns.names == null)
         {
-            if (state.isCql2())
-                query.append("FIRST ").append(state.settings.columns.maxColumnsPerKey).append(" ''..''");
-            else
-                query.append("*");
+            query.append("*");
         }
         else
         {
@@ -57,10 +54,8 @@ public class CqlReader extends CqlOperation<ByteBuffer[][]>
             }
         }
 
-        query.append(" FROM ").append(wrapInQuotesIfRequired(state.settings.schema.columnFamily));
+        query.append(" FROM ").append(wrapInQuotes(state.settings.schema.columnFamily));
 
-        if (state.isCql2())
-            query.append(" USING CONSISTENCY ").append(state.settings.command.consistencyLevel);
         query.append(" WHERE KEY=?");
         return query.toString();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/tools/stress/src/org/apache/cassandra/stress/settings/CqlVersion.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/CqlVersion.java b/tools/stress/src/org/apache/cassandra/stress/settings/CqlVersion.java
index d7d09f6..1e963c7 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/CqlVersion.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/CqlVersion.java
@@ -25,7 +25,6 @@ public enum CqlVersion
 {
 
     NOCQL(null),
-    CQL2("2.0.0"),
     CQL3("3.0.0");
 
     public final String connectVersion;
@@ -41,8 +40,6 @@ public enum CqlVersion
             return NOCQL;
         switch(version.charAt(0))
         {
-            case '2':
-                return CQL2;
             case '3':
                 return CQL3;
             default:
@@ -55,11 +52,6 @@ public enum CqlVersion
         return this != NOCQL;
     }
 
-    public boolean isCql2()
-    {
-        return this == CQL2;
-    }
-
     public boolean isCql3()
     {
         return this == CQL3;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/tools/stress/src/org/apache/cassandra/stress/settings/Legacy.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/Legacy.java b/tools/stress/src/org/apache/cassandra/stress/settings/Legacy.java
index 5c793e6..5ddb346 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/Legacy.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/Legacy.java
@@ -60,7 +60,6 @@ public class Legacy implements Serializable
         availableOptions.addOption("i",  "progress-interval",    true,   "Progress Report Interval (seconds), default:10");
         availableOptions.addOption("g",  "keys-per-call",        true,   "Number of keys to get_range_slices or multiget per call, default:1000");
         availableOptions.addOption("l",  "replication-factor",   true,   "Replication Factor to use when creating needed column families, default:1");
-        availableOptions.addOption("L",  "enable-cql",           false,  "Perform queries using CQL2 (Cassandra Query Language v 2.0.0)");
         availableOptions.addOption("L3", "enable-cql3",          false,  "Perform queries using CQL3 (Cassandra Query Language v 3.0.0)");
         availableOptions.addOption("b",  "enable-native-protocol",  false,  "Use the binary native protocol (only work along with -L3)");
         availableOptions.addOption("P",  "use-prepared-statements", false, "Perform queries using prepared statements (only applicable to CQL).");
@@ -233,9 +232,7 @@ public class Legacy implements Serializable
                 r.add("-schema", "replication(" + rep + ")");
             }
 
-            if (cmd.hasOption("L"))
-                r.add("-mode", cmd.hasOption("P") ? "prepared cql2" : "cql2");
-            else if (cmd.hasOption("L3"))
+            if (cmd.hasOption("L3"))
                 r.add("-mode", (cmd.hasOption("P") ? "prepared" : "") + (cmd.hasOption("b") ? "native" : "") +  "cql3");
             else
                 r.add("-mode", "thrift");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMode.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMode.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMode.java
index eef8c39..93cfe05 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMode.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMode.java
@@ -54,14 +54,6 @@ public class SettingsMode implements Serializable
             style = opts.usePrepared.setByUser() ? ConnectionStyle.CQL_PREPARED : ConnectionStyle.CQL;
             compression = ProtocolOptions.Compression.NONE.name();
         }
-        else if (options instanceof Cql2Options)
-        {
-            cqlVersion = CqlVersion.CQL2;
-            api = ConnectionAPI.THRIFT;
-            Cql2Options opts = (Cql2Options) options;
-            style = opts.usePrepared.setByUser() ? ConnectionStyle.CQL_PREPARED : ConnectionStyle.CQL;
-            compression = ProtocolOptions.Compression.NONE.name();
-        }
         else if (options instanceof ThriftOptions)
         {
             ThriftOptions opts = (ThriftOptions) options;
@@ -110,18 +102,6 @@ public class SettingsMode implements Serializable
         }
     }
 
-    private static final class Cql2Options extends GroupedOptions
-    {
-        final OptionSimple api = new OptionSimple("cql2", "", null, "", true);
-        final OptionSimple usePrepared = new OptionSimple("prepared", "", null, "", false);
-
-        @Override
-        public List<? extends Option> options()
-        {
-            return Arrays.asList(usePrepared, api);
-        }
-    }
-
     private static final class ThriftOptions extends GroupedOptions
     {
         final OptionSimple api = new OptionSimple("thrift", "", null, "", true);
@@ -146,7 +126,7 @@ public class SettingsMode implements Serializable
             return new SettingsMode(opts);
         }
 
-        GroupedOptions options = GroupedOptions.select(params, new ThriftOptions(), new Cql2Options(), new Cql3Options(), new Cql3SimpleNativeOptions());
+        GroupedOptions options = GroupedOptions.select(params, new ThriftOptions(), new Cql3Options(), new Cql3SimpleNativeOptions());
         if (options == null)
         {
             printHelp();
@@ -158,7 +138,7 @@ public class SettingsMode implements Serializable
 
     public static void printHelp()
     {
-        GroupedOptions.printOptions(System.out, "-mode", new ThriftOptions(), new Cql2Options(), new Cql3Options(), new Cql3SimpleNativeOptions());
+        GroupedOptions.printOptions(System.out, "-mode", new ThriftOptions(), new Cql3Options(), new Cql3SimpleNativeOptions());
     }
 
     public static Runnable helpPrinter()


[2/4] Remove CQL2

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/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
deleted file mode 100644
index 3d5c3f1..0000000
--- a/src/java/org/apache/cassandra/cql/QueryProcessor.java
+++ /dev/null
@@ -1,913 +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.cql;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.*;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.TimeoutException;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.*;
-import org.apache.cassandra.cli.CliUtils;
-import org.apache.cassandra.cql.hooks.ExecutionContext;
-import org.apache.cassandra.cql.hooks.PostPreparationHook;
-import org.apache.cassandra.cql.hooks.PreExecutionHook;
-import org.apache.cassandra.cql.hooks.PreparationContext;
-import org.apache.cassandra.db.CounterCell;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.CellName;
-import org.apache.cassandra.db.composites.CellNameType;
-import org.apache.cassandra.db.composites.Composite;
-import org.apache.cassandra.db.context.CounterContext;
-import org.apache.cassandra.db.filter.*;
-import org.apache.cassandra.db.index.SecondaryIndexManager;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.AsciiType;
-import org.apache.cassandra.db.marshal.TypeParser;
-import org.apache.cassandra.dht.*;
-import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.serializers.MarshalException;
-import org.apache.cassandra.service.StorageProxy;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.thrift.Column;
-import org.apache.cassandra.thrift.CqlMetadata;
-import org.apache.cassandra.thrift.CqlResult;
-import org.apache.cassandra.thrift.CqlResultType;
-import org.apache.cassandra.thrift.CqlRow;
-import org.apache.cassandra.thrift.CqlPreparedResult;
-import org.apache.cassandra.thrift.ThriftValidation;
-import org.apache.cassandra.thrift.ThriftClientState;
-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 org.antlr.runtime.*;
-
-
-import static org.apache.cassandra.thrift.ThriftValidation.validateColumnFamily;
-
-public class QueryProcessor
-{
-    public static final SemanticVersion CQL_VERSION = new SemanticVersion("2.0.0");
-
-    private static final Logger logger = LoggerFactory.getLogger(QueryProcessor.class);
-
-    public static final String DEFAULT_KEY_NAME = CFMetaData.DEFAULT_KEY_ALIAS.toUpperCase();
-
-    private static final List<PreExecutionHook> preExecutionHooks = new CopyOnWriteArrayList<>();
-    private static final List<PostPreparationHook> postPreparationHooks = new CopyOnWriteArrayList<>();
-
-    public static void addPreExecutionHook(PreExecutionHook hook)
-    {
-        preExecutionHooks.add(hook);
-    }
-
-    public static void removePreExecutionHook(PreExecutionHook hook)
-    {
-        preExecutionHooks.remove(hook);
-    }
-
-    public static void addPostPreparationHook(PostPreparationHook hook)
-    {
-        postPreparationHooks.add(hook);
-    }
-
-    public static void removePostPreparationHook(PostPreparationHook hook)
-    {
-        postPreparationHooks.remove(hook);
-    }
-
-    private static List<org.apache.cassandra.db.Row> getSlice(CFMetaData metadata, SelectStatement select, List<ByteBuffer> variables, long now)
-    throws InvalidRequestException, ReadTimeoutException, UnavailableException, IsBootstrappingException
-    {
-        List<ReadCommand> commands = new ArrayList<ReadCommand>();
-
-        // ...of a list of column names
-        if (!select.isColumnRange())
-        {
-            SortedSet<CellName> columnNames = getColumnNames(select, metadata, variables);
-            validateColumnNames(columnNames);
-
-            for (Term rawKey: select.getKeys())
-            {
-                ByteBuffer key = rawKey.getByteBuffer(metadata.getKeyValidator(),variables);
-
-                validateKey(key);
-                commands.add(new SliceByNamesReadCommand(metadata.ksName, key, select.getColumnFamily(), now, new NamesQueryFilter(columnNames)));
-            }
-        }
-        // ...a range (slice) of column names
-        else
-        {
-            AbstractType<?> at = metadata.comparator.asAbstractType();
-            Composite start = metadata.comparator.fromByteBuffer(select.getColumnStart().getByteBuffer(at,variables));
-            Composite finish = metadata.comparator.fromByteBuffer(select.getColumnFinish().getByteBuffer(at,variables));
-
-            for (Term rawKey : select.getKeys())
-            {
-                ByteBuffer key = rawKey.getByteBuffer(metadata.getKeyValidator(),variables);
-
-                validateKey(key);
-                validateSliceFilter(metadata, start, finish, select.isColumnsReversed());
-                commands.add(new SliceFromReadCommand(metadata.ksName,
-                                                      key,
-                                                      select.getColumnFamily(),
-                                                      now,
-                                                      new SliceQueryFilter(start, finish, select.isColumnsReversed(), select.getColumnsLimit())));
-            }
-        }
-
-        return StorageProxy.read(commands, select.getConsistencyLevel());
-    }
-
-    private static SortedSet<CellName> getColumnNames(SelectStatement select, CFMetaData metadata, List<ByteBuffer> variables)
-    throws InvalidRequestException
-    {
-        String keyString = metadata.getCQL2KeyName();
-        List<Term> selectColumnNames = select.getColumnNames();
-        SortedSet<CellName> columnNames = new TreeSet<>(metadata.comparator);
-        for (Term column : selectColumnNames)
-        {
-            // skip the key for the slice op; we'll add it to the resultset in extractThriftColumns
-            if (!column.getText().equalsIgnoreCase(keyString))
-                columnNames.add(metadata.comparator.cellFromByteBuffer(column.getByteBuffer(metadata.comparator.asAbstractType(),variables)));
-        }
-        return columnNames;
-    }
-
-    private static List<org.apache.cassandra.db.Row> multiRangeSlice(CFMetaData metadata, SelectStatement select, List<ByteBuffer> variables, long now)
-    throws ReadTimeoutException, UnavailableException, InvalidRequestException
-    {
-        IPartitioner<?> p = StorageService.getPartitioner();
-
-        AbstractType<?> keyType = Schema.instance.getCFMetaData(metadata.ksName, select.getColumnFamily()).getKeyValidator();
-
-        ByteBuffer startKeyBytes = (select.getKeyStart() != null)
-                                   ? select.getKeyStart().getByteBuffer(keyType,variables)
-                                   : null;
-
-        ByteBuffer finishKeyBytes = (select.getKeyFinish() != null)
-                                    ? select.getKeyFinish().getByteBuffer(keyType,variables)
-                                    : null;
-
-        RowPosition startKey = RowPosition.forKey(startKeyBytes, p), finishKey = RowPosition.forKey(finishKeyBytes, p);
-        if (startKey.compareTo(finishKey) > 0 && !finishKey.isMinimum(p))
-        {
-            if (p instanceof RandomPartitioner)
-                throw new InvalidRequestException("Start key sorts after end key. This is not allowed; you probably should not specify end key at all, under RandomPartitioner");
-            else
-                throw new InvalidRequestException("Start key must sort before (or equal to) finish key in your partitioner!");
-        }
-        AbstractBounds<RowPosition> bounds = new Bounds<RowPosition>(startKey, finishKey);
-
-        IDiskAtomFilter columnFilter = filterFromSelect(select, metadata, variables);
-        validateFilter(metadata, columnFilter);
-
-        List<Relation> columnRelations = select.getColumnRelations();
-        List<IndexExpression> expressions = new ArrayList<IndexExpression>(columnRelations.size());
-        for (Relation columnRelation : columnRelations)
-        {
-            // Left and right side of relational expression encoded according to comparator/validator.
-            ByteBuffer entity = columnRelation.getEntity().getByteBuffer(metadata.comparator.asAbstractType(), variables);
-            ByteBuffer value = columnRelation.getValue().getByteBuffer(metadata.getValueValidatorForFullCellName(entity), variables);
-
-            expressions.add(new IndexExpression(entity,
-                                                IndexExpression.Operator.valueOf(columnRelation.operator().toString()),
-                                                value));
-        }
-
-        int limit = select.isKeyRange() && select.getKeyStart() != null
-                  ? select.getNumRecords() + 1
-                  : select.getNumRecords();
-
-        List<org.apache.cassandra.db.Row> rows = StorageProxy.getRangeSlice(new RangeSliceCommand(metadata.ksName,
-                                                                                                  select.getColumnFamily(),
-                                                                                                  now,
-                                                                                                  columnFilter,
-                                                                                                  bounds,
-                                                                                                  expressions,
-                                                                                                  limit),
-                                                                            select.getConsistencyLevel());
-
-        // if start key was set and relation was "greater than"
-        if (select.getKeyStart() != null && !select.includeStartKey() && !rows.isEmpty())
-        {
-            if (rows.get(0).key.key.equals(startKeyBytes))
-                rows.remove(0);
-        }
-
-        // if finish key was set and relation was "less than"
-        if (select.getKeyFinish() != null && !select.includeFinishKey() && !rows.isEmpty())
-        {
-            int lastIndex = rows.size() - 1;
-            if (rows.get(lastIndex).key.key.equals(finishKeyBytes))
-                rows.remove(lastIndex);
-        }
-
-        return rows.subList(0, select.getNumRecords() < rows.size() ? select.getNumRecords() : rows.size());
-    }
-
-    private static IDiskAtomFilter filterFromSelect(SelectStatement select, CFMetaData metadata, List<ByteBuffer> variables)
-    throws InvalidRequestException
-    {
-        if (select.isColumnRange() || select.getColumnNames().size() == 0)
-        {
-            AbstractType<?> comparator = metadata.comparator.asAbstractType();
-            return new SliceQueryFilter(metadata.comparator.fromByteBuffer(select.getColumnStart().getByteBuffer(comparator, variables)),
-                                        metadata.comparator.fromByteBuffer(select.getColumnFinish().getByteBuffer(comparator, variables)),
-                                        select.isColumnsReversed(),
-                                        select.getColumnsLimit());
-        }
-        else
-        {
-            return new NamesQueryFilter(getColumnNames(select, metadata, variables));
-        }
-    }
-
-    /* Test for SELECT-specific taboos */
-    private static void validateSelect(String keyspace, SelectStatement select, List<ByteBuffer> variables) throws InvalidRequestException
-    {
-        select.getConsistencyLevel().validateForRead(keyspace);
-
-        // Finish key w/o start key (KEY < foo)
-        if (!select.isKeyRange() && (select.getKeyFinish() != null))
-            throw new InvalidRequestException("Key range clauses must include a start key (i.e. KEY > term)");
-
-        // Key range and by-key(s) combined (KEY > foo AND KEY = bar)
-        if (select.isKeyRange() && select.getKeys().size() > 0)
-            throw new InvalidRequestException("You cannot combine key range and by-key clauses in a SELECT");
-
-        // Start and finish keys, *and* column relations (KEY > foo AND KEY < bar and name1 = value1).
-        if (select.isKeyRange() && (select.getKeyFinish() != null) && (select.getColumnRelations().size() > 0))
-            throw new InvalidRequestException("You cannot combine key range and by-column clauses in a SELECT");
-
-        // Can't use more than one KEY =
-        if (!select.isMultiKey() && select.getKeys().size() > 1)
-            throw new InvalidRequestException("You cannot use more than one KEY = in a SELECT");
-
-        if (select.getColumnRelations().size() > 0)
-        {
-            ColumnFamilyStore cfstore = Keyspace.open(keyspace).getColumnFamilyStore(select.getColumnFamily());
-            CellNameType comparator = cfstore.metadata.comparator;
-            AbstractType<?> at = comparator.asAbstractType();
-            SecondaryIndexManager idxManager = cfstore.indexManager;
-            for (Relation relation : select.getColumnRelations())
-            {
-                ByteBuffer name = relation.getEntity().getByteBuffer(at, variables);
-                if ((relation.operator() == RelationType.EQ) && idxManager.indexes(comparator.cellFromByteBuffer(name)))
-                    return;
-            }
-            throw new InvalidRequestException("No indexed columns present in by-columns clause with \"equals\" operator");
-        }
-    }
-
-    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 validateKeyAlias(CFMetaData cfm, String key) throws InvalidRequestException
-    {
-        assert key.toUpperCase().equals(key); // should always be uppercased by caller
-        String realKeyAlias = cfm.getCQL2KeyName().toUpperCase();
-        if (!realKeyAlias.equals(key))
-            throw new InvalidRequestException(String.format("Expected key '%s' to be present in WHERE clause for '%s'", realKeyAlias, cfm.cfName));
-    }
-
-    private static void validateColumnNames(Iterable<CellName> columns)
-    throws InvalidRequestException
-    {
-        for (CellName name : columns)
-        {
-            if (name.dataSize() > org.apache.cassandra.db.Cell.MAX_NAME_LENGTH)
-                throw new InvalidRequestException(String.format("column name is too long (%s > %s)",
-                                                                name.dataSize(),
-                                                                org.apache.cassandra.db.Cell.MAX_NAME_LENGTH));
-            if (name.isEmpty())
-                throw new InvalidRequestException("zero-length column name");
-        }
-    }
-
-    public static void validateColumnName(CellName column)
-    throws InvalidRequestException
-    {
-        validateColumnNames(Arrays.asList(column));
-    }
-
-    public static void validateColumn(CFMetaData metadata, CellName name, ByteBuffer value)
-    throws InvalidRequestException
-    {
-        validateColumnName(name);
-        AbstractType<?> validator = metadata.getValueValidator(name);
-
-        try
-        {
-            if (validator != null)
-                validator.validate(value);
-        }
-        catch (MarshalException me)
-        {
-            throw new InvalidRequestException(String.format("Invalid column value for column (name=%s); %s",
-                                                            ByteBufferUtil.bytesToHex(name.toByteBuffer()),
-                                                            me.getMessage()));
-        }
-    }
-
-    private static void validateFilter(CFMetaData metadata, IDiskAtomFilter filter)
-    throws InvalidRequestException
-    {
-        if (filter instanceof SliceQueryFilter)
-            validateSliceFilter(metadata, (SliceQueryFilter)filter);
-        else
-            validateColumnNames(((NamesQueryFilter)filter).columns);
-    }
-
-    private static void validateSliceFilter(CFMetaData metadata, SliceQueryFilter range)
-    throws InvalidRequestException
-    {
-        validateSliceFilter(metadata, range.start(), range.finish(), range.reversed);
-    }
-
-    private static void validateSliceFilter(CFMetaData metadata, Composite start, Composite finish, boolean reversed)
-    throws InvalidRequestException
-    {
-        CellNameType comparator = metadata.comparator;
-        Comparator<Composite> orderedComparator = reversed ? comparator.reverseComparator(): comparator;
-        if (!start.isEmpty() && !finish.isEmpty() && orderedComparator.compare(start, finish) > 0)
-            throw new InvalidRequestException("range finish must come after start in traversal order");
-    }
-
-    public static CqlResult processStatement(CQLStatement statement, ExecutionContext context)
-    throws RequestExecutionException, RequestValidationException
-    {
-        String keyspace = null;
-        ThriftClientState clientState = context.clientState;
-        List<ByteBuffer> variables = context.variables;
-
-        // Some statements won't have (or don't need) a keyspace (think USE, or CREATE).
-        if (statement.type != StatementType.SELECT && StatementType.REQUIRES_KEYSPACE.contains(statement.type))
-            keyspace = clientState.getKeyspace();
-
-        CqlResult result = new CqlResult();
-
-        if (!preExecutionHooks.isEmpty())
-            for (PreExecutionHook hook : preExecutionHooks)
-                statement = hook.processStatement(statement, context);
-
-        if (logger.isDebugEnabled()) logger.debug("CQL statement type: {}", statement.type.toString());
-        CFMetaData metadata;
-        switch (statement.type)
-        {
-            case SELECT:
-                SelectStatement select = (SelectStatement)statement.statement;
-
-                final String oldKeyspace = clientState.getRawKeyspace();
-
-                if (select.isSetKeyspace())
-                {
-                    keyspace = CliUtils.unescapeSQLString(select.getKeyspace());
-                    ThriftValidation.validateKeyspace(keyspace);
-                }
-                else if (oldKeyspace == null)
-                    throw new InvalidRequestException("no keyspace has been specified");
-                else
-                    keyspace = oldKeyspace;
-
-                clientState.hasColumnFamilyAccess(keyspace, select.getColumnFamily(), Permission.SELECT);
-                metadata = validateColumnFamily(keyspace, select.getColumnFamily());
-
-                // need to do this in here because we need a CFMD.getKeyName()
-                select.extractKeyAliasFromColumns(metadata);
-
-                if (select.getKeys().size() > 0)
-                    validateKeyAlias(metadata, select.getKeyAlias());
-
-                validateSelect(keyspace, select, variables);
-
-                List<org.apache.cassandra.db.Row> rows;
-
-                long now = System.currentTimeMillis();
-                // By-key
-                if (!select.isKeyRange() && (select.getKeys().size() > 0))
-                {
-                    rows = getSlice(metadata, select, variables, now);
-                }
-                else
-                {
-                    rows = multiRangeSlice(metadata, select, variables, now);
-                }
-
-                // count resultset is a single column named "count"
-                result.type = CqlResultType.ROWS;
-                if (select.isCountOperation())
-                {
-                    validateCountOperation(select);
-
-                    ByteBuffer countBytes = ByteBufferUtil.bytes("count");
-                    result.schema = new CqlMetadata(Collections.<ByteBuffer, String>emptyMap(),
-                                                    Collections.<ByteBuffer, String>emptyMap(),
-                                                    "AsciiType",
-                                                    "LongType");
-                    List<Column> columns = Collections.singletonList(new Column(countBytes).setValue(ByteBufferUtil.bytes((long) rows.size())));
-                    result.rows = Collections.singletonList(new CqlRow(countBytes, columns));
-                    return result;
-                }
-
-                // otherwise create resultset from query results
-                result.schema = new CqlMetadata(new HashMap<ByteBuffer, String>(),
-                                                new HashMap<ByteBuffer, String>(),
-                                                TypeParser.getShortName(metadata.comparator.asAbstractType()),
-                                                TypeParser.getShortName(metadata.getDefaultValidator()));
-                List<CqlRow> cqlRows = new ArrayList<CqlRow>(rows.size());
-                for (org.apache.cassandra.db.Row row : rows)
-                {
-                    List<Column> thriftColumns = new ArrayList<Column>();
-                    if (select.isColumnRange())
-                    {
-                        if (select.isFullWildcard())
-                        {
-                            // prepend key
-                            ByteBuffer keyName = ByteBufferUtil.bytes(metadata.getCQL2KeyName());
-                            thriftColumns.add(new Column(keyName).setValue(row.key.key).setTimestamp(-1));
-                            result.schema.name_types.put(keyName, TypeParser.getShortName(AsciiType.instance));
-                            result.schema.value_types.put(keyName, TypeParser.getShortName(metadata.getKeyValidator()));
-                        }
-
-                        // preserve comparator order
-                        if (row.cf != null)
-                        {
-                            for (org.apache.cassandra.db.Cell c : row.cf.getSortedColumns())
-                            {
-                                if (c.isMarkedForDelete(now))
-                                    continue;
-
-                                ColumnDefinition cd = metadata.getColumnDefinition(c.name());
-                                if (cd != null)
-                                    result.schema.value_types.put(c.name().toByteBuffer(), TypeParser.getShortName(cd.type));
-
-                                thriftColumns.add(thriftify(c));
-                            }
-                        }
-                    }
-                    else
-                    {
-                        String keyString = metadata.getCQL2KeyName();
-
-                        // order columns in the order they were asked for
-                        for (Term term : select.getColumnNames())
-                        {
-                            if (term.getText().equalsIgnoreCase(keyString))
-                            {
-                                // preserve case of key as it was requested
-                                ByteBuffer requestedKey = ByteBufferUtil.bytes(term.getText());
-                                thriftColumns.add(new Column(requestedKey).setValue(row.key.key).setTimestamp(-1));
-                                result.schema.name_types.put(requestedKey, TypeParser.getShortName(AsciiType.instance));
-                                result.schema.value_types.put(requestedKey, TypeParser.getShortName(metadata.getKeyValidator()));
-                                continue;
-                            }
-
-                            if (row.cf == null)
-                                continue;
-
-                            ByteBuffer nameBytes;
-                            try
-                            {
-                                nameBytes = term.getByteBuffer(metadata.comparator.asAbstractType(), variables);
-                            }
-                            catch (InvalidRequestException e)
-                            {
-                                throw new AssertionError(e);
-                            }
-
-                            CellName name = metadata.comparator.cellFromByteBuffer(nameBytes);
-                            ColumnDefinition cd = metadata.getColumnDefinition(name);
-                            if (cd != null)
-                                result.schema.value_types.put(nameBytes, TypeParser.getShortName(cd.type));
-                            org.apache.cassandra.db.Cell c = row.cf.getColumn(name);
-                            if (c == null || c.isMarkedForDelete(System.currentTimeMillis()))
-                                thriftColumns.add(new Column().setName(nameBytes));
-                            else
-                                thriftColumns.add(thriftify(c));
-                        }
-                    }
-
-                    // Create a new row, add the columns to it, and then add it to the list of rows
-                    CqlRow cqlRow = new CqlRow();
-                    cqlRow.key = row.key.key;
-                    cqlRow.columns = thriftColumns;
-                    if (select.isColumnsReversed())
-                        Collections.reverse(cqlRow.columns);
-                    cqlRows.add(cqlRow);
-                }
-
-                result.rows = cqlRows;
-                return result;
-
-            case INSERT: // insert uses UpdateStatement
-            case UPDATE:
-                UpdateStatement update = (UpdateStatement)statement.statement;
-                update.getConsistencyLevel().validateForWrite(keyspace);
-
-                keyspace = update.keyspace == null ? clientState.getKeyspace() : update.keyspace;
-                // permission is checked in prepareRowMutations()
-                List<IMutation> rowMutations = update.prepareRowMutations(keyspace, clientState, variables);
-
-                for (IMutation mutation : rowMutations)
-                {
-                    validateKey(mutation.key());
-                }
-
-                StorageProxy.mutateWithTriggers(rowMutations, update.getConsistencyLevel(), false);
-
-                result.type = CqlResultType.VOID;
-                return result;
-
-            case BATCH:
-                BatchStatement batch = (BatchStatement) statement.statement;
-                batch.getConsistencyLevel().validateForWrite(keyspace);
-
-                if (batch.getTimeToLive() != 0)
-                    throw new InvalidRequestException("Global TTL on the BATCH statement is not supported.");
-
-                for (AbstractModification up : batch.getStatements())
-                {
-                    if (up.isSetConsistencyLevel())
-                        throw new InvalidRequestException(
-                                "Consistency level must be set on the BATCH, not individual statements");
-
-                    if (batch.isSetTimestamp() && up.isSetTimestamp())
-                        throw new InvalidRequestException(
-                                "Timestamp must be set either on BATCH or individual statements");
-                }
-
-                List<IMutation> mutations = batch.getMutations(keyspace, clientState, variables);
-                for (IMutation mutation : mutations)
-                {
-                    validateKey(mutation.key());
-                }
-
-                StorageProxy.mutateWithTriggers(mutations, batch.getConsistencyLevel(), false);
-
-                result.type = CqlResultType.VOID;
-                return result;
-
-            case USE:
-                clientState.validateLogin();
-                clientState.setKeyspace(CliUtils.unescapeSQLString((String) statement.statement));
-
-                result.type = CqlResultType.VOID;
-                return result;
-
-            case TRUNCATE:
-                Pair<String, String> columnFamily = (Pair<String, String>)statement.statement;
-                keyspace = columnFamily.left == null ? clientState.getKeyspace() : columnFamily.left;
-
-                validateColumnFamily(keyspace, columnFamily.right);
-                clientState.hasColumnFamilyAccess(keyspace, columnFamily.right, Permission.MODIFY);
-
-                try
-                {
-                    StorageProxy.truncateBlocking(keyspace, columnFamily.right);
-                }
-                catch (TimeoutException e)
-                {
-                    throw new TruncateException(e);
-                }
-                catch (IOException e)
-                {
-                    throw new RuntimeException(e);
-                }
-
-                result.type = CqlResultType.VOID;
-                return result;
-
-            case DELETE:
-                DeleteStatement delete = (DeleteStatement)statement.statement;
-
-                keyspace = delete.keyspace == null ? clientState.getKeyspace() : delete.keyspace;
-                // permission is checked in prepareRowMutations()
-                List<IMutation> deletions = delete.prepareRowMutations(keyspace, clientState, variables);
-                for (IMutation deletion : deletions)
-                {
-                    validateKey(deletion.key());
-                }
-
-                StorageProxy.mutateWithTriggers(deletions, delete.getConsistencyLevel(), false);
-
-                result.type = CqlResultType.VOID;
-                return result;
-
-            case CREATE_KEYSPACE:
-                CreateKeyspaceStatement create = (CreateKeyspaceStatement)statement.statement;
-                create.validate();
-                ThriftValidation.validateKeyspaceNotSystem(create.getName());
-                clientState.hasAllKeyspacesAccess(Permission.CREATE);
-
-                try
-                {
-                    KSMetaData ksm = KSMetaData.newKeyspace(create.getName(),
-                                                            create.getStrategyClass(),
-                                                            create.getStrategyOptions(),
-                                                            true);
-                    ThriftValidation.validateKeyspaceNotYetExisting(ksm.name);
-                    MigrationManager.announceNewKeyspace(ksm);
-                }
-                catch (ConfigurationException e)
-                {
-                    InvalidRequestException ex = new InvalidRequestException(e.getMessage());
-                    ex.initCause(e);
-                    throw ex;
-                }
-
-                result.type = CqlResultType.VOID;
-                return result;
-
-            case CREATE_COLUMNFAMILY:
-                CreateColumnFamilyStatement createCf = (CreateColumnFamilyStatement)statement.statement;
-                clientState.hasKeyspaceAccess(keyspace, Permission.CREATE);
-
-                try
-                {
-                    MigrationManager.announceNewColumnFamily(createCf.getCFMetaData(keyspace, variables));
-                }
-                catch (ConfigurationException e)
-                {
-                    InvalidRequestException ex = new InvalidRequestException(e.toString());
-                    ex.initCause(e);
-                    throw ex;
-                }
-
-                result.type = CqlResultType.VOID;
-                return result;
-
-            case CREATE_INDEX:
-                CreateIndexStatement createIdx = (CreateIndexStatement)statement.statement;
-                clientState.hasColumnFamilyAccess(keyspace, createIdx.getColumnFamily(), Permission.ALTER);
-                CFMetaData oldCfm = Schema.instance.getCFMetaData(keyspace, createIdx.getColumnFamily());
-                if (oldCfm == null)
-                    throw new InvalidRequestException("No such column family: " + createIdx.getColumnFamily());
-
-                boolean columnExists = false;
-                ByteBuffer columnName = createIdx.getColumnName().getByteBuffer();
-                // mutating oldCfm directly would be bad, but mutating a copy is fine.
-                CFMetaData cfm = oldCfm.clone();
-                for (ColumnDefinition cd : cfm.regularColumns())
-                {
-                    if (cd.name.bytes.equals(columnName))
-                    {
-                        if (cd.getIndexType() != null)
-                            throw new InvalidRequestException("Index already exists");
-                        if (logger.isDebugEnabled())
-                            logger.debug("Updating column {} definition for index {}", cfm.comparator.getString(cfm.comparator.fromByteBuffer(columnName)), createIdx.getIndexName());
-                        cd.setIndexType(IndexType.KEYS, Collections.<String, String>emptyMap());
-                        cd.setIndexName(createIdx.getIndexName());
-                        columnExists = true;
-                        break;
-                    }
-                }
-                if (!columnExists)
-                    throw new InvalidRequestException("No column definition found for column " + oldCfm.comparator.getString(cfm.comparator.fromByteBuffer(columnName)));
-
-                try
-                {
-                    cfm.addDefaultIndexNames();
-                    MigrationManager.announceColumnFamilyUpdate(cfm, true); // As far as metadata are concerned, CQL2 == thrift
-                }
-                catch (ConfigurationException e)
-                {
-                    InvalidRequestException ex = new InvalidRequestException(e.toString());
-                    ex.initCause(e);
-                    throw ex;
-                }
-
-                result.type = CqlResultType.VOID;
-                return result;
-
-            case DROP_INDEX:
-                DropIndexStatement dropIdx = (DropIndexStatement)statement.statement;
-                keyspace = clientState.getKeyspace();
-                dropIdx.setKeyspace(keyspace);
-                clientState.hasColumnFamilyAccess(keyspace, dropIdx.getColumnFamily(), Permission.ALTER);
-
-                try
-                {
-                    CFMetaData updatedCF = dropIdx.generateCFMetadataUpdate();
-                    MigrationManager.announceColumnFamilyUpdate(updatedCF, true); // As far as metadata are concerned, CQL2 == thrift
-                }
-                catch (ConfigurationException e)
-                {
-                    InvalidRequestException ex = new InvalidRequestException(e.toString());
-                    ex.initCause(e);
-                    throw ex;
-                }
-
-                result.type = CqlResultType.VOID;
-                return result;
-
-            case DROP_KEYSPACE:
-                String deleteKeyspace = (String)statement.statement;
-                ThriftValidation.validateKeyspaceNotSystem(deleteKeyspace);
-                clientState.hasKeyspaceAccess(deleteKeyspace, Permission.DROP);
-
-                try
-                {
-                    MigrationManager.announceKeyspaceDrop(deleteKeyspace);
-                }
-                catch (ConfigurationException e)
-                {
-                    InvalidRequestException ex = new InvalidRequestException(e.getMessage());
-                    ex.initCause(e);
-                    throw ex;
-                }
-
-                result.type = CqlResultType.VOID;
-                return result;
-
-            case DROP_COLUMNFAMILY:
-                String deleteColumnFamily = (String)statement.statement;
-                clientState.hasColumnFamilyAccess(keyspace, deleteColumnFamily, Permission.DROP);
-
-                try
-                {
-                    MigrationManager.announceColumnFamilyDrop(keyspace, deleteColumnFamily);
-                }
-                catch (ConfigurationException e)
-                {
-                    InvalidRequestException ex = new InvalidRequestException(e.getMessage());
-                    ex.initCause(e);
-                    throw ex;
-                }
-
-                result.type = CqlResultType.VOID;
-                return result;
-
-            case ALTER_TABLE:
-                AlterTableStatement alterTable = (AlterTableStatement) statement.statement;
-
-                validateColumnFamily(keyspace, alterTable.columnFamily);
-                clientState.hasColumnFamilyAccess(keyspace, alterTable.columnFamily, Permission.ALTER);
-
-                try
-                {
-                    MigrationManager.announceColumnFamilyUpdate(alterTable.getCFMetaData(keyspace), true); // As far as metadata are concerned, CQL2 == thrift
-                }
-                catch (ConfigurationException e)
-                {
-                    InvalidRequestException ex = new InvalidRequestException(e.getMessage());
-                    ex.initCause(e);
-                    throw ex;
-                }
-
-                result.type = CqlResultType.VOID;
-                return result;
-        }
-        return null;    // We should never get here.
-    }
-
-    public static CqlResult process(String queryString, ThriftClientState clientState)
-    throws RequestValidationException, RequestExecutionException
-    {
-        logger.trace("CQL QUERY: {}", queryString);
-        return processStatement(getStatement(queryString),
-                                new ExecutionContext(clientState, queryString, Collections.<ByteBuffer>emptyList()));
-    }
-
-    public static CqlPreparedResult prepare(String queryString, ThriftClientState clientState)
-    throws RequestValidationException
-    {
-        logger.trace("CQL QUERY: {}", queryString);
-
-        CQLStatement statement = getStatement(queryString);
-        int statementId = makeStatementId(queryString);
-        logger.trace("Discovered "+ statement.boundTerms + " bound variables.");
-
-        clientState.getPrepared().put(statementId, statement);
-        logger.trace(String.format("Stored prepared statement #%d with %d bind markers",
-                                   statementId,
-                                   statement.boundTerms));
-
-        if (!postPreparationHooks.isEmpty())
-        {
-            PreparationContext context = new PreparationContext(clientState, queryString, statement);
-            for (PostPreparationHook hook : postPreparationHooks)
-                hook.processStatement(statement, context);
-        }
-
-        return new CqlPreparedResult(statementId, statement.boundTerms);
-    }
-
-    public static CqlResult processPrepared(CQLStatement statement, ThriftClientState clientState, List<ByteBuffer> variables)
-    throws RequestValidationException, RequestExecutionException
-    {
-        // Check to see if there are any bound variables to verify
-        if (!(variables.isEmpty() && (statement.boundTerms == 0)))
-        {
-            if (variables.size() != statement.boundTerms)
-                throw new InvalidRequestException(String.format("there were %d markers(?) in CQL but %d bound variables",
-                                                                statement.boundTerms,
-                                                                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, new ExecutionContext(clientState, null, variables));
-    }
-
-    private static final int makeStatementId(String cql)
-    {
-        // use the hash of the string till something better is provided
-        return cql.hashCode();
-    }
-
-    private static Column thriftify(org.apache.cassandra.db.Cell c)
-    {
-        ByteBuffer value = (c instanceof CounterCell)
-                           ? ByteBufferUtil.bytes(CounterContext.instance().total(c.value()))
-                           : c.value();
-        return new Column(c.name().toByteBuffer()).setValue(value).setTimestamp(c.timestamp());
-    }
-
-    private static CQLStatement getStatement(String queryStr) throws SyntaxException
-    {
-        try
-        {
-            // 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;
-        }
-        catch (RuntimeException re)
-        {
-            SyntaxException ire = new SyntaxException("Failed parsing statement: [" + queryStr + "] reason: " + re.getClass().getSimpleName() + " " + re.getMessage());
-            throw ire;
-        }
-        catch (RecognitionException e)
-        {
-            SyntaxException ire = new SyntaxException("Invalid or malformed CQL query string: " + e.getMessage());
-            throw ire;
-        }
-    }
-
-    private static void validateCountOperation(SelectStatement select) throws InvalidRequestException
-    {
-        if (select.isWildcard())
-            return; // valid count(*)
-
-        if (!select.isColumnRange())
-        {
-            List<Term> columnNames = select.getColumnNames();
-            String firstColumn = columnNames.get(0).getText();
-
-            if (columnNames.size() == 1 && (firstColumn.equals("*") || firstColumn.equals("1")))
-                return; // valid count(*) || count(1)
-        }
-
-        throw new InvalidRequestException("Only COUNT(*) and COUNT(1) operations are currently supported.");
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/cql/Relation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/Relation.java b/src/java/org/apache/cassandra/cql/Relation.java
deleted file mode 100644
index 669c5f2..0000000
--- a/src/java/org/apache/cassandra/cql/Relation.java
+++ /dev/null
@@ -1,86 +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.cql;
-
-/**
- * Relations encapsulate the relationship between an entity of some kind, and
- * a value (term). For example, <key> > "start" or "colname1" = "somevalue".
- *
- */
-public class Relation
-{
-    private final Term entity;
-    private final RelationType relationType;
-    private final Term value;
-
-    /**
-     * Creates a new relation.
-     *
-     * @param entity the kind of relation this is; what the term is being compared to.
-     * @param type the type that describes how this entity relates to the value.
-     * @param value the value being compared.
-     */
-    public Relation(Term entity, String type, Term value)
-    {
-        this.entity = entity;
-        this.relationType = RelationType.forString(type);
-        this.value = value;
-    }
-
-    public RelationType operator()
-    {
-        return relationType;
-    }
-
-    public Term getEntity()
-    {
-        return entity;
-    }
-
-    public Term getValue()
-    {
-        return value;
-    }
-
-    @Override
-    public String toString()
-    {
-        return String.format("Relation(%s, %s, %s)", entity, relationType, value);
-    }
-}
-
-enum RelationType
-{
-    EQ, LT, LTE, GTE, GT;
-
-    public static RelationType forString(String s)
-    {
-        if (s.equals("="))
-            return EQ;
-        else if (s.equals("<"))
-            return LT;
-        else if (s.equals("<="))
-            return LTE;
-        else if (s.equals(">="))
-            return GTE;
-        else if (s.equals(">"))
-            return GT;
-
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/cql/SelectExpression.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/SelectExpression.java b/src/java/org/apache/cassandra/cql/SelectExpression.java
deleted file mode 100644
index 4d3cb2f..0000000
--- a/src/java/org/apache/cassandra/cql/SelectExpression.java
+++ /dev/null
@@ -1,164 +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.cql;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Select expressions are analogous to the projection in a SQL query. They
- * determine which columns will appear in the result set.  SelectExpression
- * instances encapsulate a parsed expression from a <code>SELECT</code>
- * statement.
- *
- * See: doc/cql/CQL.html#SpecifyingColumns
- */
-public class SelectExpression
-{
-    public static final int MAX_COLUMNS_DEFAULT = 10000;
-
-    private int numColumns = MAX_COLUMNS_DEFAULT;
-    private boolean reverseColumns = false;
-    private final boolean hasFirstSet;
-    private final boolean wildcard;
-    private final Term start, finish;
-    private final List<Term> columns;
-
-    /**
-     * Create a new SelectExpression for a range (slice) of columns.
-     *
-     * @param start the starting column name
-     * @param finish the finishing column name
-     * @param count the number of columns to limit the results to
-     * @param reverse true to reverse column order
-     * @param wildcard determines weather this statement is wildcard
-     * @param firstSet determines weather "FIRST" keyword was set
-     */
-    public SelectExpression(Term start, Term finish, int count, boolean reverse, boolean wildcard, boolean firstSet)
-    {
-        this.start = start;
-        this.finish = finish;
-        numColumns = count;
-        reverseColumns = reverse;
-        this.wildcard = wildcard;
-        hasFirstSet = firstSet;
-        this.columns = null;
-    }
-
-    /**
-     * Create a new SelectExpression for a list of columns.
-     *
-     * @param first the first (possibly only) column name to select on.
-     * @param count the number of columns to limit the results on
-     * @param reverse true to reverse column order
-     * @param firstSet determines weather "FIRST" keyword was set
-     */
-    public SelectExpression(Term first, int count, boolean reverse, boolean firstSet)
-    {
-        wildcard = false;
-        columns = new ArrayList<Term>();
-        columns.add(first);
-        numColumns = count;
-        reverseColumns = reverse;
-        hasFirstSet = firstSet;
-        start = null;
-        finish = null;
-    }
-
-    /**
-     * Add an additional column name to a SelectExpression.
-     *
-     * @param addTerm
-     */
-    public void and(Term addTerm)
-    {
-        assert !isColumnRange();    // Not possible when invoked by parser
-        columns.add(addTerm);
-    }
-
-    public boolean isColumnRange()
-    {
-        return (start != null);
-    }
-
-    public boolean isColumnList()
-    {
-        return !isColumnRange();
-    }
-    public int getColumnsLimit()
-    {
-        return numColumns;
-    }
-
-    public boolean isColumnsReversed()
-    {
-        return reverseColumns;
-    }
-
-    public void setColumnsReversed(boolean reversed)
-    {
-        reverseColumns = reversed;
-    }
-
-    public void setColumnsLimit(int limit)
-    {
-        numColumns = limit;
-    }
-
-    /**
-     * @return weather expression includes "FIRST" keyword
-     */
-    public boolean hasFirstSet()
-    {
-        return hasFirstSet;
-    }
-
-    public Term getStart()
-    {
-        return start;
-    }
-
-    public Term getFinish()
-    {
-        return finish;
-    }
-
-    public List<Term> getColumns()
-    {
-        return columns;
-    }
-
-    public boolean isWildcard()
-    {
-        return wildcard;
-    }
-
-    public String toString()
-    {
-        return String.format("SelectExpression [numColumns=%s, reverseColumns=%s, hasFirstSet=%s, wildcard=%s, start=%s, finish=%s, columns=%s]",
-                             numColumns,
-                             reverseColumns,
-                             hasFirstSet,
-                             wildcard,
-                             start,
-                             finish,
-                             columns);
-    }
-
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/cql/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/SelectStatement.java b/src/java/org/apache/cassandra/cql/SelectStatement.java
deleted file mode 100644
index b4ccd56..0000000
--- a/src/java/org/apache/cassandra/cql/SelectStatement.java
+++ /dev/null
@@ -1,188 +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.cql;
-
-import java.util.List;
-import java.util.Set;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.db.ConsistencyLevel;
-
-/**
- * Encapsulates a completely parsed SELECT query, including the target
- * column family, expression, result count, and ordering clause.
- *
- */
-public class SelectStatement
-{
-    private final SelectExpression expression;
-    private final boolean isCountOper;
-    private final String columnFamily;
-    private final String keyspace;
-    private final ConsistencyLevel cLevel;
-    private final WhereClause clause;
-    private final int numRecords;
-
-    public SelectStatement(SelectExpression expression, boolean isCountOper, String keyspace, String columnFamily,
-            ConsistencyLevel cLevel, WhereClause clause, int numRecords)
-    {
-        this.expression = expression;
-        this.isCountOper = isCountOper;
-        this.keyspace = keyspace;
-        this.columnFamily = columnFamily;
-        this.cLevel = cLevel;
-        this.clause = (clause != null) ? clause : new WhereClause();
-        this.numRecords = numRecords;
-    }
-
-    public boolean isKeyRange()
-    {
-        return clause.isKeyRange();
-    }
-
-    public Set<Term> getKeys()
-    {
-        return clause.getKeys();
-    }
-
-    public Term getKeyStart()
-    {
-        return clause.getStartKey();
-    }
-
-    public Term getKeyFinish()
-    {
-        return clause.getFinishKey();
-    }
-
-    public List<Relation> getColumnRelations()
-    {
-        return clause.getColumnRelations();
-    }
-
-    public boolean isColumnRange()
-    {
-        return expression.isColumnRange();
-    }
-
-    public boolean isWildcard()
-    {
-        return expression.isWildcard();
-    }
-    public boolean isFullWildcard()
-    {
-        return expression.isWildcard() && !expression.hasFirstSet();
-    }
-
-    public List<Term> getColumnNames()
-    {
-        return expression.getColumns();
-    }
-
-    public Term getColumnStart()
-    {
-        return expression.getStart();
-    }
-
-    public Term getColumnFinish()
-    {
-        return expression.getFinish();
-    }
-
-    public boolean isSetKeyspace()
-    {
-        return keyspace != null;
-    }
-
-    public String getKeyspace()
-    {
-        return keyspace;
-    }
-
-    public String getColumnFamily()
-    {
-        return columnFamily;
-    }
-
-    public boolean isColumnsReversed()
-    {
-        return expression.isColumnsReversed();
-    }
-
-    public ConsistencyLevel getConsistencyLevel()
-    {
-        return cLevel;
-    }
-
-    public int getNumRecords()
-    {
-        return numRecords;
-    }
-
-    public int getColumnsLimit()
-    {
-        return expression.getColumnsLimit();
-    }
-
-    public boolean isCountOperation()
-    {
-        return isCountOper;
-    }
-
-    public boolean includeStartKey()
-    {
-        return clause.includeStartKey();
-    }
-
-    public boolean includeFinishKey()
-    {
-        return clause.includeFinishKey();
-    }
-
-    public String getKeyAlias()
-    {
-        return clause.getKeyAlias();
-    }
-
-    public boolean isMultiKey()
-    {
-        return clause.isMultiKey();
-    }
-
-    public void extractKeyAliasFromColumns(CFMetaData cfm)
-    {
-        clause.extractKeysFromColumns(cfm);
-    }
-
-    public List<Relation> getClauseRelations()
-    {
-        return clause.getClauseRelations();
-    }
-
-    public String toString()
-    {
-        return String.format("SelectStatement [expression=%s, isCountOper=%s, columnFamily=%s, keyspace=%s, cLevel=%s, clause=%s, numRecords=%s]",
-                             expression,
-                             isCountOper,
-                             columnFamily,
-                             keyspace,
-                             cLevel,
-                             clause,
-                             numRecords);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/cql/StatementType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/StatementType.java b/src/java/org/apache/cassandra/cql/StatementType.java
deleted file mode 100644
index 94db6a3..0000000
--- a/src/java/org/apache/cassandra/cql/StatementType.java
+++ /dev/null
@@ -1,32 +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.cql;
-
-import java.util.EnumSet;
-
-public enum StatementType
-{
-    SELECT, INSERT, UPDATE, BATCH, USE, TRUNCATE, DELETE, CREATE_KEYSPACE, CREATE_COLUMNFAMILY, CREATE_INDEX, DROP_INDEX,
-        DROP_KEYSPACE, DROP_COLUMNFAMILY, ALTER_TABLE;
-
-    /** Statement types that don't require a keyspace to be set */
-    private static final EnumSet<StatementType> TOP_LEVEL = EnumSet.of(USE, CREATE_KEYSPACE, DROP_KEYSPACE);
-
-    /** Statement types that require a keyspace to be set */
-    public static final EnumSet<StatementType> REQUIRES_KEYSPACE = EnumSet.complementOf(TOP_LEVEL);
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/cql/Term.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/Term.java b/src/java/org/apache/cassandra/cql/Term.java
deleted file mode 100644
index 6e47bfa..0000000
--- a/src/java/org/apache/cassandra/cql/Term.java
+++ /dev/null
@@ -1,208 +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.cql;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.AsciiType;
-import org.apache.cassandra.db.marshal.FloatType;
-import org.apache.cassandra.db.marshal.IntegerType;
-import org.apache.cassandra.db.marshal.LexicalUUIDType;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.serializers.MarshalException;
-
-/** A term parsed from a CQL statement. */
-public class Term
-{
-    private final String text;
-    private final TermType type;
-    private Integer bindIndex = -1;
-
-    public Term(String text, TermType type)
-    {
-        this.text = text == null ? "" : text;
-        this.type = type;
-    }
-
-    /**
-     * Create new Term instance from a string, and an integer that corresponds
-     * with the token ID from CQLParser.
-     *
-     * @param text the text representation of the term.
-     * @param type the term's type as an integer token ID.
-     */
-    public Term(String text, int type)
-    {
-        this(text == null ? "" : text, TermType.forInt(type));
-    }
-
-    public Term(long value, TermType type)
-    {
-        this(String.valueOf(value), type);
-    }
-
-    protected Term()
-    {
-        this("", TermType.STRING);
-    }
-
-    public Term(String text, int type, int index)
-    {
-        this(text, type);
-        this.bindIndex = index;
-    }
-
-    /**
-     * Returns the text parsed to create this term.
-     *
-     * @return the string term acquired from a CQL statement.
-     */
-    public String getText()
-    {
-        return text;
-    }
-
-    /**
-     * Returns the typed value, serialized to a ByteBuffer according to a
-     * comparator/validator.
-     *
-     * @return a ByteBuffer of the value.
-     * @throws InvalidRequestException if unable to coerce the string to its type.
-     */
-    public ByteBuffer getByteBuffer(AbstractType<?> validator, List<ByteBuffer> variables) throws InvalidRequestException
-    {
-        try
-        {
-            if (!isBindMarker()) return validator.fromStringCQL2(text);
-
-            // must be a marker term so check for a CqlBindValue stored in the term
-            if (bindIndex == null)
-                throw new AssertionError("a marker Term was encountered with no index value");
-
-            return variables.get(bindIndex);
-        }
-        catch (MarshalException e)
-        {
-            throw new InvalidRequestException(e.getMessage());
-        }
-    }
-
-    /**
-     * Returns the typed value, serialized to a ByteBuffer.
-     *
-     * @return a ByteBuffer of the value.
-     * @throws InvalidRequestException if unable to coerce the string to its type.
-     */
-    public ByteBuffer getByteBuffer() throws InvalidRequestException
-    {
-        switch (type)
-        {
-            case STRING:
-                return AsciiType.instance.fromString(text);
-            case INTEGER:
-                return IntegerType.instance.fromString(text);
-            case UUID:
-                // we specifically want the Lexical class here, not "UUIDType," because we're supposed to have
-                // a uuid-shaped string here, and UUIDType also accepts integer or date strings (and turns them into version 1 uuids).
-                return LexicalUUIDType.instance.fromString(text);
-            case FLOAT:
-              return FloatType.instance.fromString(text);
-        }
-
-        // FIXME: handle scenario that should never happen
-        return null;
-    }
-
-    /**
-     * Obtain the term's type.
-     *
-     * @return the type
-     */
-    public TermType getType()
-    {
-        return type;
-    }
-
-    public String toString()
-    {
-        return String.format("Term(%s, type=%s)", getText(), type);
-    }
-
-    public boolean isBindMarker()
-    {
-        return type==TermType.QMARK;
-    }
-
-    @Override
-    public int hashCode()
-    {
-        final int prime = 31;
-        int result = 1;
-        result = prime * result + ((text == null) ? 0 : text.hashCode());
-        result = prime * result + ((type == null) ? 0 : type.hashCode());
-        return result;
-    }
-
-    @Override
-    public boolean equals(Object obj)
-    {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-        Term other = (Term) obj;
-        if (type==TermType.QMARK) return false; // markers are never equal
-        if (text == null)
-        {
-            if (other.text != null)
-                return false;
-        } else if (!text.equals(other.text))
-            return false;
-        if (type != other.type)
-            return false;
-        return true;
-    }
-
-
-}
-
-enum TermType
-{
-    STRING, INTEGER, UUID, FLOAT, QMARK;
-
-    static TermType forInt(int type)
-    {
-        if ((type == CqlParser.STRING_LITERAL) || (type == CqlParser.IDENT))
-            return STRING;
-        else if (type == CqlParser.INTEGER)
-            return INTEGER;
-        else if (type == CqlParser.UUID)
-          return UUID;
-        else if (type == CqlParser.FLOAT)
-            return FLOAT;
-        else if (type == CqlParser.QMARK)
-            return QMARK;
-
-        // FIXME: handled scenario that should never occur.
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/cql/UpdateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/UpdateStatement.java b/src/java/org/apache/cassandra/cql/UpdateStatement.java
deleted file mode 100644
index 16a0d76..0000000
--- a/src/java/org/apache/cassandra/cql/UpdateStatement.java
+++ /dev/null
@@ -1,288 +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.cql;
-
-import java.nio.ByteBuffer;
-import java.util.*;
-
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.CellName;
-import org.apache.cassandra.db.composites.CellNameType;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.thrift.ThriftClientState;
-
-import static org.apache.cassandra.cql.QueryProcessor.validateColumn;
-import static org.apache.cassandra.cql.QueryProcessor.validateKey;
-import static org.apache.cassandra.thrift.ThriftValidation.validateColumnFamily;
-
-/**
- * An <code>UPDATE</code> statement parsed from a CQL query statement.
- *
- */
-public class UpdateStatement extends AbstractModification
-{
-    private Map<Term, Operation> columns;
-    private List<Term> columnNames, columnValues;
-    private final List<Term> keys;
-
-    /**
-     * Creates a new UpdateStatement from a column family name, columns map, consistency
-     * level, and key term.
-     *
-     * @param keyspace Keyspace (optional)
-     * @param columnFamily column family name
-     * @param keyName alias key name
-     * @param columns a map of column name/values pairs
-     * @param keys the keys to update
-     * @param attrs additional attributes for statement (CL, timestamp, timeToLive)
-     */
-    public UpdateStatement(String keyspace,
-                           String columnFamily,
-                           String keyName,
-                           Map<Term, Operation> columns,
-                           List<Term> keys,
-                           Attributes attrs)
-    {
-        super(keyspace, columnFamily, keyName, attrs);
-
-        this.columns = columns;
-        this.keys = keys;
-    }
-
-    /**
-     * Creates a new UpdateStatement from a column family name, a consistency level,
-     * key, and lists of column names and values.  It is intended for use with the
-     * alternate update format, <code>INSERT</code>.
-     *
-     * @param keyspace Keyspace (optional)
-     * @param columnFamily column family name
-     * @param keyName alias key name
-     * @param columnNames list of column names
-     * @param columnValues list of column values (corresponds to names)
-     * @param keys the keys to update
-     * @param attrs additional attributes for statement (CL, timestamp, timeToLive)
-     */
-    public UpdateStatement(String keyspace,
-                           String columnFamily,
-                           String keyName,
-                           List<Term> columnNames,
-                           List<Term> columnValues,
-                           List<Term> keys,
-                           Attributes attrs)
-    {
-        super(keyspace, columnFamily, keyName, attrs);
-
-        this.columnNames = columnNames;
-        this.columnValues = columnValues;
-        this.keys = keys;
-    }
-
-    /**
-     * Returns the consistency level of this <code>UPDATE</code> statement, either
-     * one parsed from the CQL statement, or the default level otherwise.
-     *
-     * @return the consistency level as a Thrift enum.
-     */
-    public ConsistencyLevel getConsistencyLevel()
-    {
-        return (cLevel != null) ? cLevel : defaultConsistency;
-    }
-
-    /**
-     * True if an explicit consistency level was parsed from the statement.
-     *
-     * @return true if a consistency was parsed, false otherwise.
-     */
-    public boolean isSetConsistencyLevel()
-    {
-        return (cLevel != null);
-    }
-
-    /** {@inheritDoc} */
-    public List<IMutation> prepareRowMutations(String keyspace, ThriftClientState clientState, List<ByteBuffer> variables)
-    throws InvalidRequestException, UnauthorizedException
-    {
-        return prepareRowMutations(keyspace, clientState, null, variables);
-    }
-
-    /** {@inheritDoc} */
-    public List<IMutation> prepareRowMutations(String keyspace, ThriftClientState clientState, Long timestamp, List<ByteBuffer> variables)
-    throws InvalidRequestException, UnauthorizedException
-    {
-        boolean hasCommutativeOperation = false;
-
-        for (Map.Entry<Term, Operation> column : getColumns().entrySet())
-        {
-            if (!column.getValue().isUnary())
-                hasCommutativeOperation = true;
-
-            if (hasCommutativeOperation && column.getValue().isUnary())
-                throw new InvalidRequestException("Mix of commutative and non-commutative operations is not allowed.");
-        }
-
-        CFMetaData metadata = validateColumnFamily(keyspace, columnFamily, hasCommutativeOperation);
-        if (hasCommutativeOperation)
-            getConsistencyLevel().validateCounterForWrite(metadata);
-
-        QueryProcessor.validateKeyAlias(metadata, keyName);
-
-        clientState.hasColumnFamilyAccess(keyspace, columnFamily, Permission.MODIFY);
-
-        List<IMutation> mutations = new LinkedList<>();
-
-        for (Term key: keys)
-            mutations.add(mutationForKey(keyspace, key.getByteBuffer(getKeyType(keyspace),variables), metadata, timestamp, clientState, variables));
-
-        return mutations;
-    }
-
-    /**
-     * Compute a mutation for a single key
-     *
-     *
-     * @param keyspace working keyspace
-     * @param key key to change
-     * @param metadata information about CF
-     * @param timestamp global timestamp to use for every key mutation
-     *
-     * @param clientState
-     * @return mutation
-     *
-     * @throws InvalidRequestException on the wrong request
-     */
-    private IMutation mutationForKey(String keyspace, ByteBuffer key, CFMetaData metadata, Long timestamp, ThriftClientState clientState, List<ByteBuffer> variables)
-    throws InvalidRequestException
-    {
-        validateKey(key);
-        CellNameType comparator = metadata.comparator;
-        AbstractType<?> at = comparator.asAbstractType();
-
-        // if true we need to wrap Mutation into CounterMutation
-        boolean hasCounterColumn = false;
-        Mutation mutation = new Mutation(keyspace, key);
-
-        for (Map.Entry<Term, Operation> column : getColumns().entrySet())
-        {
-            CellName colName = comparator.cellFromByteBuffer(column.getKey().getByteBuffer(at, variables));
-            Operation op = column.getValue();
-
-            if (op.isUnary())
-            {
-                if (hasCounterColumn)
-                    throw new InvalidRequestException("Mix of commutative and non-commutative operations is not allowed.");
-
-                ByteBuffer colValue = op.a.getByteBuffer(metadata.getValueValidator(colName),variables);
-
-                validateColumn(metadata, colName, colValue);
-                mutation.add(columnFamily,
-                             colName,
-                             colValue,
-                             (timestamp == null) ? getTimestamp(clientState) : timestamp,
-                             getTimeToLive());
-            }
-            else
-            {
-                hasCounterColumn = true;
-
-                if (!column.getKey().getText().equals(op.a.getText()))
-                    throw new InvalidRequestException("Only expressions like X = X + <long> are supported.");
-
-                long value;
-
-                try
-                {
-                    value = Long.parseLong(op.b.getText());
-                }
-                catch (NumberFormatException e)
-                {
-                    throw new InvalidRequestException(String.format("'%s' is an invalid value, should be a long.",
-                                                      op.b.getText()));
-                }
-
-                mutation.addCounter(columnFamily, colName, value);
-            }
-        }
-
-        return (hasCounterColumn) ? new CounterMutation(mutation, getConsistencyLevel()) : mutation;
-    }
-
-    public String getColumnFamily()
-    {
-        return columnFamily;
-    }
-
-    public List<Term> getKeys()
-    {
-        return keys;
-    }
-
-    public Map<Term, Operation> getColumns() throws InvalidRequestException
-    {
-        // Created from an UPDATE
-        if (columns != null)
-            return columns;
-
-        // Created from an INSERT
-
-        // Don't hate, validate.
-        if (columnNames.size() != columnValues.size())
-            throw new InvalidRequestException("unmatched column names/values");
-        if (columnNames.size() < 1)
-            throw new InvalidRequestException("no columns specified for INSERT");
-
-        columns = new HashMap<Term, Operation>();
-
-        for (int i = 0; i < columnNames.size(); i++)
-            columns.put(columnNames.get(i), new Operation(columnValues.get(i)));
-
-        return columns;
-    }
-
-    public String toString()
-    {
-        return String.format("UpdateStatement(keyspace=%s, columnFamily=%s, keys=%s, columns=%s, consistency=%s, timestamp=%s, timeToLive=%s)",
-                             keyspace,
-                             columnFamily,
-                             keys,
-                             columns,
-                             getConsistencyLevel(),
-                             timestamp,
-                             timeToLive);
-    }
-
-    public AbstractType<?> getKeyType(String keyspace)
-    {
-        return Schema.instance.getCFMetaData(keyspace, columnFamily).getKeyValidator();
-    }
-
-    public List<Term> getColumnNames()
-    {
-        return columnNames;
-    }
-
-    public List<Term> getColumnValues()
-    {
-        return columnValues;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/cql/WhereClause.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/WhereClause.java b/src/java/org/apache/cassandra/cql/WhereClause.java
deleted file mode 100644
index 3df5102..0000000
--- a/src/java/org/apache/cassandra/cql/WhereClause.java
+++ /dev/null
@@ -1,192 +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.cql;
-
-import java.util.ArrayList;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.cassandra.config.CFMetaData;
-
-/**
- * WhereClauses encapsulate all of the predicates of a SELECT query.
- *
- */
-public class WhereClause
-{
-    // all relations (except for `<key> IN (.., .., ..)` which can be directly interpreted) from parser
-    // are stored into this array and are filtered to the keys/columns by extractKeysFromColumns(...)
-    private final List<Relation> clauseRelations = new ArrayList<Relation>();
-    private final List<Relation> columns = new ArrayList<Relation>();
-
-    // added to either by the parser from an IN clause or by extractKeysFromColumns
-    private final Set<Term> keys = new LinkedHashSet<Term>();
-    private Term startKey, finishKey;
-    private boolean includeStartKey = false, includeFinishKey = false, multiKey = false;
-    // set by extractKeysFromColumns
-    private String keyAlias = null;
-
-    /**
-     * Create a new WhereClause with the first parsed relation.
-     *
-     * @param firstRelation key or column relation
-     */
-    public WhereClause(Relation firstRelation)
-    {
-        and(firstRelation);
-    }
-
-    public WhereClause()
-    {}
-
-    /**
-     * Add an additional relation to this WHERE clause.
-     *
-     * @param relation the relation to add.
-     */
-    public void and(Relation relation)
-    {
-        clauseRelations.add(relation);
-    }
-
-    /**
-     * The same as KEY = <key> to avoid using Relation object
-     * @param key key to include into clause
-     */
-    public void andKeyEquals(Term key)
-    {
-        keys.add(key);
-    }
-
-    public List<Relation> getColumnRelations()
-    {
-        return columns;
-    }
-
-    public boolean isKeyRange()
-    {
-        return startKey != null;
-    }
-
-    public Term getStartKey()
-    {
-        return startKey;
-    }
-
-    public Term getFinishKey()
-    {
-        return finishKey;
-    }
-
-    public Set<Term> getKeys()
-    {
-        return keys;
-    }
-
-    public boolean includeStartKey()
-    {
-        return includeStartKey;
-    }
-
-    public boolean includeFinishKey()
-    {
-        return includeFinishKey;
-    }
-
-    public void setKeyAlias(String alias)
-    {
-        keyAlias = alias.toUpperCase();
-    }
-
-    public boolean isMultiKey() {
-        return multiKey;
-    }
-
-    public void setMultiKey(boolean multiKey)
-    {
-        this.multiKey = multiKey;
-    }
-
-    public String getKeyAlias()
-    {
-        // TODO fix special casing here, key alias should always be set post-extract
-        // key alias as not related to keys in here, it can be unset when we have a query like
-        // SELECT * FROM <CF> WHERE key = 1 and col > 2 and col < 3;
-        // it will be always set when statement looks like this
-        // SELECT * FROM <CF> WHERE <key> IN (.., .., ..);
-        // key is NULL when KEY keyword is used or when key alias given by user was not recognized
-        // validateKeyAlias will throw an exception for us in that case
-        return keyAlias == null ? QueryProcessor.DEFAULT_KEY_NAME : keyAlias;
-    }
-
-    public void extractKeysFromColumns(CFMetaData cfm)
-    {
-        String realKeyAlias = cfm.getCQL2KeyName();
-
-        if (!keys.isEmpty())
-            return; // we already have key(s) set (<key> IN (.., ...) construction used)
-
-        for (Relation relation : clauseRelations)
-        {
-            String name = relation.getEntity().getText().toUpperCase();
-            if (name.equals(realKeyAlias))
-            {
-                if (keyAlias == null) // setting found key as an alias
-                    keyAlias = name;
-
-                if (relation.operator() == RelationType.EQ)
-                {
-                    keys.add(relation.getValue());
-                }
-                else if ((relation.operator() == RelationType.GT) || (relation.operator() == RelationType.GTE))
-                {
-                    startKey = relation.getValue();
-                    includeStartKey = relation.operator() == RelationType.GTE;
-                }
-                else if ((relation.operator() == RelationType.LT) || (relation.operator() == RelationType.LTE))
-                {
-                    finishKey = relation.getValue();
-                    includeFinishKey = relation.operator() == RelationType.LTE;
-                }
-            }
-            else
-            {
-                columns.add(relation);
-            }
-        }
-    }
-
-    public List<Relation> getClauseRelations()
-    {
-        return clauseRelations;
-    }
-
-    public String toString()
-    {
-        return String.format("WhereClause [keys=%s, startKey=%s, finishKey=%s, columns=%s, includeStartKey=%s, includeFinishKey=%s, multiKey=%s, keyAlias=%s]",
-                             keys,
-                             startKey,
-                             finishKey,
-                             columns,
-                             includeStartKey,
-                             includeFinishKey,
-                             multiKey,
-                             keyAlias);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/cql/hooks/ExecutionContext.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/hooks/ExecutionContext.java b/src/java/org/apache/cassandra/cql/hooks/ExecutionContext.java
deleted file mode 100644
index deb785c..0000000
--- a/src/java/org/apache/cassandra/cql/hooks/ExecutionContext.java
+++ /dev/null
@@ -1,48 +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.cql.hooks;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import com.google.common.base.Optional;
-
-import org.apache.cassandra.thrift.ThriftClientState;
-
-/**
- * Contextual information about the execution of a CQLStatement.
- * Used by {@link org.apache.cassandra.cql.hooks.PreExecutionHook}
- *
- * The CQL string representing the statement being executed is optional
- * and is not present for prepared statements. Contexts created for the
- * execution of regular (i.e. non-prepared) statements will always
- * contain a CQL string.
- */
-public class ExecutionContext
-{
-    public final ThriftClientState clientState;
-    public final Optional<String> queryString;
-    public final List<ByteBuffer> variables;
-
-    public ExecutionContext(ThriftClientState clientState, String queryString, List<ByteBuffer> variables)
-    {
-        this.clientState = clientState;
-        this.queryString = Optional.fromNullable(queryString);
-        this.variables = variables;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/cql/hooks/PostPreparationHook.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/hooks/PostPreparationHook.java b/src/java/org/apache/cassandra/cql/hooks/PostPreparationHook.java
deleted file mode 100644
index 1de9c70..0000000
--- a/src/java/org/apache/cassandra/cql/hooks/PostPreparationHook.java
+++ /dev/null
@@ -1,38 +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.cql.hooks;
-
-import org.apache.cassandra.cql.CQLStatement;
-import org.apache.cassandra.exceptions.RequestValidationException;
-
-/**
- * Run directly after a CQL Statement is prepared in
- * {@link org.apache.cassandra.cql.QueryProcessor}.
- */
-public interface PostPreparationHook
-{
-    /**
-     * Called in QueryProcessor, once a CQL statement has been prepared.
-     *
-     * @param statement the statement to perform additional processing on
-     * @param context preparation context containing additional info
-     *                about the operation and statement
-     * @throws RequestValidationException
-     */
-    void processStatement(CQLStatement statement, PreparationContext context) throws RequestValidationException;
-}


[3/4] Remove CQL2

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/cql/Cql.g
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/Cql.g b/src/java/org/apache/cassandra/cql/Cql.g
deleted file mode 100644
index 3c41f44..0000000
--- a/src/java/org/apache/cassandra/cql/Cql.g
+++ /dev/null
@@ -1,656 +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.
- * 
- */
- 
-grammar Cql;
-
-options {
-    language = Java;
-}
-
-@header {
-    package org.apache.cassandra.cql;
-    import java.util.Map;
-    import java.util.HashMap;
-    import java.util.Collections;
-    import java.util.List;
-    import java.util.ArrayList;
-    import org.apache.cassandra.exceptions.SyntaxException;
-    import org.apache.cassandra.utils.Pair;
-    import org.apache.cassandra.db.ConsistencyLevel;
-
-    import static org.apache.cassandra.cql.AlterTableStatement.OperationType;
-}
-
-@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 SyntaxException
-    {
-        if (recognitionErrors.size() > 0)
-            throw new SyntaxException(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 && Long.parseLong(value.getText()) > 0)
-            throw new MissingTokenException(102, stream, value);
-    }
-}
-
-@lexer::header {
-    package org.apache.cassandra.cql;
-    import org.apache.cassandra.exceptions.SyntaxException;
-}
-
-@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 SyntaxException
-    {
-        if (recognitionErrors.size() > 0)
-            throw new SyntaxException(recognitionErrors.get((recognitionErrors.size()-1)));
-    }
-}
-
-query returns [CQLStatement stmnt]
-    : selectStatement   { $stmnt = new CQLStatement(StatementType.SELECT, $selectStatement.expr, currentBindMarkerIdx); }
-    | insertStatement endStmnt { $stmnt = new CQLStatement(StatementType.INSERT, $insertStatement.expr, currentBindMarkerIdx); }
-    | updateStatement endStmnt { $stmnt = new CQLStatement(StatementType.UPDATE, $updateStatement.expr, currentBindMarkerIdx); }
-    | batchStatement { $stmnt = new CQLStatement(StatementType.BATCH, $batchStatement.expr, currentBindMarkerIdx); }
-    | useStatement      { $stmnt = new CQLStatement(StatementType.USE, $useStatement.keyspace, currentBindMarkerIdx); }
-    | truncateStatement { $stmnt = new CQLStatement(StatementType.TRUNCATE, $truncateStatement.cf, currentBindMarkerIdx); }
-    | deleteStatement endStmnt { $stmnt = new CQLStatement(StatementType.DELETE, $deleteStatement.expr, currentBindMarkerIdx); }
-    | createKeyspaceStatement { $stmnt = new CQLStatement(StatementType.CREATE_KEYSPACE, $createKeyspaceStatement.expr, currentBindMarkerIdx); }
-    | createColumnFamilyStatement { $stmnt = new CQLStatement(StatementType.CREATE_COLUMNFAMILY, $createColumnFamilyStatement.expr, currentBindMarkerIdx); }
-    | createIndexStatement { $stmnt = new CQLStatement(StatementType.CREATE_INDEX, $createIndexStatement.expr, currentBindMarkerIdx); }
-    | dropIndexStatement   { $stmnt = new CQLStatement(StatementType.DROP_INDEX, $dropIndexStatement.expr, currentBindMarkerIdx); }
-    | dropKeyspaceStatement { $stmnt = new CQLStatement(StatementType.DROP_KEYSPACE, $dropKeyspaceStatement.ksp, currentBindMarkerIdx); }
-    | dropColumnFamilyStatement { $stmnt = new CQLStatement(StatementType.DROP_COLUMNFAMILY, $dropColumnFamilyStatement.cfam, currentBindMarkerIdx); }
-    | alterTableStatement { $stmnt = new CQLStatement(StatementType.ALTER_TABLE, $alterTableStatement.expr, currentBindMarkerIdx); }
-    ;
-
-// USE <KEYSPACE>;
-useStatement returns [String keyspace]
-    : K_USE name=( IDENT | INTEGER | STRING_LITERAL ) { $keyspace = $name.text; } endStmnt
-    ;
-
-/**
- * SELECT
- *  (REVERSED)? <expression>
- * FROM
- *     <CF>
- * USING
- *     CONSISTENCY <LEVEL>
- * WHERE
- *     KEY = "key1" AND KEY = "key2" AND
- *     COL > 1 AND COL < 100
- * LIMIT <NUMBER>;
- */
-selectStatement returns [SelectStatement expr]
-    : { 
-          int numRecords = 10000;
-          SelectExpression expression = null;
-          boolean isCountOp = false;
-          ConsistencyLevel cLevel = ConsistencyLevel.ONE;
-      }
-      K_SELECT
-          ( s1=selectExpression                 { expression = s1; }
-          | K_COUNT '(' s2=selectExpression ')' { expression = s2; isCountOp = true; }
-          )
-          K_FROM (keyspace=(IDENT | STRING_LITERAL | INTEGER) '.')? columnFamily=( IDENT | STRING_LITERAL | INTEGER )
-          ( K_USING K_CONSISTENCY K_LEVEL { cLevel = ConsistencyLevel.valueOf($K_LEVEL.text.toUpperCase()); } )?
-          ( K_WHERE whereClause )?
-          ( K_LIMIT rows=INTEGER { numRecords = Integer.parseInt($rows.text); } )?
-          endStmnt
-      {
-          return new SelectStatement(expression,
-                                     isCountOp,
-                                     $keyspace.text,
-                                     $columnFamily.text,
-                                     cLevel,
-                                     $whereClause.clause,
-                                     numRecords);
-      }
-    ;
-
-// [FIRST n] [REVERSED] name1[[[,name2],nameN],...]
-// [FIRST n] [REVERSED] name1..nameN
-selectExpression returns [SelectExpression expr]
-    : {
-          int count = 10000;
-          boolean reversed = false;
-          boolean hasFirstSet = false;
-      }
-      ( K_FIRST { hasFirstSet = true; } cols=INTEGER { count = Integer.parseInt($cols.text); } )?
-      ( K_REVERSED { reversed = true; } )?
-      ( first=term { $expr = new SelectExpression(first, count, reversed, hasFirstSet); }
-            (',' next=term { $expr.and(next); })*
-      | start=term RANGEOP finish=term { $expr = new SelectExpression(start, finish, count, reversed, false, hasFirstSet); }
-      | '\*' { $expr = new SelectExpression(new Term(), new Term(), count, reversed, true, hasFirstSet); }
-      )
-    ;
-
-// relation [[AND relation] ...]
-whereClause returns [WhereClause clause]
-    @init {
-        WhereClause inClause = new WhereClause();
-    }
-    : first=relation { $clause = new WhereClause(first); } 
-          (K_AND next=relation { $clause.and(next); })*
-      | key_alias=term { inClause.setKeyAlias(key_alias.getText()); }
-           K_IN '(' f1=term { inClause.andKeyEquals(f1); }
-                  (',' fN=term { inClause.andKeyEquals(fN); } )* ')'
-        { inClause.setMultiKey(true); $clause = inClause; }
-    ;
-
-/**
- * INSERT INTO
- *    <CF>
- *    (KEY, <column>, <column>, ...)
- * VALUES
- *    (<key>, <value>, <value>, ...)
- * (USING
- *    CONSISTENCY <level>
- *   (AND TIMESTAMP <long>)?
- * )?;
- *
- * Consistency level is set to ONE by default
- */
-insertStatement returns [UpdateStatement expr]
-    : {
-          Attributes attrs = new Attributes();
-
-          List<Term> columnNames  = new ArrayList<Term>();
-          List<Term> columnValues = new ArrayList<Term>();
-      }
-      K_INSERT K_INTO (keyspace=(IDENT | STRING_LITERAL | INTEGER) '.')? columnFamily=( IDENT | STRING_LITERAL | INTEGER )
-          '(' key_alias=term ( ',' column_name=term  { columnNames.add($column_name.item); } )+ ')'
-        K_VALUES
-          '(' key=term ( ',' column_value=term { columnValues.add($column_value.item); })+ ')'
-        ( usingClause[attrs] )?
-      {
-          return new UpdateStatement($keyspace.text, $columnFamily.text, key_alias.getText(), columnNames, columnValues, Collections.singletonList(key), 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.setConsistencyLevel(ConsistencyLevel.valueOf($K_LEVEL.text.toUpperCase())); }
-    | K_TIMESTAMP ts=INTEGER { attrs.setTimestamp(Long.valueOf($ts.text)); }
-    ;
-
-usingClauseObjective[Attributes attrs]
-    : usingClauseDeleteObjective[attrs]
-    | K_TTL t=INTEGER        { attrs.setTimeToLive(Integer.parseInt($t.text)); }
-    ;
-
-/**
- * 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]
-    : {
-          Attributes attrs = new Attributes();
-          attrs.setConsistencyLevel(ConsistencyLevel.ONE);
-
-          List<AbstractModification> statements = new ArrayList<AbstractModification>();
-      }
-      K_BEGIN K_BATCH ( usingClause[attrs] )?
-          s1=batchStatementObjective ';'? { statements.add(s1); } ( sN=batchStatementObjective ';'? { statements.add(sN); } )*
-      K_APPLY K_BATCH endStmnt
-      {
-          return new BatchStatement(statements, attrs);
-      }
-    ;
-
-batchStatementObjective returns [AbstractModification statement]
-    : i=insertStatement  { $statement = i; }
-    | u=updateStatement  { $statement = u; }
-    | d=deleteStatement  { $statement = d; }
-    ;
-
-/**
- * UPDATE
- *     <CF>
- * (USING
- *     CONSISTENCY.ONE
- *    (AND TIMESTAMP <long>)?
- * )?
- * SET
- *     name1 = value1,
- *     name2 = value2
- * WHERE
- *     KEY = keyname;
- */
-updateStatement returns [UpdateStatement expr]
-    : {
-          Attributes attrs = new Attributes();
-          Map<Term, Operation> columns = new HashMap<Term, Operation>();
-          List<Term> keyList = null;
-      }
-      K_UPDATE (keyspace=(IDENT | STRING_LITERAL | INTEGER) '.')? columnFamily=( IDENT | STRING_LITERAL | INTEGER )
-          ( usingClause[attrs] )?
-          K_SET termPairWithOperation[columns] (',' termPairWithOperation[columns])*
-          K_WHERE ( key_alias=term ('=' key=term { keyList = Collections.singletonList(key); }
-                                    |
-                                    K_IN '(' keys=termList { keyList = $keys.items; } ')' ))
-      {
-          return new UpdateStatement($keyspace.text, $columnFamily.text, key_alias.getText(), columns, keyList, attrs);
-      }
-    ;
-
-/**
- * DELETE
- *     name1, name2
- * FROM
- *     <CF>
- * USING
- *     CONSISTENCY.<LVL>
- * WHERE
- *     KEY = keyname;
- */
-deleteStatement returns [DeleteStatement expr]
-    : {
-          Attributes attrs = new Attributes();
-          List<Term> keyList = null;
-          List<Term> columnsList = Collections.emptyList();
-      }
-      K_DELETE
-          ( cols=termList { columnsList = $cols.items; })?
-          K_FROM (keyspace=(IDENT | STRING_LITERAL | INTEGER) '.')? columnFamily=( IDENT | STRING_LITERAL | INTEGER )
-          ( usingClauseDelete[attrs] )?
-          ( K_WHERE key_alias=term ('=' key=term           { keyList = Collections.singletonList(key); }
-                                   | K_IN '(' keys=termList { keyList = $keys.items; } ')')
-                  )?
-      {
-          return new DeleteStatement(columnsList, $keyspace.text, $columnFamily.text, key_alias.getText(), keyList, attrs);
-      }
-    ;
-
-
-
-/** CREATE KEYSPACE <KEYSPACE> WITH attr1 = value1 AND attr2 = value2; */
-createKeyspaceStatement returns [CreateKeyspaceStatement expr]
-    : {
-          Map<String, String> attrs = new HashMap<String, String>();
-      }
-      K_CREATE K_KEYSPACE keyspace=( IDENT | STRING_LITERAL | INTEGER )
-          K_WITH  a1=( COMPIDENT | IDENT ) '=' v1=( STRING_LITERAL | INTEGER | IDENT ) { attrs.put($a1.text, $v1.text); }
-          ( K_AND aN=( COMPIDENT | IDENT ) '=' vN=( STRING_LITERAL | INTEGER | IDENT ) { attrs.put($aN.text, $vN.text); } )*
-          endStmnt
-      {
-          return new CreateKeyspaceStatement($keyspace.text, attrs);
-      }
-    ;
-    
-/**
- * CREATE COLUMNFAMILY <CF> (
- *     <name1> <type>,
- *     <name2> <type>,
- *     <name3> <type>
- * ) WITH comparator = <type> [AND ...];
- */
-createColumnFamilyStatement returns [CreateColumnFamilyStatement expr]
-    : K_CREATE K_COLUMNFAMILY name=( IDENT | STRING_LITERAL | INTEGER ) { $expr = new CreateColumnFamilyStatement($name.text); }
-      ( '(' createCfamColumns[expr] ( ',' createCfamColumns[expr] )* ')' )?
-      ( K_WITH prop1=(COMPIDENT | IDENT) '=' arg1=createCfamKeywordArgument { $expr.addProperty($prop1.text, $arg1.arg); }
-          ( K_AND propN=(COMPIDENT | IDENT) '=' argN=createCfamKeywordArgument { $expr.addProperty($propN.text, $argN.arg); } )*
-      )?
-      endStmnt
-    ;
-
-createCfamColumns[CreateColumnFamilyStatement expr]
-    : n=term v=createCfamColumnValidator { $expr.addColumn(n, $v.validator); }
-    | k=term v=createCfamColumnValidator K_PRIMARY K_KEY { $expr.setKeyAlias(k.getText()); $expr.setKeyType($v.validator); }
-    ;
-
-createCfamColumnValidator returns [String validator]
-    : comparatorType { $validator = $comparatorType.text; }
-    | STRING_LITERAL { $validator = $STRING_LITERAL.text; }
-    ;
-
-createCfamKeywordArgument returns [String arg]
-    : comparatorType { $arg = $comparatorType.text; }
-    | value=( STRING_LITERAL | IDENT | INTEGER | FLOAT ) { $arg = $value.text; }
-    ;
-
-/** CREATE INDEX [indexName] ON columnFamily (columnName); */
-createIndexStatement returns [CreateIndexStatement expr]
-    : K_CREATE K_INDEX (idxName=IDENT)? K_ON cf=( IDENT | STRING_LITERAL | INTEGER ) '(' columnName=term ')' endStmnt
-      { $expr = new CreateIndexStatement($idxName.text, $cf.text, columnName); }
-    ;
-/**
- * DROP INDEX ON <CF>.<COLUMN_OR_INDEX_NAME>
- * DROP INDEX <INDEX_NAME>
- */
-dropIndexStatement returns [DropIndexStatement expr]
-    :
-      K_DROP K_INDEX index=( IDENT | STRING_LITERAL | INTEGER ) endStmnt
-      { $expr = new DropIndexStatement($index.text); }
-    ;
-
-/** DROP KEYSPACE <KSP>; */
-dropKeyspaceStatement returns [String ksp]
-    : K_DROP K_KEYSPACE name=( IDENT | STRING_LITERAL | INTEGER ) endStmnt { $ksp = $name.text; }
-    ;
-
-
-alterTableStatement returns [AlterTableStatement expr]
-    :
-    {
-        OperationType type = null;
-        String columnFamily = null, columnName = null, validator = null;
-        Map<String, String> propertyMap = null;
-    }
-    K_ALTER K_COLUMNFAMILY name=( IDENT | STRING_LITERAL | INTEGER ) { columnFamily = $name.text; }
-          ( K_ALTER { type = OperationType.ALTER; }
-               (col=( IDENT | STRING_LITERAL | INTEGER ) { columnName = $col.text; })
-               K_TYPE alterValidator=comparatorType { validator = $alterValidator.text; }
-          | K_ADD { type = OperationType.ADD; }
-               (col=( IDENT | STRING_LITERAL | INTEGER ) { columnName = $col.text; })
-               addValidator=comparatorType { validator = $addValidator.text; }
-          | K_DROP { type = OperationType.DROP; }
-               (col=( IDENT | STRING_LITERAL | INTEGER ) { columnName = $col.text; })
-          | K_WITH { type = OperationType.OPTS; propertyMap = new HashMap<String, String>(); }
-               prop1=(COMPIDENT | IDENT) '=' arg1=createCfamKeywordArgument { propertyMap.put($prop1.text, $arg1.arg); }
-               ( K_AND propN=(COMPIDENT | IDENT) '=' argN=createCfamKeywordArgument { propertyMap.put($propN.text, $argN.arg); } )* )
-    endStmnt
-      {
-          $expr = new AlterTableStatement(columnFamily, type, columnName, validator, propertyMap);
-      }
-    ;
-
-/** DROP COLUMNFAMILY <CF>; */
-dropColumnFamilyStatement returns [String cfam]
-    : K_DROP K_COLUMNFAMILY name=( IDENT | STRING_LITERAL | INTEGER ) endStmnt { $cfam = $name.text; }
-    ;
-
-comparatorType
-    : 'blob' | 'ascii' | 'text' | 'varchar' | 'int' | 'varint' | 'bigint' | 'uuid' | 'counter' | 'boolean' | 'timestamp' | 'float' | 'double' | 'decimal'
-    ;
-
-term returns [Term item]
-    : (( t=K_KEY | t=STRING_LITERAL | t=INTEGER | t=UUID | t=IDENT | t=FLOAT ) { $item = new Term($t.text, $t.type); }
-       | t=QMARK { $item = new Term($t.text, $t.type, ++currentBindMarkerIdx); }
-      )
-    ;
-
-termList returns [List<Term> items]
-    : { $items = new ArrayList<Term>(); }
-      t1=term { $items.add(t1); } (',' tN=term { $items.add(tN); })*
-    ;
-
-// term = term
-termPair[Map<Term, Term> columns]
-    :   key=term '=' value=term { columns.put(key, value); }
-    ;
-
-intTerm returns [Term integer]
-    : t=INTEGER { $integer = new Term($t.text, $t.type); }
-    ;
-
-termPairWithOperation[Map<Term, Operation> columns]
-    : key=term '=' (value=term { columns.put(key, new Operation(value)); }
-		               | c=term ( '+'  v=term { columns.put(key, new Operation(c, org.apache.cassandra.cql.Operation.OperationType.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, org.apache.cassandra.cql.Operation.OperationType.MINUS, v)); } ))
-    ;
-
-// Note: ranges are inclusive so >= and >, and < and <= all have the same semantics.  
-relation returns [Relation rel]
-    : name=term type=('=' | '<' | '<=' | '>=' | '>') t=term
-      { return new Relation($name.item, $type.text, $t.item); }
-    ;
-
-// TRUNCATE <CF>;
-truncateStatement returns [Pair<String,String> cf]
-    : K_TRUNCATE (keyspace=( IDENT | STRING_LITERAL | INTEGER ) '.')? columnFamily=( IDENT | STRING_LITERAL | INTEGER ) { $cf = Pair.create($keyspace.text, $columnFamily.text); } endStmnt
-    ;
-
-endStmnt
-    : ';'?  EOF
-    ;
-
-
-// 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
-               | T W O
-               | T H R E E
-               )
-               ;
-K_USE:         U S E;
-K_FIRST:       F I R S T;
-K_REVERSED:    R E V E R S E D;
-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;
-
-// 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
-    : '\''
-      { StringBuilder b = new StringBuilder(); }
-      ( c=~('\'') { b.appendCodePoint(c);}
-      | '\'' '\''            { b.appendCodePoint('\'');}
-      )*
-      '\''
-      { setText(b.toString()); }
-    ;
-
-fragment DIGIT
-    : '0'..'9'
-    ;
-
-fragment LETTER
-    : ('A'..'Z' | 'a'..'z')
-    ;
-    
-fragment HEX
-    : ('A'..'F' | 'a'..'f' | '0'..'9')
-    ;
-
-RANGEOP
-    : '..'
-    ;
-
-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
-    : d=INTEGER r=RANGEOP
-      {
-          $d.setType(INTEGER);
-          emit($d);
-          $r.setType(RANGEOP);
-          emit($r);
-      }
-      | 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/26217071/src/java/org/apache/cassandra/cql/CreateColumnFamilyStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/CreateColumnFamilyStatement.java b/src/java/org/apache/cassandra/cql/CreateColumnFamilyStatement.java
deleted file mode 100644
index e568dd7..0000000
--- a/src/java/org/apache/cassandra/cql/CreateColumnFamilyStatement.java
+++ /dev/null
@@ -1,253 +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.cql;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.composites.SimpleDenseCellNameType;
-import org.apache.cassandra.db.ColumnFamilyType;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.TypeParser;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.SyntaxException;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.io.compress.CompressionParameters;
-
-/** A <code>CREATE COLUMNFAMILY</code> parsed from a CQL query statement. */
-public class CreateColumnFamilyStatement
-{
-    private final String name;
-    private final Map<Term, String> columns = new HashMap<Term, String>();
-    private final List<String> keyValidator = new ArrayList<String>();
-    private ByteBuffer keyAlias = null;
-    private final CFPropDefs cfProps = new CFPropDefs();
-
-    public CreateColumnFamilyStatement(String name)
-    {
-        this.name = name;
-    }
-
-    /** Perform validation of parsed params */
-    private void validate(List<ByteBuffer> variables) throws InvalidRequestException
-    {
-        // Ensure that exactly one key has been specified.
-        if (keyValidator.size() < 1)
-            throw new InvalidRequestException("You must specify a PRIMARY KEY");
-        else if (keyValidator.size() > 1)
-            throw new InvalidRequestException("You may only specify one PRIMARY KEY");
-
-        AbstractType<?> comparator;
-
-        try
-        {
-            cfProps.validate();
-            comparator = cfProps.getComparator();
-        }
-        catch (ConfigurationException e)
-        {
-            throw new InvalidRequestException(e.toString());
-        }
-        catch (SyntaxException e)
-        {
-            throw new InvalidRequestException(e.toString());
-        }
-
-        for (Map.Entry<Term, String> column : columns.entrySet())
-        {
-            ByteBuffer name = column.getKey().getByteBuffer(comparator, variables);
-
-            if (keyAlias != null && keyAlias.equals(name))
-                throw new InvalidRequestException("Invalid column name: "
-                                                  + column.getKey().getText()
-                                                  + ", because it equals to the key_alias.");
-
-        }
-    }
-
-    /** Map a column name to a validator for its value */
-    public void addColumn(Term term, String comparator)
-    {
-        columns.put(term, comparator);
-    }
-
-    public void setKeyType(String validator)
-    {
-        keyValidator.add(validator);
-    }
-
-    public String getKeyType()
-    {
-        return keyValidator.get(0);
-    }
-
-    public void setKeyAlias(String alias)
-    {
-        // if we got KEY in input we don't need to set an alias
-        if (!alias.toUpperCase().equals("KEY"))
-            keyAlias = ByteBufferUtil.bytes(alias);
-    }
-
-    /** Map a keyword to the corresponding value */
-    public void addProperty(String name, String value)
-    {
-        cfProps.addProperty(name, value);
-    }
-
-    /** Name of the column family to create */
-    public String getName()
-    {
-        return name;
-    }
-
-    // Column definitions
-    private List<ColumnDefinition> getColumns(CFMetaData cfm) throws InvalidRequestException
-    {
-        List<ColumnDefinition> columnDefs = new ArrayList<>(columns.size());
-
-        for (Map.Entry<Term, String> col : columns.entrySet())
-        {
-            try
-            {
-                ByteBuffer columnName = cfm.comparator.asAbstractType().fromStringCQL2(col.getKey().getText());
-                String validatorClassName = CFPropDefs.comparators.containsKey(col.getValue())
-                                          ? CFPropDefs.comparators.get(col.getValue())
-                                          : col.getValue();
-                AbstractType<?> validator = TypeParser.parse(validatorClassName);
-                columnDefs.add(ColumnDefinition.regularDef(cfm, columnName, validator, null));
-            }
-            catch (ConfigurationException e)
-            {
-                InvalidRequestException ex = new InvalidRequestException(e.toString());
-                ex.initCause(e);
-                throw ex;
-            }
-            catch (SyntaxException e)
-            {
-                InvalidRequestException ex = new InvalidRequestException(e.toString());
-                ex.initCause(e);
-                throw ex;
-            }
-        }
-
-        return columnDefs;
-    }
-
-    /**
-     * Returns a CFMetaData instance based on the parameters parsed from this
-     * <code>CREATE</code> statement, or defaults where applicable.
-     *
-     * @param keyspace keyspace to apply this column family to
-     * @return a CFMetaData instance corresponding to the values parsed from this statement
-     * @throws InvalidRequestException on failure to validate parsed parameters
-     */
-    public CFMetaData getCFMetaData(String keyspace, List<ByteBuffer> variables) throws InvalidRequestException
-    {
-        validate(variables);
-
-        CFMetaData newCFMD;
-        try
-        {
-            AbstractType<?> comparator = cfProps.getComparator();
-
-            newCFMD = new CFMetaData(keyspace,
-                                     name,
-                                     ColumnFamilyType.Standard,
-                                     new SimpleDenseCellNameType(comparator));
-
-            if (CFMetaData.DEFAULT_COMPRESSOR != null && cfProps.compressionParameters.isEmpty())
-                cfProps.compressionParameters.put(CompressionParameters.SSTABLE_COMPRESSION, CFMetaData.DEFAULT_COMPRESSOR);
-            int maxCompactionThreshold = getPropertyInt(CFPropDefs.KW_MAXCOMPACTIONTHRESHOLD, CFMetaData.DEFAULT_MAX_COMPACTION_THRESHOLD);
-            int minCompactionThreshold = getPropertyInt(CFPropDefs.KW_MINCOMPACTIONTHRESHOLD, CFMetaData.DEFAULT_MIN_COMPACTION_THRESHOLD);
-            if (minCompactionThreshold <= 0 || maxCompactionThreshold <= 0)
-                throw new ConfigurationException("Disabling compaction by setting compaction thresholds to 0 has been deprecated, set the compaction option 'enabled' to false instead.");
-
-            newCFMD.addAllColumnDefinitions(getColumns(newCFMD))
-                   .comment(cfProps.getProperty(CFPropDefs.KW_COMMENT))
-                   .readRepairChance(getPropertyDouble(CFPropDefs.KW_READREPAIRCHANCE, CFMetaData.DEFAULT_READ_REPAIR_CHANCE))
-                   .dcLocalReadRepairChance(getPropertyDouble(CFPropDefs.KW_DCLOCALREADREPAIRCHANCE, CFMetaData.DEFAULT_DCLOCAL_READ_REPAIR_CHANCE))
-                   .gcGraceSeconds(getPropertyInt(CFPropDefs.KW_GCGRACESECONDS, CFMetaData.DEFAULT_GC_GRACE_SECONDS))
-                   .defaultValidator(cfProps.getValidator())
-                   .minCompactionThreshold(minCompactionThreshold)
-                   .maxCompactionThreshold(maxCompactionThreshold)
-                   .keyValidator(TypeParser.parse(CFPropDefs.comparators.get(getKeyType())))
-                   .compactionStrategyClass(cfProps.compactionStrategyClass)
-                   .compactionStrategyOptions(cfProps.compactionStrategyOptions)
-                   .compressionParameters(CompressionParameters.create(cfProps.compressionParameters))
-                   .caching(CFMetaData.Caching.fromString(getPropertyString(CFPropDefs.KW_CACHING, CFMetaData.DEFAULT_CACHING_STRATEGY.toString())))
-                   .rowsPerPartitionToCache(CFMetaData.RowsPerPartitionToCache.fromString(cfProps.getPropertyString(CFPropDefs.KW_ROWS_PER_PARTITION_TO_CACHE, CFMetaData.DEFAULT_ROWS_PER_PARTITION_TO_CACHE.toString())))
-                   .speculativeRetry(CFMetaData.SpeculativeRetry.fromString(getPropertyString(CFPropDefs.KW_SPECULATIVE_RETRY, CFMetaData.DEFAULT_SPECULATIVE_RETRY.toString())))
-                   .bloomFilterFpChance(getPropertyDouble(CFPropDefs.KW_BF_FP_CHANCE, null))
-                   .memtableFlushPeriod(getPropertyInt(CFPropDefs.KW_MEMTABLE_FLUSH_PERIOD, 0))
-                   .defaultTimeToLive(getPropertyInt(CFPropDefs.KW_DEFAULT_TIME_TO_LIVE, CFMetaData.DEFAULT_DEFAULT_TIME_TO_LIVE))
-                   .populateIoCacheOnFlush(getPropertyBoolean(CFPropDefs.KW_POPULATE_IO_CACHE_ON_FLUSH, CFMetaData.DEFAULT_POPULATE_IO_CACHE_ON_FLUSH));
-
-            // CQL2 can have null keyAliases
-            if (keyAlias != null)
-                newCFMD.addColumnDefinition(ColumnDefinition.partitionKeyDef(newCFMD, keyAlias, newCFMD.getKeyValidator(), null));
-        }
-        catch (ConfigurationException e)
-        {
-            throw new InvalidRequestException(e.toString());
-        }
-        catch (SyntaxException e)
-        {
-            throw new InvalidRequestException(e.toString());
-        }
-        return newCFMD;
-    }
-
-    private String getPropertyString(String key, String defaultValue)
-    {
-        return cfProps.getPropertyString(key, defaultValue);
-    }
-
-    private Boolean getPropertyBoolean(String key, Boolean defaultValue)
-    {
-        return cfProps.getPropertyBoolean(key, defaultValue);
-    }
-
-    private Double getPropertyDouble(String key, Double defaultValue) throws InvalidRequestException
-    {
-        return cfProps.getPropertyDouble(key, defaultValue);
-    }
-
-    private Integer getPropertyInt(String key, Integer defaultValue) throws InvalidRequestException
-    {
-        return cfProps.getPropertyInt(key, defaultValue);
-    }
-
-    private Set<String> getPropertySet(String key, Set<String> defaultValue)
-    {
-        return cfProps.getPropertySet(key, defaultValue);
-    }
-
-    public Map<Term, String> getColumns()
-    {
-        return columns;
-    }
-
-}
-

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/cql/CreateIndexStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/CreateIndexStatement.java b/src/java/org/apache/cassandra/cql/CreateIndexStatement.java
deleted file mode 100644
index 54b5eef..0000000
--- a/src/java/org/apache/cassandra/cql/CreateIndexStatement.java
+++ /dev/null
@@ -1,51 +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.cql;
-
-/** A <code>CREATE INDEX</code> statement parsed from a CQL query. */
-public class CreateIndexStatement
-{
-    private final String columnFamily;
-    private final String indexName;
-    private final Term columnName;
-
-    public CreateIndexStatement(String indexName, String columnFamily, Term columnName)
-    {
-        this.indexName = indexName;
-        this.columnFamily = columnFamily;
-        this.columnName = columnName;
-    }
-
-    /** Column family namespace. */
-    public String getColumnFamily()
-    {
-        return columnFamily;
-    }
-
-    /** Column name to index. */
-    public Term getColumnName()
-    {
-        return columnName;
-    }
-
-    /** Index name (or null). */
-    public String getIndexName()
-    {
-        return indexName;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/cql/CreateKeyspaceStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/CreateKeyspaceStatement.java b/src/java/org/apache/cassandra/cql/CreateKeyspaceStatement.java
deleted file mode 100644
index 8c2aa1b..0000000
--- a/src/java/org/apache/cassandra/cql/CreateKeyspaceStatement.java
+++ /dev/null
@@ -1,80 +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.cql;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.cassandra.exceptions.InvalidRequestException;
-
-/** A <code>CREATE KEYSPACE</code> statement parsed from a CQL query. */
-public class CreateKeyspaceStatement
-{
-    private final String name;
-    private final Map<String, String> attrs;
-    private String strategyClass;
-    private final Map<String, String> strategyOptions = new HashMap<String, String>();
-
-    /**
-     * Creates a new <code>CreateKeyspaceStatement</code> instance for a given
-     * keyspace name and keyword arguments.
-     *
-     * @param name the name of the keyspace to create
-     * @param attrs map of the raw keyword arguments that followed the <code>WITH</code> keyword.
-     */
-    public CreateKeyspaceStatement(String name, Map<String, String> attrs)
-    {
-        this.name = name;
-        this.attrs = attrs;
-    }
-
-    /**
-     * The <code>CqlParser</code> only goes as far as extracting the keyword arguments
-     * from these statements, so this method is responsible for processing and
-     * validating, and must be called prior to access.
-     *
-     * @throws InvalidRequestException if arguments are missing or unacceptable
-     */
-    public void validate() throws InvalidRequestException
-    {
-        // required
-        if (!attrs.containsKey("strategy_class"))
-            throw new InvalidRequestException("missing required argument \"strategy_class\"");
-        strategyClass = attrs.get("strategy_class");
-
-        // optional
-        for (String key : attrs.keySet())
-            if ((key.contains(":")) && (key.startsWith("strategy_options")))
-                strategyOptions.put(key.split(":")[1], attrs.get(key));
-    }
-
-    public String getName()
-    {
-        return name;
-    }
-
-    public String getStrategyClass()
-    {
-        return strategyClass;
-    }
-
-    public Map<String, String> getStrategyOptions()
-    {
-        return strategyOptions;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/cql/DeleteStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/DeleteStatement.java b/src/java/org/apache/cassandra/cql/DeleteStatement.java
deleted file mode 100644
index 71942e4..0000000
--- a/src/java/org/apache/cassandra/cql/DeleteStatement.java
+++ /dev/null
@@ -1,123 +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.cql;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.db.composites.CellName;
-import org.apache.cassandra.db.IMutation;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.thrift.ThriftClientState;
-
-import static org.apache.cassandra.thrift.ThriftValidation.validateColumnFamily;
-import static org.apache.cassandra.cql.QueryProcessor.validateColumnName;
-
-/**
- * A <code>DELETE</code> parsed from a CQL query statement.
- *
- */
-public class DeleteStatement extends AbstractModification
-{
-    private List<Term> columns;
-    private List<Term> keys;
-
-    public DeleteStatement(List<Term> columns, String keyspace, String columnFamily, String keyName, List<Term> keys, Attributes attrs)
-    {
-        super(keyspace, columnFamily, keyName, attrs);
-
-        this.columns = columns;
-        this.keys = keys;
-    }
-
-    public List<Term> getColumns()
-    {
-        return columns;
-    }
-
-    public List<Term> getKeys()
-    {
-        return keys;
-    }
-
-    public List<IMutation> prepareRowMutations(String keyspace, ThriftClientState clientState, List<ByteBuffer> variables)
-    throws InvalidRequestException, UnauthorizedException
-    {
-        return prepareRowMutations(keyspace, clientState, null, variables);
-    }
-
-    public List<IMutation> prepareRowMutations(String keyspace, ThriftClientState clientState, Long timestamp, List<ByteBuffer> variables)
-    throws InvalidRequestException, UnauthorizedException
-    {
-        CFMetaData metadata = validateColumnFamily(keyspace, columnFamily);
-
-        clientState.hasColumnFamilyAccess(keyspace, columnFamily, Permission.MODIFY);
-        AbstractType<?> keyType = Schema.instance.getCFMetaData(keyspace, columnFamily).getKeyValidator();
-
-        List<IMutation> mutations = new ArrayList<IMutation>(keys.size());
-
-        for (Term key : keys)
-            mutations.add(mutationForKey(key.getByteBuffer(keyType, variables), keyspace, timestamp, clientState, variables, metadata));
-
-        return mutations;
-    }
-
-    public Mutation mutationForKey(ByteBuffer key, String keyspace, Long timestamp, ThriftClientState clientState, List<ByteBuffer> variables, CFMetaData metadata)
-    throws InvalidRequestException
-    {
-        Mutation mutation = new Mutation(keyspace, key);
-
-        QueryProcessor.validateKeyAlias(metadata, keyName);
-
-        if (columns.size() < 1)
-        {
-            // No columns, delete the partition
-            mutation.delete(columnFamily, (timestamp == null) ? getTimestamp(clientState) : timestamp);
-        }
-        else
-        {
-            // Delete specific columns
-            AbstractType<?> at = metadata.comparator.asAbstractType();
-            for (Term column : columns)
-            {
-                CellName columnName = metadata.comparator.cellFromByteBuffer(column.getByteBuffer(at, variables));
-                validateColumnName(columnName);
-                mutation.delete(columnFamily, columnName, (timestamp == null) ? getTimestamp(clientState) : timestamp);
-            }
-        }
-
-        return mutation;
-    }
-
-    public String toString()
-    {
-        return String.format("DeleteStatement(columns=%s, keyspace=%s, columnFamily=%s, consistency=%s keys=%s)",
-                             columns,
-                             keyspace,
-                             columnFamily,
-                             cLevel,
-                             keys);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/cql/DropIndexStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/DropIndexStatement.java b/src/java/org/apache/cassandra/cql/DropIndexStatement.java
deleted file mode 100644
index bc9bbbc..0000000
--- a/src/java/org/apache/cassandra/cql/DropIndexStatement.java
+++ /dev/null
@@ -1,80 +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.cql;
-
-import org.apache.cassandra.config.*;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-
-public class DropIndexStatement
-{
-    public final String indexName;
-    private String keyspace;
-
-    public DropIndexStatement(String indexName)
-    {
-        this.indexName = indexName;
-    }
-
-    public void setKeyspace(String keyspace)
-    {
-        this.keyspace = keyspace;
-    }
-
-    public String getColumnFamily() throws InvalidRequestException
-    {
-        return findIndexedCF().cfName;
-    }
-
-    public CFMetaData generateCFMetadataUpdate() throws InvalidRequestException
-    {
-        return updateCFMetadata(findIndexedCF());
-    }
-
-    private CFMetaData updateCFMetadata(CFMetaData cfm)
-    {
-        ColumnDefinition column = findIndexedColumn(cfm);
-        assert column != null;
-        CFMetaData cloned = cfm.clone();
-        ColumnDefinition toChange = cloned.getColumnDefinition(column.name);
-        assert toChange.getIndexName() != null && toChange.getIndexName().equals(indexName);
-        toChange.setIndexName(null);
-        toChange.setIndexType(null, null);
-        return cloned;
-    }
-
-    private CFMetaData findIndexedCF() throws InvalidRequestException
-    {
-        KSMetaData ksm = Schema.instance.getKSMetaData(keyspace);
-        for (CFMetaData cfm : ksm.cfMetaData().values())
-        {
-            if (findIndexedColumn(cfm) != null)
-                return cfm;
-        }
-        throw new InvalidRequestException("Index '" + indexName + "' could not be found in any of the column families of keyspace '" + keyspace + "'");
-    }
-
-    private ColumnDefinition findIndexedColumn(CFMetaData cfm)
-    {
-        for (ColumnDefinition column : cfm.regularColumns())
-        {
-            if (column.getIndexType() != null && column.getIndexName() != null && column.getIndexName().equals(indexName))
-                return column;
-        }
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/cql/Operation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/Operation.java b/src/java/org/apache/cassandra/cql/Operation.java
deleted file mode 100644
index 0f06433..0000000
--- a/src/java/org/apache/cassandra/cql/Operation.java
+++ /dev/null
@@ -1,55 +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.cql;
-
-public class Operation
-{
-    public static enum OperationType
-    { PLUS, MINUS }
-
-    public final OperationType type;
-    public final Term a, b;
-
-    // unary operation
-    public Operation(Term a)
-    {
-        this.a = a;
-        type = null;
-        b = null;
-    }
-
-    // binary operation
-    public Operation(Term a, OperationType type, Term b)
-    {
-        this.a = a;
-        this.type = type;
-        this.b = b;
-    }
-
-    public boolean isUnary()
-    {
-        return type == null && b == null;
-    }
-
-    public String toString()
-    {
-        return (isUnary())
-                ? String.format("UnaryOperation(%s)", a)
-                : String.format("BinaryOperation(%s, %s, %s)", a, type, b);
-    }
-}


[4/4] git commit: Remove CQL2

Posted by al...@apache.org.
Remove CQL2

patch by Aleksey Yeschenko; reviewed by Jonathan Ellis for
CASSANDRA-5918


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

Branch: refs/heads/trunk
Commit: 26217071280f909f81fc3711df406a572b547c58
Parents: 3d7c5fd
Author: Aleksey Yeschenko <al...@yeschenko.com>
Authored: Tue Feb 18 06:50:15 2014 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Thu Feb 20 22:36:28 2014 +0300

----------------------------------------------------------------------
 CHANGES.txt                                     |   4 +
 NEWS.txt                                        |   9 +
 build.xml                                       |  21 +-
 doc/cql/CQL.css                                 |  48 -
 doc/cql/CQL.textile                             | 712 ---------------
 interface/cassandra.thrift                      |   8 +-
 .../org/apache/cassandra/thrift/Cassandra.java  |   6 +-
 .../cassandra/thrift/cassandraConstants.java    |   2 +-
 pylib/cqlshlib/helptopics.py                    |   3 +-
 .../org/apache/cassandra/config/CFMetaData.java |  11 -
 .../cassandra/config/ColumnDefinition.java      |   2 +-
 .../cassandra/cql/AbstractModification.java     | 126 ---
 .../cassandra/cql/AlterTableStatement.java      | 202 ----
 .../org/apache/cassandra/cql/Attributes.java    |  76 --
 .../apache/cassandra/cql/BatchStatement.java    | 104 ---
 .../org/apache/cassandra/cql/CFPropDefs.java    | 307 -------
 .../org/apache/cassandra/cql/CQLStatement.java  |  32 -
 src/java/org/apache/cassandra/cql/Cql.g         | 656 -------------
 .../cql/CreateColumnFamilyStatement.java        | 253 -----
 .../cassandra/cql/CreateIndexStatement.java     |  51 --
 .../cassandra/cql/CreateKeyspaceStatement.java  |  80 --
 .../apache/cassandra/cql/DeleteStatement.java   | 123 ---
 .../cassandra/cql/DropIndexStatement.java       |  80 --
 .../org/apache/cassandra/cql/Operation.java     |  55 --
 .../apache/cassandra/cql/QueryProcessor.java    | 913 -------------------
 src/java/org/apache/cassandra/cql/Relation.java |  86 --
 .../apache/cassandra/cql/SelectExpression.java  | 164 ----
 .../apache/cassandra/cql/SelectStatement.java   | 188 ----
 .../org/apache/cassandra/cql/StatementType.java |  32 -
 src/java/org/apache/cassandra/cql/Term.java     | 208 -----
 .../apache/cassandra/cql/UpdateStatement.java   | 288 ------
 .../org/apache/cassandra/cql/WhereClause.java   | 192 ----
 .../cassandra/cql/hooks/ExecutionContext.java   |  48 -
 .../cql/hooks/PostPreparationHook.java          |  38 -
 .../cassandra/cql/hooks/PreExecutionHook.java   |  46 -
 .../cassandra/cql/hooks/PreparationContext.java |  39 -
 .../apache/cassandra/cql3/QueryProcessor.java   |   1 -
 .../apache/cassandra/db/composites/CType.java   |   6 +-
 .../cassandra/db/composites/CellNameType.java   |   2 +-
 .../cassandra/db/marshal/AbstractType.java      |   6 -
 .../cassandra/db/marshal/TimeUUIDType.java      |  35 -
 .../apache/cassandra/db/marshal/UUIDType.java   |  41 +-
 .../apache/cassandra/service/ClientState.java   |  45 +-
 .../cassandra/thrift/CassandraServer.java       | 165 +---
 .../cassandra/thrift/ThriftClientState.java     |  18 -
 .../transport/messages/StartupMessage.java      |  11 +-
 .../cassandra/triggers/TriggerExecutor.java     |   7 +-
 .../org/apache/cassandra/utils/UUIDGen.java     |  21 -
 .../org/apache/cassandra/stress/Operation.java  |   4 -
 .../stress/operations/CqlCounterAdder.java      |   7 +-
 .../stress/operations/CqlCounterGetter.java     |  16 +-
 .../operations/CqlIndexedRangeSlicer.java       |  15 +-
 .../stress/operations/CqlInserter.java          |   9 +-
 .../stress/operations/CqlOperation.java         |  61 +-
 .../stress/operations/CqlRangeSlicer.java       |   8 +-
 .../cassandra/stress/operations/CqlReader.java  |   9 +-
 .../cassandra/stress/settings/CqlVersion.java   |   8 -
 .../cassandra/stress/settings/Legacy.java       |   5 +-
 .../cassandra/stress/settings/SettingsMode.java |  24 +-
 59 files changed, 74 insertions(+), 5663 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index efc54f8..755dd9a 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,3 +1,7 @@
+3.0
+ * Remove CQL2 (CASSANDRA-5918)
+
+
 2.1.0-beta2
  * Fix ABTC NPE (CASSANDRA-6692)
  * Allow nodetool to use a file or prompt for password (CASSANDRA-6660)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 865830c..b2d8377 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -13,6 +13,15 @@ restore snapshots created with the previous major version using the
 'sstableloader' tool. You can upgrade the file format of your snapshots
 using the provided 'sstableupgrade' tool.
 
+3.0
+===
+
+Upgrading
+---------
+   - CQL2 has been removed entirely in this release (previously deprecated
+     in 2.0.0). Please switch to CQL3 if you haven't already done so.
+
+
 2.1
 ===
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index 7afe3bf..3a5f8bd 100644
--- a/build.xml
+++ b/build.xml
@@ -202,28 +202,12 @@
     <!--
        This generates the CQL grammar files from Cql.g
     -->
-    <target name="check-gen-cql2-grammar">
-        <uptodate property="cql2current"
-                srcfile="${build.src.java}/org/apache/cassandra/cql/Cql.g"
-                targetfile="${build.src.gen-java}/org/apache/cassandra/cql/Cql.tokens"/>
-    </target>
     <target name="check-gen-cql3-grammar">
         <uptodate property="cql3current"
                 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-cql2-grammar" depends="check-gen-cql2-grammar" unless="cql2current">
-      <echo>Building Grammar ${build.src.java}/org/apache/cassandra/cql/Cql.g  ...</echo>
-      <java classname="org.antlr.Tool"
-            classpath="${build.lib}/antlr-3.2.jar"
-            fork="true"
-            failonerror="true">
-         <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>
-    </target>
     <target name="gen-cql3-grammar" depends="check-gen-cql3-grammar" unless="cql3current">
       <echo>Building Grammar ${build.src.java}/org/apache/cassandra/cql3/Cql.g  ...</echo>
       <java classname="org.antlr.Tool"
@@ -244,9 +228,6 @@
         <taskdef classpathref="wikitext.classpath" resource="wikitexttasks.properties" />
         <wikitext-to-html markupLanguage="Textile">
             <fileset dir="${basedir}">
-                <include name="doc/cql/*.textile"/>
-            </fileset>
-            <fileset dir="${basedir}">
                 <include name="doc/cql3/*.textile"/>
             </fileset>
         </wikitext-to-html>
@@ -672,7 +653,7 @@
         depends="maven-ant-tasks-retrieve-build,build-project" description="Compile Cassandra classes"/>
     <target name="codecoverage" depends="cobertura-instrument,test,cobertura-report" description="Create code coverage report"/>
 
-    <target depends="init,gen-cli-grammar,gen-cql2-grammar,gen-cql3-grammar"
+    <target depends="init,gen-cli-grammar,gen-cql3-grammar"
             name="build-project">
         <echo message="${ant.project.name}: ${ant.file}"/>
         <!-- Order matters! -->

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/doc/cql/CQL.css
----------------------------------------------------------------------
diff --git a/doc/cql/CQL.css b/doc/cql/CQL.css
deleted file mode 100644
index 2e4a41c..0000000
--- a/doc/cql/CQL.css
+++ /dev/null
@@ -1,48 +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.
- * 
- */
-
-/* Just a sample of some things that might be helpful in rendering CQL.html */
-
-pre.sample {
-  padding: 1ex;
-  border: 1px solid black;
-  background-color: #ffe0e0;
-}
-pre.syntax {
-  padding: 1ex;
-  border: 1px solid black;
-  background-color: #e0e0ff;
-}
-table {
-  width: 80%;
-  border-collapse: collapse;
-  border: 1px solid black;
-}
-td {
-  padding: 2px 1ex;
-  border: 1px solid black;
-}
-body {
-  background-color: white;
-}
-h2 {
-  margin-top: 3em;
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/doc/cql/CQL.textile
----------------------------------------------------------------------
diff --git a/doc/cql/CQL.textile b/doc/cql/CQL.textile
deleted file mode 100644
index 1cfe883..0000000
--- a/doc/cql/CQL.textile
+++ /dev/null
@@ -1,712 +0,0 @@
-==<!-- This is only a suggestion, to demonstrate that the pre.syntax and pre.sample classes should probably be rendered differently. Render as you like. --><link rel="StyleSheet" href="CQL.css" type="text/css" media="screen">==
-
-h1. Cassandra Query Language (CQL) v2.0
-
-h2. Table of Contents
-
-{toc}
-
-h2. Syntax conventions
-
-To aid in specifying the CQL syntax, we will use the following conventions in this document:
-
-* Language rules will be given in a BNF(Backus-Naur Form)-like notation, looking like this:
-
-bc(syntax). 
-<Start> ::= <CQL_Statement>*
-          ;
-
-* Nonterminal symbols in syntax rules will have @<angle brackets>@.
-* Terminal symbols will be shown in @"single quotes"@.
-* As an additional shortcut notation to BNF, optional symbols (that can occur zero or one times) will be followed by a @?@ marker. Optional symbols that can occur zero or any number of times will be followed by a @*@ marker. Multiple symbols may be grouped together in @(parentheses)@ to signify that they are all optional or repeatable together.
-* In a few cases where meaning is more easily conveyed through prose, we explain a symbol and its expansion without BNF.
-* Sample code will also be shown in a code block:
-
-bc(sample). 
-SELECT sample_usage FROM cql;
-
-* References to keywords or pieces of CQL code in running text will be shown in a @fixed-width font@.
-
-h2. Overall syntax
-
-CQL consists of statements. As in SQL, some statements directly make changes to data, some look up data, and some change the way data is stored.
-
-All statements end with a semicolon.
-
-bc(syntax). 
-<CQL_Statement> ::= <statementBody> ";"
-                  ;
-<statementBody> ::= <useStatement>
-                  | <selectStatement>
-                  | <dataChangeStatement>
-                  | <schemaChangeStatement>
-                  ;
-<dataChangeStatement> ::= <insertStatement>
-                        | <updateStatement>
-                        | <batchStatement>
-                        | <deleteStatement>
-                        | <truncateStatement>
-                        ;
-<schemaChangeStatement> ::= <createKeyspaceStatement>
-                          | <createColumnFamilyStatement>
-                          | <createIndexStatement>
-                          | <dropKeyspaceStatement>
-                          | <dropColumnFamilyStatement>
-                          | <dropIndexStatement>
-                          | <alterTableStatement>
-                          ;
-
-String literals and identifiers (including keyspace and column family names) are case-sensitive, but CQL keywords are not. We show CQL keywords in this document in @UPPERCASE@ merely as a convention to aid readability.
-
-Literal values can be expressed in several ways in CQL.
-
-bc(syntax). 
-<term> ::= "KEY"
-         | <identifier>
-         | <stringLiteral>
-         | <integer>
-         | <float>
-         | <uuid>
-         ;
-
-* An @<identifier>@ is a letter followed by any sequence of letters, digits, or the underscore (@_@).
-* A @<stringLiteral>@ is encased in @'single quotes'@. A single quote itself can be represented in a string literal by doubling it, as in SQL: @'Single quote -> '' <-'@.
-* An @<integer>@ consists of an optional minus sign (@-@) followed by one or more digits (@0-9@).
-* A @<uuid>@ can be expressed in the canonical UUID form: 32 hex digits (@0-9@ or @a-f@, case insensitive), separated by dashes (@-@) after the 8th, 12th, 16th, and 20th digits. Example: @01234567-0123-0123-0123-0123456789ab@
-* A @<float>@ is a series of one or more decimal digits, followed by a period (@.@), and one or more decimal digits following. Note that there is no provision for "e" notation, no optional @+@ sign, and the forms @.42@ and @42.@ are not accepted. Use @0.42@ and @42.0@.
-* Whitespace is not significant except to separate terms, and inside string literals.
-
-Comments in CQL can begin with a double dash (@--@) or a double slash (@//@) and extend to the end of the line. Multiline comments are enclosed in @/* ... */@.
-
-h2(#storageTypes). Data Storage Types
-
-__Syntax:__
-
-bc(syntax). 
-<storageType> ::= "ascii"
-                | "bigint"
-                | "blob"
-                | "boolean"
-                | "counter"
-                | "decimal"
-                | "double"
-                | "float"
-                | "int"
-                | "text"
-                | "timestamp"
-                | "uuid"
-                | "varchar"
-                | "varint"
-                ;
-
-The following table gives additional information on the available 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 long)|
-|decimal|Variable-precision decimal|
-|double|64-bit IEEE-754 floating point|
-|float|32-bit IEEE-754 floating point|
-|int|32-bit signed int|
-|text|UTF8 encoded string|
-|timestamp|A timestamp. See "Working with dates":#usingdates below for more information.|
-|uuid|Type 1 or type 4 UUID|
-|varchar|UTF8 encoded string|
-|varint|Arbitrary-precision integer|
-
-_Note: In addition to the recognized types listed above, it is also possible to supply a string containing the name of a class (a sub-class of @AbstractType@ loadable by Cassandra). The class name should either be fully qualified, or relative to the @org.apache.cassandra.db.marshal@ package._
-
-h3(#usingdates). Working with dates
-
-Values serialized with the @timestamp@ type are encoded as 64-bit signed integers representing a number of milliseconds since the standard base time known as "the epoch": January 1 1970 at 00:00:00 GMT.
-
-Timestamp types can be input in CQL as simple long integers, giving the number of milliseconds since the epoch, as defined above.
-
-Timestamp types can also be input as string literals in any of the following ISO 8601 formats, each representing the time and date Jan 2, 2003, at 04:05:00 AM, GMT.:
-
-* @2011-02-03 04:05+0000@
-* @2011-02-03 04:05:00+0000@
-* @2011-02-03T04:05+0000@
-* @2011-02-03T04:05:00+0000@
-
-The @+0000@ above is an RFC 822 4-digit time zone specification; @+0000@ refers to GMT. US Pacific Standard Time is @-0800@. The time zone may be omitted if desired-- the date will be interpreted as being in the time zone under which the coordinating Cassandra node is configured.
-
-* @2011-02-03 04:05@
-* @2011-02-03 04:05:00@
-* @2011-02-03T04:05@
-* @2011-02-03T04:05:00@
-
-There are clear difficulties inherent in relying on the time zone configuration being as expected, though, so it is recommended that the time zone always be specified for timestamps when feasible.
-
-The time of day may also be omitted, if the date is the only piece that matters:
-
-* @2011-02-03@
-* @2011-02-03+0000@
-
-In that case, the time of day will default to 00:00:00, in the specified or default time zone.
-
-h2. USE
-
-__Syntax:__
-
-bc(syntax). 
-<useStatement> ::= "USE" <term>
-                 ;
-
-__Sample:__
-
-bc(sample). 
-USE myApp;
-
-A @USE@ statement consists of the @USE@ keyword, followed by a valid keyspace name.  Its purpose is to assign the per-connection, current working keyspace.  All subsequent keyspace-specific actions will be performed in the context of the keyspace selected, unless otherwise specified, until another USE statement is issued or the connection terminates.
-
-h2. SELECT
-
-__Syntax:__
-
-bc(syntax). 
-<selectStatement> ::= "SELECT" <whatToSelect>
-                        "FROM" ( <name> "." )? <name>
-                               ( "USING" "CONSISTENCY" <consistencylevel> )?
-                               ( "WHERE" <selectWhereClause> )?
-                               ( "LIMIT" <integer> )?
-                    ;
-<whatToSelect> ::= <term> ( "," <term> )*
-                 | ("FIRST" <integer> )? "REVERSED"? <columnRange>
-                 | "COUNT" "(" <countTarget> ")"
-                 ;
-<columnRange> ::= <term> ".." <term>
-                | "*"
-                ;
-<countTarget> ::= "*"
-                | "1"
-                ;
-<name> ::= <identifier>
-         | <stringLiteral>
-         | <integer>
-         ;
-<selectWhereClause> ::= <relation> ( "AND" <relation> )*
-                      | <term> "IN" "(" <term> ( "," <term> )* ")"
-                      ;
-<relation> ::= <term> <relationOperator> <term>
-             ;
-<relationOperator> ::= "=" | "<" | ">" | "<=" | ">="
-                     ;
-
-__Sample:__
-
-bc(sample). 
-SELECT Name, Occupation FROM People WHERE key IN (199, 200, 207);
-SELECT FIRST 3 REVERSED 'time199'..'time100' FROM Events;
-SELECT COUNT(*) FROM system.Migrations;
-
-A @SELECT@ is used to read one or more records from a Cassandra column family. It returns a result-set of rows, where each row consists of a key and a collection of columns corresponding to the query.
-
-h3. Specifying Columns
-
-bc(sample). 
-SELECT col1, col2 FROM ...
-SELECT range_lo..range_hi FROM ...
-SELECT * FROM ...
-SELECT FIRST 4 REVERSED range_hi..range_lo FROM ...
-
-The @SELECT@ expression determines which columns will appear in the results and can take a few different forms, as shown above. The simplest is a comma-separated list of column names. Note that column names in Cassandra can be specified with string literals or integers, in addition to identifiers.
-
-It is also possible to specify a range of column names. The range notation consists of start and end column names, separated by two periods (@..@). The set of columns returned for a range is start and end inclusive. A single star (@*@) may be used as a range to request "all columns".
-
-When using a range, it is sometimes useful to limit the number of columns that can be returned as part of each row (since Cassandra is schemaless, it is not necessarily possible to determine ahead of time how many columns will be in the result set). To accomplish this, use the @FIRST@ clause with an integer to specify an upper limit on the number of columns returned per row. The default limit is 10,000 columns.
-
-The @REVERSED@ option causes the sort order of the columns returned to be reversed. This affects the @FIRST@ clause; when limiting the columns returned, the columns at the end of the range will be selected instead of the ones at the beginning of the range.
-
-A @SELECT@ expression may also be @COUNT(*)@. In this case, the result will be only one value: the number of rows which matched the query.
-
-It is worth noting that unlike the projection in a SQL SELECT, there is no guarantee that the results will contain all of the columns specified, because Cassandra is schemaless.
-
-h3. Column Family
-
-bc(sample). 
-SELECT ... FROM MyApp.LocationSnapshots ...;
-SELECT ... FROM EventTimeline ...;
-
-The @FROM@ clause is used to specify the Cassandra column family applicable to a @SELECT@ query. Unlike other operations on column families, the keyspace in which the column family exists may also be specified by giving its name before the column family name, and separating them by a dot (@.@). If the keyspace is not specified, the current keyspace will be used, as per normal.
-
-h3. Consistency Level
-
-bc(sample). 
-SELECT ... USING CONSISTENCY QUORUM;
-
-Following the column family clause is an optional "consistency level specification":#consistency.
-
-h3. Filtering rows
-
-bc(sample). 
-SELECT ... WHERE KEY = 11194251 AND startdate = '2011-10-08-0500';
-SELECT ... WHERE KEY >= 'AM' and KEY =< 'AZ' AND module = 17;
-SELECT ... WHERE keyalias IN ('key1', 'key2', 'key3', ...);
-
-The @WHERE@ clause provides for filtering the rows that appear in results.  The clause can filter on a key name, or range of keys, and in the case of indexed columns, on column values.  Key filters are specified using the @KEY@ keyword or key alias name, followed by a relational operator (one of @=@, @>@, @>=@, @<@, and @<=@), and then a term value.  When terms appear on both sides of a relational operator it is assumed the filter applies to an indexed column. With column index filters, the term on the left of the operator must be the name of the indexed column, and the term on the right is the value to filter __on__.
-
-__Note: The greater-than and less-than operators (@>@ and @<@) result in key ranges that are inclusive of the terms. There is no supported notion of "strictly" greater-than or less-than; these operators are merely supported as aliases to @>=@ and @<=@.__
-
-h3. Limits
-
-bc(sample). 
-SELECT ... WHERE favoriteArtist = 'The Mighty Mighty Bosstones' LIMIT 90000;
-
-The @LIMIT@ option to a @SELECT@ expression limits the number of rows returned by a query. @LIMIT@ defaults to 10,000 when left unset.
-
-h2. INSERT
-
-__Syntax:__
-
-bc(syntax). 
-<insertStatement> ::= "INSERT" "INTO" <name>
-                               "(" <term> "," <term> ( "," <term> )* ")"
-                      "VALUES" "(" <term> "," <term> ( "," <term> )* ")"
-                      ( "USING" <usingOption> ( "AND" <usingOption> )* )?
-                    ;
-<usingOption> ::= "CONSISTENCY" <consistencylevel>
-                | "TIMESTAMP" <integer>
-                | "TTL" <integer>
-                ;
-
-__Sample:__
-
-bc(sample). 
-INSERT INTO NerdMovies (KEY, 11924)
-                VALUES ('Serenity', 'Nathan Fillion')
-      USING CONSISTENCY LOCAL_QUORUM AND TTL 86400;
-
-An @INSERT@ is used to write one or more columns to a record in a Cassandra column family. No results are returned.
-
-The first column name in the @INSERT@ list must be the name of the column family key. Also, there must be more than one column name specified (Cassandra rows are not considered to exist with only a key and no associated columns).
-
-Unlike in SQL, the semantics of @INSERT@ and @UPDATE@ are identical. In either case a record is created if none existed before, and updated when it does.  For information on query modifiers and types, see the "@UPDATE@":#update section below.
-
-h2(#update). UPDATE
-
-__Syntax:__
-
-bc(syntax). 
-<updateStatement> ::= "UPDATE" <name>
-                        ( "USING" <usingOption> ( "AND" <usingOption> )* )?
-                        "SET" <assignment> ( "," <assignment> )*
-                        "WHERE" <updateWhereClause>
-                    ;
-<assignment> ::= <term> "=" <term>
-               | <term> "=" <term> "+" <term>
-               | <term> "=" <term> "-" <term>
-               ;
-<updateWhereClause> ::= <term> "=" <term>
-                      | <term> "IN" "(" <term> ( "," <term> )* ")"
-                      ;
-
-__Sample:__
-
-bc(sample). 
-UPDATE NerdMovies USING CONSISTENCY ALL AND TTL 400
-       SET 'A 1194' = 'The Empire Strikes Back',
-           'B 1194' = 'Han Solo'
-     WHERE KEY = B70DE1D0-9908-4AE3-BE34-5573E5B09F14;
-UPDATE UserActionCounts SET total = total + 2 WHERE keyalias = 523;
-
-An @UPDATE@ is used to write one or more columns to a record in a Cassandra column family. No results are returned. The row key can be specified using the @KEY@ keyword or by a key alias set per column family.
-
-h3. Column Family
-
-Statements begin with the @UPDATE@ keyword followed by a Cassandra column family name.
-
-h3. Consistency Level
-
-bc(sample). 
-UPDATE Foo USING CONSISTENCY EACH_QUORUM ...
-
-Following the column family identifier is an optional @USING@ clause, which can specify the "consistency level":#consistency for the update, or the timestamp and/or the TTL for the new columns.
-
-h3. Timestamp
-
-bc(sample). 
-UPDATE Foo USING TIMESTAMP=1318452291034 ...
-
-@UPDATE@ supports setting client-supplied optional timestamp for modification.
-
-h3. TTL
-
-bc(sample). 
-UPDATE Foo USING TTL=6800 ...
-
-@UPDATE@ supports setting a time to live (TTL), in seconds, for each of the added columns.
-
-h3. Specifying Columns and Row
-
-bc(sample). 
-UPDATE ... SET col1 = val1, col2 = val2 WHERE KEY = key1;
-UPDATE ... SET col3 = val3 WHERE KEY IN (key1, key2, key3);
-UPDATE ... SET col4 = 22 WHERE keyalias = key4;
-
-Rows are created or updated by supplying column names and values, after the @SET@ keyword, in term assignment format. Multiple columns can be set by separating the name/value pairs using commas.  Each update statement requires a precise set of row keys to be specified using a @WHERE@ clause and the @KEY@ keyword or key alias.
-
-h3. Updating Counter Columns
-
-bc(sample). 
-UPDATE ... SET name1 = name1 + <value> ...;
-UPDATE ... SET name1 = name1 - <value> ...;
-
-Counter columns can be incremented or decremented by an arbitrary numeric value though the assignment of an expression that adds or substracts the value.
-
-h2. DELETE
-
-__Syntax:__
-
-bc(syntax). 
-<deleteStatement> ::= "DELETE" ( <term> ( "," <term> )* )?
-                        "FROM" <name>
-                        ( "USING" <deleteOption> ( "AND" <deleteOption> )* )?
-                        "WHERE" <updateWhereClause>
-                    ;
-<deleteOption> ::= "CONSISTENCY" <consistencylevel>
-                 | "TIMESTAMP" <integer>
-                 ;
-
-__Sample:__
-
-bc(sample). 
-DELETE col1, col2, col3 FROM Planeteers USING CONSISTENCY ONE WHERE KEY = 'Captain';
-DELETE FROM MastersOfTheUniverse WHERE KEY IN ('Man-At-Arms', 'Teela');
-
-A @DELETE@ is used to perform the removal of one or more columns from one or more rows. The key can be given using the @KEY@ keyword or by the key alias set per column family.
-
-h3. Specifying Columns
-
-Following the @DELETE@ keyword is an optional comma-delimited list of column name terms. When no column names are specified, the remove applies to the entire row(s) matched by the "WHERE clause":#deleterows.
-
-h3. Column Family
-
-The column family name follows the list of column names and the keyword @FROM@.
-
-h3. Consistency Level
-
-Following the column family identifier is an optional "consistency level specification":#consistency.
-
-h3(#deleterows). Specifying Rows
-
-bc(sample). 
-DELETE ... WHERE KEY = 'some_key_value';
-DELETE ... WHERE keyalias IN (key1, key2);
-
-The @WHERE@ clause is used to determine to which row(s) a @DELETE@ applies. The first form allows the specification of a single keyname using the @KEY@ keyword (or by key alias) and the @=@ operator.  The second form allows a list of keyname terms to be specified using the @IN@ notation and a parenthesized list of comma-delimited keyname terms.
-
-h2. TRUNCATE
-
-__Syntax:__
-
-bc(syntax). 
-<truncateStatement> ::= "TRUNCATE" <name>
-                      ;
-
-__Sample:__
-
-bc(sample). 
-TRUNCATE super_important_data;
-
-@TRUNCATE@ accepts a single argument for the column family name, and permanently removes all data from said column family.
-
-h2. BATCH
-
-__Syntax:__
-
-bc(syntax). 
-<batchStatement> ::= "BEGIN" "BATCH"
-                        ( "USING" <usingOption> ( "AND" <usingOption> )* )?
-                        <batchStatementMember> ( ";" <batchStatementMember> )*
-                     "APPLY" "BATCH"
-                   ;
-<batchStatementMember> ::= <insertStatement>
-                         | <updateStatement>
-                         | <deleteStatement>
-                         ;
-
-__Sample:__
-
-bc(sample). 
-BEGIN BATCH USING CONSISTENCY QUORUM AND TTL 8640000
-  INSERT INTO users (KEY, password, name) VALUES ('user2', 'ch@ngem3b', 'second user')
-  UPDATE users SET password = 'ps22dhds' WHERE KEY = 'user2'
-  INSERT INTO users (KEY, password) VALUES ('user3', 'ch@ngem3c')
-  DELETE name FROM users WHERE key = 'user2'
-  INSERT INTO users (KEY, password, name) VALUES ('user4', 'ch@ngem3c', 'Andrew')
-APPLY BATCH;
-
-@BATCH@ supports setting a client-supplied optional global timestamp which will be used for each of the operations included in the batch.
-
-A single consistency level is used for the entire batch. It appears after the @BEGIN BATCH@ statement, and uses the standard "consistency level specification":#consistency. Batched statements default to @CONSISTENCY.ONE@ when left unspecified.
-
-Only data modification statements (specifically, @UPDATE@, @INSERT@, and @DELETE@) are allowed in a @BATCH@ statement. @BATCH@ is _not_ an analogue for SQL transactions.
-
-_NOTE: While there are no isolation guarantees, @UPDATE@ queries are atomic within a given record._
-
-h2. CREATE KEYSPACE
-
-__Syntax:__
-
-bc(syntax). 
-<createKeyspaceStatement> ::= "CREATE" "KEYSPACE" <name>
-                                 "WITH" <optionName> "=" <optionVal>
-                                 ( "AND" <optionName> "=" <optionVal> )*
-                            ;
-<optionName> ::= <identifier>
-               | <optionName> ":" <identifier>
-               | <optionName> ":" <integer>
-               ;
-<optionVal> ::= <stringLiteral>
-              | <identifier>
-              | <integer>
-              ;
-
-__Sample:__
-
-bc(sample). 
-CREATE KEYSPACE Excelsior WITH strategy_class = 'SimpleStrategy'
-    AND strategy_options:replication_factor = 1;
-CREATE KEYSPACE Excalibur WITH strategy_class = 'NetworkTopologyStrategy'
-    AND strategy_options:DC1 = 1 AND strategy_options:DC2 = 3;
-
-The @CREATE KEYSPACE@ statement creates a new top-level namespace (aka "keyspace"). Valid names are any string constructed of alphanumeric characters and underscores. Names which do not work as valid identifiers or integers should be quoted as string literals. Properties such as replication strategy and count are specified during creation using the following accepted keyword arguments:
-
-|_. keyword|_. required|_. description|
-|strategy_class|yes|The name of the replication strategy class which should be used for the new keyspace. Some often-used classes are @SimpleStrategy@ and @NetworkTopologyStrategy@.|
-|strategy_options|no|Most strategies require additional arguments which can be supplied by appending the option name to the @strategy_options@ keyword, separated by a colon (@:@).  For example, a strategy option of "DC1" with a value of "1" would be specified as @strategy_options:DC1 = 1@; replication_factor for SimpleStrategy could be @strategy_options:replication_factor=3@.|
-
-h2. CREATE COLUMNFAMILY
-
-__Syntax:__
-
-bc(syntax). 
-<createColumnFamilyStatement> ::= "CREATE" "COLUMNFAMILY" <name>
-                                    "(" <term> <storageType> "PRIMARY" "KEY"
-                                        ( "," <term> <storageType> )* ")"
-                                   ( "WITH" <optionName> "=" <cfOptionVal>
-                                     ( "AND" <optionName> "=" <cfOptionVal> )* )?
-                                ;
-<optionName> ::= <identifier>
-               | <optionName> ":" <identifier>
-               | <optionName> ":" <integer>
-               ;
-
-<cfOptionVal> ::= <storageType>
-                | <identifier>
-                | <stringLiteral>
-                | <integer>
-                | <float>
-                ;
-
-__Sample:__
-
-bc(sample). 
-CREATE COLUMNFAMILY Fish (KEY blob PRIMARY KEY);
-CREATE COLUMNFAMILY FastFoodEatings (user text PRIMARY KEY)
-    WITH comparator=timestamp AND default_validation=int;
-CREATE COLUMNFAMILY MonkeyTypes (
-    KEY uuid PRIMARY KEY,
-    species text,
-    alias text,
-    population varint
-) WITH comment='Important biological records'
-   AND read_repair_chance = 1.0;
-
-@CREATE COLUMNFAMILY@ statements create new column family namespaces under the current keyspace. Valid column family names are strings of alphanumeric characters and underscores, which begin with a letter.
-
-h3(#keytypes). Specifying Key Type
-
-bc(sample). 
-CREATE ... (KEY ascii PRIMARY KEY, ... ) ...
-
-When creating a new column family, you must specify the key type. The list of possible types is identical to column comparators/validators (see "Data Storage Types":#storageTypes), except it probably does not make sense to use @counter@ for a key. It's important to note that the key type you use must be compatible with the partitioner in use. For example, @OrderPreservingPartitioner@ and @CollatingOrderPreservingPartitioner@ both require UTF-8 keys. If you use an identifier for the primary key name, instead of the @KEY@ keyword, a key alias will be set automatically.
-
-h3. Specifying Column Types (optional)
-
-bc(sample). 
-CREATE ... ( ... , name1 type1, name2 type2, ... ) ...
-
-It is possible to assign columns a type during column family creation. Columns configured with a type are validated accordingly when a write occurs, and intelligent CQL drivers and interfaces will be able to decode the column values correctly when receiving them. Column types are specified as a parenthesized, comma-separated list of column term and type pairs. See "Data Storage Types":#storageTypes for the list of recognized types.
-
-h3(#cfopts). Column Family Options (optional)
-
-bc(sample). 
-CREATE COLUMNFAMILY ... WITH keyword1 = arg1 AND keyword2 = arg2;
-
-A number of optional keyword arguments can be supplied to control the configuration of a new column family.
-
-|_. keyword|_. default|_. description|
-|comparator|text|Determines the storage type of column names (which itself determines the sorting and validation of column names). Valid values are listed in the "Data Storage Types":#storageTypes table above.|
-|comment|none|A free-form, human-readable comment.|
-|read_repair_chance|1.0|The probability with which read repairs should be invoked on non-quorum reads.|
-|gc_grace_seconds|864000|Time to wait before garbage collecting tombstones (deletion markers).|
-|default_validation|text|Determines the default storage type of column values (which itself determines the validation for column values). This option does not affect the types of columns which were defined in a @CREATE COLUMNFAMILY@ statement-- only new columns. Valid values are listed in the "Data Storage Types":#storageTypes table above.|
-|min_compaction_threshold|4|Minimum number of SSTables needed to start a minor compaction.|
-|max_compaction_threshold|32|Maximum number of SSTables allowed before a minor compaction is forced.|
-|replicate_on_write|false| |
-|compaction_strategy_options|none|CompactionStrategy specific options such as "sstable_size_in_mb" for LeveledCompactionStrategy and "min_sstable_size" for SizeTieredCompactionStrategy|
-|compression_parameters|none|Compression parameters such as "sstable_compressor" and "chunk_length_kb"|
-
-h2. CREATE INDEX
-
-__Syntax:__
-
-bc(syntax). 
-<createIndexStatement> ::= "CREATE" "INDEX" <identifier>? "ON"
-                               <name> "(" <term> ")"
-                         ;
-
-__Sample:__
-
-bc(sample). 
-CREATE INDEX userIndex ON NerdMovies (user);
-CREATE INDEX ON Mutants (abilityId);
-
-A @CREATE INDEX@ statement is used to create a new, automatic secondary index on the given column family, for the named column. A name for the index itself can be specified before the @ON@ keyword, if desired. A single column name must be specified inside the parentheses. It is not necessary for the column to exist on any current rows (Cassandra is schemaless), but the column must already have a type (specified during the @CREATE COLUMNFAMILY@, or added afterwards with @ALTER COLUMNFAMILY@.
-
-h2. DROP KEYSPACE
-
-__Syntax:__
-
-bc(syntax). 
-<dropKeyspaceStatement> ::= "DROP" "KEYSPACE" <name>
-                          ;
-
-__Sample:__
-
-bc(sample). DROP KEYSPACE MyTwitterClone;
-
-A @DROP KEYSPACE@ statement results in the immediate, irreversible removal of a keyspace, including all column families in it, and all data contained in those column families.
-
-h2. DROP COLUMNFAMILY
-
-__Syntax:__
-
-bc(syntax). 
-<dropColumnFamilyStatement> ::= "DROP" "COLUMNFAMILY" <name>
-                              ;
-
-__Sample:__
-
-bc(sample). DROP COLUMNFAMILY worldSeriesAttendees;
-
-A @DROP COLUMNFAMILY@ statement results in the immediate, irreversible removal of a column family, including all data contained in it.
-
-h2. DROP INDEX
-
-__Syntax:__
-
-bc(syntax). 
-<dropIndexStatement> ::= "DROP" "INDEX" <name>
-                       ;
-
-__Sample:__
-
-bc(sample). DROP INDEX cf_col_idx;
-
-A @DROP INDEX@ statement is used to drop an existing secondary index.
-
-h2. ALTER COLUMNFAMILY
-
-__Syntax:__
-
-bc(syntax). 
-<alterTableStatement> ::= "ALTER" "COLUMNFAMILY" <name> <alterInstructions>
-                        ;
-<alterInstructions> ::= "ALTER" <name> "TYPE" <storageType>
-                      | "ADD" <name> <storageType>
-                      | "DROP" <name>
-                      | "WITH" <optionName> "=" <cfOptionVal>
-                               ( "AND" <optionName> "=" <cfOptionVal> )*
-                      ;
-
-An @ALTER@ statement is used to manipulate column family column metadata. It allows you to add new columns, drop existing columns, data storage type of existing columns, or change the column family properties. No results are returned.
-
-Specify the name of the column family to be changed after the @ALTER COLUMNFAMILY@ keywords, followed by the type of change desired (@ALTER@, @ADD@, @DROP@, or @WITH@), and provide the rest of the needed information, as explained below.
-
-h3. Changing the type of a typed column
-
-bc(sample). 
-ALTER COLUMNFAMILY addamsFamily ALTER lastKnownLocation TYPE uuid;
-
-@ALTER COLUMNFAMILY ... ALTER@ changes the expected storage type for a column. The column must either be the key alias or already have a type in the column family metadata. The column may or may not already exist in current rows-- but be aware that no validation of existing data is done. The bytes stored in values for that column will remain unchanged, and if existing data is not deserializable according to the new type, this may cause your CQL driver or interface to report errors.
-
-h3. Adding a typed column
-
-bc(sample). 
-ALTER COLUMNFAMILY addamsFamily ADD gravesite varchar;
-
-The @ALTER COLUMNFAMILY ... ADD@ variant adds a typed column to a column family. The column must not already have a type in the column family metadata. The same warnings from the above @ALTER@ section, about there being no validation of existing data, apply here as well.
-
-h3. Dropping a typed column
-
-bc(sample). 
-ALTER COLUMNFAMILY addamsFamily DROP gender;
-
-An @ALTER COLUMNFAMILY ... DROP@ statement removes the type of a column from the column family metadata. Note that this does _not_ remove the column from current rows; it just removes the metadata saying that the bytes stored under that column are expected to be deserializable according to a certain type.
-
-h3. Modifying column family properties
-
-bc(sample). 
-ALTER COLUMNFAMILY addamsFamily WITH comment = 'A most excellent and useful column family' AND read_repair_chance = 0.2;
-
-An @ALTER COLUMNFAMILY ... WITH@ statement makes adjustments to the column family properties, as defined when the column family was created (see "CREATE COLUMNFAMILY options":#cfopts for information about the supported options and values).
-
-Note that setting any @compaction_strategy_options:*@ parameters has the effect of erasing all previous @compaction_strategy_options:*@ parameters, so you will need to re-specify any such parameters which have already been set, if you want to keep them. The same note applies to the set of @compression_parameters:*@ parameters.
-
-h2. Common Idioms
-
-h3(#consistency). Specifying Consistency
-
-bc(syntax). 
-<consistency> ::= "ANY"
-                | "ONE"
-                | "QUORUM"
-                | "ALL"
-                | "LOCAL_QUORUM"
-                | "EACH_QUORUM"
-                ;
-
-bc(sample). 
-... USING CONSISTENCY LOCAL_QUORUM ...
-
-Consistency level specifications are made up the keywords @USING CONSISTENCY@, followed by a consistency level identifier. Valid consistency level identifiers are as listed above. When not specified, @USING CONSISTENCY ONE@ is the default.
-
-Consult your Cassandra documentation for information about how consistency levels work.
-
-h1. Versioning
-
-Versioning of the CQL language adheres to the "Semantic Versioning":http://semver.org guidelines.  Versions take the form X.Y.Z where X, Y, and Z are integer values representing major, minor, and patch level respectively.  There is no correlation between Cassandra release versions and the CQL language version.
-
-|_. version|_. description|
-|Major|The major version _must_ be bumped when backward incompatible changes are introduced.  This should rarely (if ever) occur.|
-|Minor|Minor version increments occur when new, but backward compatible, functionality is introduced.|
-|Patch|The patch version is incremented when bugs are fixed.|
-
-h1. Changes
-
-pre.. 
-Wed, 12 Oct 2011 16:53:00 -0500 - paul cannon
- * Rework whole doc, adding syntax specifics and additional explanations
-
-Fri, 09 Sep 2011 11:43:00 -0500 - Jonathan Ellis
- * add int data type
-
-pre. Wed, 07 Sep 2011 09:01:00 -0500 - Jonathan Ellis
- * Updated version to 2.0; Documented row-based count()
- * Updated list of supported data types
-
-pre. Wed, 10 Aug 2011 11:22:00 -0500 - Eric Evans
- * Improved INSERT vs. UPDATE wording.
- * Documented counter column incr/descr.
-
-pre. Sat, 01 Jun 2011 15:58:00 -0600 - Pavel Yaskevich
- * Updated to support ALTER (CASSANDRA-1709)
-
-pre. Tue, 22 Mar 2011 18:10:28 -0700 - Eric Evans <ee...@rackspace.com>
- * Initial version, 1.0.0

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/interface/cassandra.thrift
----------------------------------------------------------------------
diff --git a/interface/cassandra.thrift b/interface/cassandra.thrift
index a223eb5..e46b85e 100644
--- a/interface/cassandra.thrift
+++ b/interface/cassandra.thrift
@@ -55,7 +55,7 @@ namespace rb CassandraThrift
 # An effort should be made not to break forward-client-compatibility either
 # (e.g. one should avoid removing obsolete fields from the IDL), but no
 # guarantees in this respect are made by the Cassandra project.
-const string VERSION = "19.39.0"
+const string VERSION = "20.0.0"
 
 
 #
@@ -843,7 +843,7 @@ service Cassandra {
 
 
   /**
-   * @deprecated Will become a no-op in 2.2. Please use the CQL3 version instead.
+   * @deprecated Throws InvalidRequestException since 3.0. Please use the CQL3 version instead.
    */
   CqlResult execute_cql_query(1:required binary query, 2:required Compression compression)
     throws (1:InvalidRequestException ire,
@@ -863,7 +863,7 @@ service Cassandra {
 
 
   /**
-   * @deprecated Will become a no-op in 2.2. Please use the CQL3 version instead.
+   * @deprecated Throws InvalidRequestException since 3.0. Please use the CQL3 version instead.
    */
   CqlPreparedResult prepare_cql_query(1:required binary query, 2:required Compression compression)
     throws (1:InvalidRequestException ire)
@@ -879,7 +879,7 @@ service Cassandra {
 
 
   /**
-   * @deprecated Will become a no-op in 2.2. Please use the CQL3 version instead.
+   * @deprecated Throws InvalidRequestException since 3.0. Please use the CQL3 version instead.
    */
   CqlResult execute_prepared_cql_query(1:required i32 itemId, 2:required list<binary> values)
     throws (1:InvalidRequestException ire,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
index 15b99fa..f29805b 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
@@ -379,7 +379,7 @@ public class Cassandra {
     public String system_update_column_family(CfDef cf_def) throws InvalidRequestException, SchemaDisagreementException, org.apache.thrift.TException;
 
     /**
-     * @deprecated Will become a no-op in 2.2. Please use the CQL3 version instead.
+     * @deprecated Throws InvalidRequestException since 3.0. Please use the CQL3 version instead.
      * 
      * @param query
      * @param compression
@@ -397,7 +397,7 @@ public class Cassandra {
     public CqlResult execute_cql3_query(ByteBuffer query, Compression compression, ConsistencyLevel consistency) throws InvalidRequestException, UnavailableException, TimedOutException, SchemaDisagreementException, org.apache.thrift.TException;
 
     /**
-     * @deprecated Will become a no-op in 2.2. Please use the CQL3 version instead.
+     * @deprecated Throws InvalidRequestException since 3.0. Please use the CQL3 version instead.
      * 
      * @param query
      * @param compression
@@ -416,7 +416,7 @@ public class Cassandra {
     public CqlPreparedResult prepare_cql3_query(ByteBuffer query, Compression compression) throws InvalidRequestException, org.apache.thrift.TException;
 
     /**
-     * @deprecated Will become a no-op in 2.2. Please use the CQL3 version instead.
+     * @deprecated Throws InvalidRequestException since 3.0. Please use the CQL3 version instead.
      * 
      * @param itemId
      * @param values

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/interface/thrift/gen-java/org/apache/cassandra/thrift/cassandraConstants.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/cassandraConstants.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/cassandraConstants.java
index 81b0ffb..d9f0008 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/cassandraConstants.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/cassandraConstants.java
@@ -56,6 +56,6 @@ import org.slf4j.LoggerFactory;
 
 public class cassandraConstants {
 
-  public static final String VERSION = "19.39.0";
+  public static final String VERSION = "20.0.0";
 
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/pylib/cqlshlib/helptopics.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/helptopics.py b/pylib/cqlshlib/helptopics.py
index 710aa74..b8e5ba8 100644
--- a/pylib/cqlshlib/helptopics.py
+++ b/pylib/cqlshlib/helptopics.py
@@ -205,8 +205,7 @@ class CQLHelpTopics(object):
         terminates.
 
         As always, when a keyspace name does not work as a normal identifier or
-        number, it can be quoted using single quotes (CQL 2) or double quotes
-        (CQL 3).
+        number, it can be quoted using double quotes.
         """
 
     def help_create_table(self):

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/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 25b7314..31a46cd 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -777,17 +777,6 @@ public final class CFMetaData
         return maxCompactionThreshold;
     }
 
-    // Used by CQL2 only.
-    public String getCQL2KeyName()
-    {
-        if (partitionKeyColumns.size() > 1)
-            throw new IllegalStateException("Cannot acces column family with composite key from CQL < 3.0.0");
-
-        // For compatibility sake, we uppercase if it's the default alias as we used to return it that way in resultsets.
-        String str = partitionKeyColumns.get(0).name.toString();
-        return str.equalsIgnoreCase(DEFAULT_KEY_ALIAS) ? str.toUpperCase() : str;
-    }
-
     public CompressionParameters compressionParameters()
     {
         return compressionParameters;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/config/ColumnDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/ColumnDefinition.java b/src/java/org/apache/cassandra/config/ColumnDefinition.java
index bb1dd71..bbdc116 100644
--- a/src/java/org/apache/cassandra/config/ColumnDefinition.java
+++ b/src/java/org/apache/cassandra/config/ColumnDefinition.java
@@ -55,7 +55,7 @@ public class ColumnDefinition extends ColumnSpecification
      * column, whose name is not stored in the data contrarily to the column of
      * type REGULAR. Hence the COMPACT_VALUE type to distinguish it below.
      *
-     * Note that thrift/CQL2 only know about definitions of type REGULAR (and
+     * Note that thrift only knows about definitions of type REGULAR (and
      * the ones whose componentIndex == null).
      */
     public enum Kind

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/cql/AbstractModification.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/AbstractModification.java b/src/java/org/apache/cassandra/cql/AbstractModification.java
deleted file mode 100644
index 8da2611..0000000
--- a/src/java/org/apache/cassandra/cql/AbstractModification.java
+++ /dev/null
@@ -1,126 +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.cql;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.apache.cassandra.db.IMutation;
-import org.apache.cassandra.db.ConsistencyLevel;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.thrift.ThriftClientState;
-
-public abstract class AbstractModification
-{
-    public static final ConsistencyLevel defaultConsistency = ConsistencyLevel.ONE;
-
-    protected final String keyspace;
-    protected final String columnFamily;
-    protected final ConsistencyLevel cLevel;
-    protected final Long timestamp;
-    protected final int timeToLive;
-    protected final String keyName;
-
-    public AbstractModification(String keyspace, String columnFamily, String keyAlias, Attributes attrs)
-    {
-        this(keyspace, columnFamily, keyAlias, attrs.getConsistencyLevel(), attrs.getTimestamp(), attrs.getTimeToLive());
-    }
-
-    public AbstractModification(String keyspace, String columnFamily, String keyAlias, ConsistencyLevel cLevel, Long timestamp, int timeToLive)
-    {
-        this.keyspace = keyspace;
-        this.columnFamily = columnFamily;
-        this.cLevel = cLevel;
-        this.timestamp = timestamp;
-        this.timeToLive = timeToLive;
-        this.keyName = keyAlias.toUpperCase();
-    }
-
-    public String getKeyspace()
-    {
-        return keyspace;
-    }
-
-    public String getColumnFamily()
-    {
-        return columnFamily;
-    }
-
-    public ConsistencyLevel getConsistencyLevel()
-    {
-        return (cLevel != null) ? cLevel : defaultConsistency;
-    }
-
-    /**
-     * True if an explicit consistency level was parsed from the statement.
-     *
-     * @return true if a consistency was parsed, false otherwise.
-     */
-    public boolean isSetConsistencyLevel()
-    {
-        return cLevel != null;
-    }
-
-    public long getTimestamp(ThriftClientState clientState)
-    {
-        return timestamp == null ? clientState.getQueryState().getTimestamp() : timestamp;
-    }
-
-    public boolean isSetTimestamp()
-    {
-        return timestamp != null;
-    }
-
-    public int getTimeToLive()
-    {
-        return timeToLive;
-    }
-
-    public String getKeyName()
-    {
-        return keyName;
-    }
-
-    /**
-     * Convert statement into a list of mutations to apply on the server
-     *
-     * @param keyspace The working keyspace
-     * @param clientState current client status
-     *
-     * @return list of the mutations
-     *
-     * @throws InvalidRequestException on the wrong request
-     */
-    public abstract List<IMutation> prepareRowMutations(String keyspace, ThriftClientState clientState, List<ByteBuffer> variables)
-    throws InvalidRequestException, UnauthorizedException;
-
-    /**
-     * Convert statement into a list of mutations to apply on the server
-     *
-     * @param keyspace The working keyspace
-     * @param clientState current client status
-     * @param timestamp global timestamp to use for all mutations
-     *
-     * @return list of the mutations
-     *
-     * @throws InvalidRequestException on the wrong request
-     */
-    public abstract List<IMutation> prepareRowMutations(String keyspace, ThriftClientState clientState, Long timestamp, List<ByteBuffer> variables)
-    throws InvalidRequestException, UnauthorizedException;
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/cql/AlterTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/AlterTableStatement.java b/src/java/org/apache/cassandra/cql/AlterTableStatement.java
deleted file mode 100644
index 2fda212..0000000
--- a/src/java/org/apache/cassandra/cql/AlterTableStatement.java
+++ /dev/null
@@ -1,202 +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.cql;
-
-import org.apache.cassandra.config.*;
-import org.apache.cassandra.db.marshal.TypeParser;
-import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.io.compress.CompressionParameters;
-
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Map;
-
-public class AlterTableStatement
-{
-    public static enum OperationType
-    {
-        ADD, ALTER, DROP, OPTS
-    }
-
-    public final OperationType oType;
-    public final String columnFamily, columnName, validator;
-    private final CFPropDefs cfProps = new CFPropDefs();
-
-    public AlterTableStatement(String columnFamily, OperationType type, String columnName)
-    {
-        this(columnFamily, type, columnName, null);
-    }
-
-    public AlterTableStatement(String columnFamily, OperationType type, String columnName, String validator)
-    {
-        this(columnFamily, type, columnName, validator, null);
-    }
-
-    public AlterTableStatement(String columnFamily, OperationType type, String columnName, String validator, Map<String, String> propertyMap)
-    {
-        this.columnFamily = columnFamily;
-        this.oType = type;
-        this.columnName = columnName;
-        this.validator = CFPropDefs.comparators.get(validator); // used only for ADD/ALTER commands
-
-        if (propertyMap != null)
-        {
-            for (Map.Entry<String, String> prop : propertyMap.entrySet())
-            {
-                cfProps.addProperty(prop.getKey(), prop.getValue());
-            }
-        }
-    }
-
-    public CFMetaData getCFMetaData(String keyspace) throws ConfigurationException, InvalidRequestException, SyntaxException
-    {
-        CFMetaData meta = Schema.instance.getCFMetaData(keyspace, columnFamily);
-        CFMetaData cfm = meta.clone();
-
-        ByteBuffer columnName = this.oType == OperationType.OPTS ? null
-                                                                 : meta.comparator.subtype(0).fromStringCQL2(this.columnName);
-
-        switch (oType)
-        {
-            case ADD:
-                cfm.addColumnDefinition(ColumnDefinition.regularDef(cfm, columnName, TypeParser.parse(validator), null));
-                break;
-
-            case ALTER:
-                // We only look for the first key alias which is ok for CQL2
-                ColumnDefinition partionKeyDef = cfm.partitionKeyColumns().get(0);
-                if (partionKeyDef.name.bytes.equals(columnName))
-                {
-                    cfm.keyValidator(TypeParser.parse(validator));
-                }
-                else
-                {
-                    ColumnDefinition toUpdate = null;
-
-                    for (ColumnDefinition columnDef : cfm.regularColumns())
-                    {
-                        if (columnDef.name.bytes.equals(columnName))
-                        {
-                            toUpdate = columnDef;
-                            break;
-                        }
-                    }
-
-                    if (toUpdate == null)
-                        throw new InvalidRequestException(String.format("Column '%s' was not found in CF '%s'",
-                                    this.columnName,
-                                    columnFamily));
-
-                    cfm.addOrReplaceColumnDefinition(toUpdate.withNewType(TypeParser.parse(validator)));
-                }
-                break;
-
-            case DROP:
-                ColumnDefinition toDelete = null;
-
-                for (ColumnDefinition columnDef : cfm.regularColumns())
-                {
-                    if (columnDef.name.bytes.equals(columnName))
-                    {
-                        toDelete = columnDef;
-                    }
-                }
-
-                if (toDelete == null)
-                    throw new InvalidRequestException(String.format("Column '%s' was not found in CF '%s'",
-                                                                    this.columnName,
-                                                                    columnFamily));
-
-                cfm.removeColumnDefinition(toDelete);
-                break;
-
-            case OPTS:
-                if (cfProps == null)
-                    throw new InvalidRequestException(String.format("ALTER COLUMNFAMILY WITH invoked, but no parameters found"));
-
-                cfProps.validate();
-                applyPropertiesToCFMetadata(cfm, cfProps);
-                break;
-        }
-
-        return cfm;
-    }
-
-    public String toString()
-    {
-        return String.format("AlterTableStatement(cf=%s, type=%s, column=%s, validator=%s)",
-                             columnFamily,
-                             oType,
-                             columnName,
-                             validator);
-    }
-
-    public static void applyPropertiesToCFMetadata(CFMetaData cfm, CFPropDefs cfProps) throws InvalidRequestException, ConfigurationException
-    {
-        if (cfProps.hasProperty(CFPropDefs.KW_COMPACTION_STRATEGY_CLASS))
-            cfm.compactionStrategyClass(cfProps.compactionStrategyClass);
-
-        if (cfProps.hasProperty(CFPropDefs.KW_COMPARATOR))
-            throw new InvalidRequestException("Can't change CF comparator after creation");
-
-        if (cfProps.hasProperty(CFPropDefs.KW_COMMENT))
-            cfm.comment(cfProps.getProperty(CFPropDefs.KW_COMMENT));
-
-        if (cfProps.hasProperty(CFPropDefs.KW_DEFAULTVALIDATION))
-        {
-            try
-            {
-                cfm.defaultValidator(cfProps.getValidator());
-            }
-            catch (RequestValidationException e)
-            {
-                throw new InvalidRequestException(String.format("Invalid validation type %s",
-                                                                cfProps.getProperty(CFPropDefs.KW_DEFAULTVALIDATION)));
-            }
-        }
-
-        cfm.readRepairChance(cfProps.getPropertyDouble(CFPropDefs.KW_READREPAIRCHANCE, cfm.getReadRepairChance()));
-        cfm.dcLocalReadRepairChance(cfProps.getPropertyDouble(CFPropDefs.KW_DCLOCALREADREPAIRCHANCE, cfm.getDcLocalReadRepair()));
-        cfm.gcGraceSeconds(cfProps.getPropertyInt(CFPropDefs.KW_GCGRACESECONDS, cfm.getGcGraceSeconds()));
-        int minCompactionThreshold = cfProps.getPropertyInt(CFPropDefs.KW_MINCOMPACTIONTHRESHOLD, cfm.getMinCompactionThreshold());
-        int maxCompactionThreshold = cfProps.getPropertyInt(CFPropDefs.KW_MAXCOMPACTIONTHRESHOLD, cfm.getMaxCompactionThreshold());
-        if (minCompactionThreshold <= 0 || maxCompactionThreshold <= 0)
-            throw new ConfigurationException("Disabling compaction by setting compaction thresholds to 0 has been deprecated, set the compaction option 'enabled' to false instead.");
-        cfm.minCompactionThreshold(minCompactionThreshold);
-        cfm.maxCompactionThreshold(maxCompactionThreshold);
-        cfm.caching(CFMetaData.Caching.fromString(cfProps.getPropertyString(CFPropDefs.KW_CACHING, cfm.getCaching().toString())));
-        cfm.rowsPerPartitionToCache(CFMetaData.RowsPerPartitionToCache.fromString(cfProps.getPropertyString(CFPropDefs.KW_ROWS_PER_PARTITION_TO_CACHE, cfm.getRowsPerPartitionToCache().toString())));
-        cfm.defaultTimeToLive(cfProps.getPropertyInt(CFPropDefs.KW_DEFAULT_TIME_TO_LIVE, cfm.getDefaultTimeToLive()));
-        cfm.speculativeRetry(CFMetaData.SpeculativeRetry.fromString(cfProps.getPropertyString(CFPropDefs.KW_SPECULATIVE_RETRY, cfm.getSpeculativeRetry().toString())));
-        cfm.populateIoCacheOnFlush(cfProps.getPropertyBoolean(CFPropDefs.KW_POPULATE_IO_CACHE_ON_FLUSH, cfm.populateIoCacheOnFlush()));
-        cfm.bloomFilterFpChance(cfProps.getPropertyDouble(CFPropDefs.KW_BF_FP_CHANCE, cfm.getBloomFilterFpChance()));
-        cfm.memtableFlushPeriod(cfProps.getPropertyInt(CFPropDefs.KW_MEMTABLE_FLUSH_PERIOD, cfm.getMemtableFlushPeriod()));
-
-        if (!cfProps.compactionStrategyOptions.isEmpty())
-        {
-            cfm.compactionStrategyOptions(new HashMap<String, String>());
-            for (Map.Entry<String, String> entry : cfProps.compactionStrategyOptions.entrySet())
-                cfm.compactionStrategyOptions.put(entry.getKey(), entry.getValue());
-        }
-
-        if (!cfProps.compressionParameters.isEmpty())
-        {
-            cfm.compressionParameters(CompressionParameters.create(cfProps.compressionParameters));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/cql/Attributes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/Attributes.java b/src/java/org/apache/cassandra/cql/Attributes.java
deleted file mode 100644
index faee3b8..0000000
--- a/src/java/org/apache/cassandra/cql/Attributes.java
+++ /dev/null
@@ -1,76 +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.cql;
-
-import org.apache.cassandra.db.ConsistencyLevel;
-
-/**
- * Class to contain attributes for statements
- */
-public class Attributes
-{
-    private ConsistencyLevel cLevel;
-    private Long timestamp;
-    private int timeToLive;
-
-    public Attributes()
-    {}
-
-    public Attributes(ConsistencyLevel cLevel, Long timestamp, int timeToLive)
-    {
-        this.cLevel = cLevel;
-        this.timestamp = timestamp;
-        this.timeToLive = timeToLive;
-    }
-
-    public ConsistencyLevel getConsistencyLevel()
-    {
-        return cLevel;
-    }
-
-    public void setConsistencyLevel(ConsistencyLevel cLevel)
-    {
-        this.cLevel = cLevel;
-    }
-
-    public Long getTimestamp()
-    {
-        return timestamp;
-    }
-
-    public void setTimestamp(Long timestamp)
-    {
-        this.timestamp = timestamp;
-    }
-
-    public int getTimeToLive()
-    {
-        return timeToLive;
-    }
-
-    public void setTimeToLive(int timeToLive)
-    {
-        this.timeToLive = timeToLive;
-    }
-
-    public String toString()
-    {
-        return String.format("Attributes(consistency=%s, timestamp=%s, timeToLive=%s)", cLevel, timestamp, timeToLive);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/cql/BatchStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/BatchStatement.java b/src/java/org/apache/cassandra/cql/BatchStatement.java
deleted file mode 100644
index b141bcc..0000000
--- a/src/java/org/apache/cassandra/cql/BatchStatement.java
+++ /dev/null
@@ -1,104 +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.cql;
-
-import java.nio.ByteBuffer;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.cassandra.db.IMutation;
-import org.apache.cassandra.db.ConsistencyLevel;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.thrift.ThriftClientState;
-
-/**
- * A <code>BATCH</code> statement parsed from a CQL query.
- *
- */
-public class BatchStatement
-{
-    // statements to execute
-    protected final List<AbstractModification> statements;
-
-    // global consistency level
-    protected final ConsistencyLevel consistency;
-
-    // global timestamp to apply for each mutation
-    protected final Long timestamp;
-
-    // global time to live
-    protected final int timeToLive;
-
-    /**
-     * Creates a new BatchStatement from a list of statements and a
-     * Thrift consistency level.
-     *
-     * @param statements a list of UpdateStatements
-     * @param attrs additional attributes for statement (CL, timestamp, timeToLive)
-     */
-    public BatchStatement(List<AbstractModification> statements, Attributes attrs)
-    {
-        this.statements = statements;
-        this.consistency = attrs.getConsistencyLevel();
-        this.timestamp = attrs.getTimestamp();
-        this.timeToLive = attrs.getTimeToLive();
-    }
-
-    public List<AbstractModification> getStatements()
-    {
-        return statements;
-    }
-
-    public ConsistencyLevel getConsistencyLevel()
-    {
-        return consistency;
-    }
-
-    public int getTimeToLive()
-    {
-        return timeToLive;
-    }
-
-    public long getTimestamp()
-    {
-        return timestamp;
-    }
-
-    public List<IMutation> getMutations(String keyspace, ThriftClientState clientState, List<ByteBuffer> variables)
-    throws InvalidRequestException, UnauthorizedException
-    {
-        List<IMutation> batch = new LinkedList<IMutation>();
-
-        for (AbstractModification statement : statements) {
-            batch.addAll(statement.prepareRowMutations(keyspace, clientState, timestamp, variables));
-        }
-
-        return batch;
-    }
-
-    public boolean isSetTimestamp()
-    {
-        return timestamp != null;
-    }
-
-    public String toString()
-    {
-        return String.format("BatchStatement(statements=%s, consistency=%s)", statements, consistency);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/26217071/src/java/org/apache/cassandra/cql/CFPropDefs.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/CFPropDefs.java b/src/java/org/apache/cassandra/cql/CFPropDefs.java
deleted file mode 100644
index 42e840e..0000000
--- a/src/java/org/apache/cassandra/cql/CFPropDefs.java
+++ /dev/null
@@ -1,307 +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.cql;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import com.google.common.base.Strings;
-import com.google.common.collect.Sets;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.TypeParser;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.SyntaxException;
-import org.apache.commons.lang3.StringUtils;
-
-public class CFPropDefs {
-    private static final Logger logger = LoggerFactory.getLogger(CFPropDefs.class);
-
-    public static final String KW_COMPARATOR = "comparator";
-    public static final String KW_COMMENT = "comment";
-    public static final String KW_READREPAIRCHANCE = "read_repair_chance";
-    public static final String KW_DCLOCALREADREPAIRCHANCE = "dclocal_read_repair_chance";
-    public static final String KW_GCGRACESECONDS = "gc_grace_seconds";
-    public static final String KW_DEFAULTVALIDATION = "default_validation";
-    public static final String KW_MINCOMPACTIONTHRESHOLD = "min_compaction_threshold";
-    public static final String KW_MAXCOMPACTIONTHRESHOLD = "max_compaction_threshold";
-    public static final String KW_COMPACTION_STRATEGY_CLASS = "compaction_strategy_class";
-    public static final String KW_CACHING = "caching";
-    public static final String KW_ROWS_PER_PARTITION_TO_CACHE = "rows_per_partition_to_cache";
-    public static final String KW_DEFAULT_TIME_TO_LIVE = "default_time_to_live";
-    public static final String KW_SPECULATIVE_RETRY = "speculative_retry";
-    public static final String KW_POPULATE_IO_CACHE_ON_FLUSH = "populate_io_cache_on_flush";
-    public static final String KW_BF_FP_CHANCE = "bloom_filter_fp_chance";
-    public static final String KW_MEMTABLE_FLUSH_PERIOD = "memtable_flush_period_in_ms";
-
-    // 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_COMPARATOR);
-        keywords.add(KW_COMMENT);
-        keywords.add(KW_READREPAIRCHANCE);
-        keywords.add(KW_DCLOCALREADREPAIRCHANCE);
-        keywords.add(KW_GCGRACESECONDS);
-        keywords.add(KW_DEFAULTVALIDATION);
-        keywords.add(KW_MINCOMPACTIONTHRESHOLD);
-        keywords.add(KW_MAXCOMPACTIONTHRESHOLD);
-        keywords.add(KW_COMPACTION_STRATEGY_CLASS);
-        keywords.add(KW_CACHING);
-        keywords.add(KW_ROWS_PER_PARTITION_TO_CACHE);
-        keywords.add(KW_DEFAULT_TIME_TO_LIVE);
-        keywords.add(KW_SPECULATIVE_RETRY);
-        keywords.add(KW_POPULATE_IO_CACHE_ON_FLUSH);
-        keywords.add(KW_BF_FP_CHANCE);
-        keywords.add(KW_MEMTABLE_FLUSH_PERIOD);
-
-        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("replicate_on_write");
-
-        allowedKeywords.addAll(keywords);
-        allowedKeywords.addAll(obsoleteKeywords);
-    }
-
-    public final Map<String, String> properties = new HashMap<String, String>();
-    public Class<? extends AbstractCompactionStrategy> compactionStrategyClass;
-    public final Map<String, String> compactionStrategyOptions = new HashMap<String, String>();
-    public final Map<String, String> compressionParameters = new HashMap<String, String>();
-
-    public void validate() throws InvalidRequestException, ConfigurationException
-    {
-        compactionStrategyClass = CFMetaData.DEFAULT_COMPACTION_STRATEGY_CLASS;
-
-        // we need to remove parent:key = value pairs from the main properties
-        Set<String> propsToRemove = new HashSet<String>();
-
-        // check if we have compaction/compression options
-        for (String property : properties.keySet())
-        {
-            if (!property.contains(":"))
-                continue;
-
-            String key = property.split(":")[1];
-            String val = properties.get(property);
-
-            if (property.startsWith(COMPACTION_OPTIONS_PREFIX))
-            {
-                compactionStrategyOptions.put(key, val);
-                propsToRemove.add(property);
-            }
-
-            if (property.startsWith(COMPRESSION_PARAMETERS_PREFIX))
-            {
-                compressionParameters.put(key, val);
-                propsToRemove.add(property);
-            }
-        }
-
-        for (String property : propsToRemove)
-            properties.remove(property);
-        // 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 = getPropertyInt(KW_MINCOMPACTIONTHRESHOLD, null);
-        Integer maxCompaction = getPropertyInt(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));
-        }
-
-        Integer defaultTimeToLive = getPropertyInt(KW_DEFAULT_TIME_TO_LIVE, null);
-
-        if (defaultTimeToLive != null)
-        {
-            if (defaultTimeToLive < 0)
-                throw new InvalidRequestException(String.format("%s cannot be smaller than %s, (default %s)",
-                        KW_DEFAULT_TIME_TO_LIVE,
-                        0,
-                        CFMetaData.DEFAULT_DEFAULT_TIME_TO_LIVE));
-        }
-
-        CFMetaData.validateCompactionOptions(compactionStrategyClass, compactionStrategyOptions);
-    }
-
-    /** Map a keyword to the corresponding value */
-    public void addProperty(String name, String value)
-    {
-        properties.put(name, value);
-    }
-
-    public Boolean hasProperty(String name)
-    {
-        return properties.containsKey(name);
-    }
-
-    /* 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 AbstractType<?> getComparator() throws ConfigurationException, SyntaxException
-    {
-        return TypeParser.parse((comparators.get(getPropertyString(KW_COMPARATOR, "text")) != null)
-                                  ? comparators.get(getPropertyString(KW_COMPARATOR, "text"))
-                                  : getPropertyString(KW_COMPARATOR, "text"));
-    }
-
-    public AbstractType<?> getValidator() throws ConfigurationException, SyntaxException
-    {
-        return TypeParser.parse((comparators.get(getPropertyString(KW_DEFAULTVALIDATION, "text")) != null)
-                                  ? comparators.get(getPropertyString(KW_DEFAULTVALIDATION, "text"))
-                                  : getPropertyString(KW_DEFAULTVALIDATION, "text"));
-    }
-
-    public String getProperty(String name)
-    {
-        return properties.get(name);
-    }
-
-    public String getPropertyString(String key, String defaultValue)
-    {
-        String value = properties.get(key);
-        return value != null ? value : defaultValue;
-    }
-
-    // Return a property value, typed as a Boolean
-    public Boolean getPropertyBoolean(String key, Boolean defaultValue)
-    {
-        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 getPropertyDouble(String key, Double defaultValue) throws InvalidRequestException
-    {
-        Double result;
-        String value = properties.get(key);
-
-        if (value == null)
-            result = defaultValue;
-        else
-        {
-            try
-            {
-                result = Double.valueOf(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 getPropertyInt(String key, Integer defaultValue) throws InvalidRequestException
-    {
-        Integer result;
-        String value = properties.get(key);
-
-        if (value == null)
-            result = defaultValue;
-        else
-        {
-            try
-            {
-                result = Integer.valueOf(value);
-            }
-            catch (NumberFormatException e)
-            {
-                throw new InvalidRequestException(String.format("%s not valid for \"%s\"", value, key));
-            }
-        }
-        return result;
-    }
-
-    public Set<String> getPropertySet(String key, Set<String> defaultValue)
-    {
-        String value = properties.get(key);
-        if (Strings.isNullOrEmpty(value))
-            return defaultValue;
-        return Sets.newHashSet(StringUtils.split(value, ','));
-    }
-
-    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/26217071/src/java/org/apache/cassandra/cql/CQLStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/CQLStatement.java b/src/java/org/apache/cassandra/cql/CQLStatement.java
deleted file mode 100644
index 157ce67..0000000
--- a/src/java/org/apache/cassandra/cql/CQLStatement.java
+++ /dev/null
@@ -1,32 +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.cql;
-
-public class CQLStatement
-{
-    public final StatementType type;
-    public final Object statement;
-    public final int boundTerms;
-
-    public CQLStatement(StatementType type, Object statement, int lastMarker)
-    {
-        this.type = type;
-        this.statement = statement;
-        this.boundTerms = lastMarker + 1;
-    }
-}