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 2013/05/06 22:16:05 UTC

git commit: Add alias support to SELECT statement

Updated Branches:
  refs/heads/trunk df723af8a -> 08df0b860


Add alias support to SELECT statement

patch by Aleksey Yeschenko; reviewed by Sylvain Lebresne for
CASSANDRA-5075


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

Branch: refs/heads/trunk
Commit: 08df0b8604cd34c4f1fb384f9f6eaba86203d116
Parents: df723af
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Mon May 6 23:11:52 2013 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Mon May 6 23:11:52 2013 +0300

----------------------------------------------------------------------
 CHANGES.txt                                        |    1 +
 NEWS.txt                                           |    5 +
 bin/cqlsh                                          |    2 +-
 doc/cql3/CQL.textile                               |   13 +-
 pylib/cqlshlib/cql3handling.py                     |    4 +-
 .../apache/cassandra/cql3/ColumnIdentifier.java    |    5 +-
 src/java/org/apache/cassandra/cql3/Cql.g           |   30 ++-
 src/java/org/apache/cassandra/cql3/ResultSet.java  |    5 +-
 .../cassandra/cql3/statements/RawSelector.java     |   48 +----
 .../cassandra/cql3/statements/SelectStatement.java |  171 +++++++--------
 .../cassandra/cql3/statements/Selectable.java      |   69 ++++++
 .../cassandra/cql3/statements/Selection.java       |   64 +++---
 12 files changed, 236 insertions(+), 181 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/08df0b86/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 3bb2a71..60accb1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -40,6 +40,7 @@
  * Reenable ALTER TABLE DROP with new semantics (CASSANDRA-3919)
  * Add binary protocol versioning (CASSANDRA-5436)
  * Swap THshaServer for TThreadedSelectorServer (CASSANDRA-5530)
+ * Add alias support to SELECT statement (CASSANDRA-5075)
 
 
 1.2.5

http://git-wip-us.apache.org/repos/asf/cassandra/blob/08df0b86/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index c4a57c6..4256095 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -55,6 +55,11 @@ Operations
     - CAS uses gc_grace_seconds to determine how long to keep unused paxos
       state around for, or a minimum of three hours.
 
+Features
+--------
+    - Alias support has been added to CQL3 SELECT statement. Refer to
+      CQL3 documentation (http://cassandra.apache.org/doc/cql3/CQL.html) for details.
+
 
 1.2.5
 =====

http://git-wip-us.apache.org/repos/asf/cassandra/blob/08df0b86/bin/cqlsh
----------------------------------------------------------------------
diff --git a/bin/cqlsh b/bin/cqlsh
index 919c030..cac9e28 100755
--- a/bin/cqlsh
+++ b/bin/cqlsh
@@ -32,7 +32,7 @@ exit 1
 from __future__ import with_statement
 
 description = "CQL Shell for Apache Cassandra"
-version = "3.0.1"
+version = "3.0.2"
 
 from StringIO import StringIO
 from itertools import groupby

http://git-wip-us.apache.org/repos/asf/cassandra/blob/08df0b86/doc/cql3/CQL.textile
----------------------------------------------------------------------
diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile
index 4c0522d..b131f8f 100644
--- a/doc/cql3/CQL.textile
+++ b/doc/cql3/CQL.textile
@@ -580,9 +580,9 @@ bc(syntax)..
                   ( ALLOW FILTERING )?
 
 <select-clause> ::= <selection-list>
-                  | COUNT '(' ( '*' | '1' ) ')'
+                  | COUNT '(' ( '*' | '1' ) ')' (AS <identifier>)?
 
-<selection-list> ::= <selector> ( ',' <selector> )*
+<selection-list> ::= <selector> (AS <identifier>)? ( ',' <selector> (AS <identifier>)? )*
                    | '*'
 
 <selector> ::= <identifier>
@@ -604,6 +604,8 @@ __Sample:__
 bc(sample).. 
 SELECT name, occupation FROM users WHERE userid IN (199, 200, 207);
 
+SELECT name AS user_name, occupation AS user_occupation FROM users;
+
 SELECT time, value
 FROM events
 WHERE event_type = 'myEvent'
@@ -611,6 +613,9 @@ WHERE event_type = 'myEvent'
   AND time <= 2012-01-01
 
 SELECT COUNT(*) FROM users;
+
+SELECT COUNT(*) AS user_count FROM users;
+
 p. 
 The @SELECT@ statements reads one or more columns for one or more rows in a table. It returns a result-set of rows, where each row contains the collection of columns corresponding to the query.
 
@@ -620,6 +625,8 @@ The @<select-clause>@ determines which columns needs to be queried and returned
 
 A @<selector>@ is either a column name to retrieve, or a @<function>@ of one or multiple column names. The functions allows are the same that for @<term>@ and are describe in the "function section":#function. In addition to these generic functions, the @WRITETIME@ (resp. @TTL@) function allows to select the timestamp of when the column was inserted (resp. the time to live (in seconds) for the column (or null if the column has no expiration set)).
 
+Any @<selector>@ can be aliased using @AS@ keyword (see examples). Please note that @<where-clause>@ and @<order-by>@ clause should refer to the columns by their original names and not by their aliases.
+
 The @COUNT@ keyword can be used with parenthesis enclosing @*@. If so, the query will return a single result: the number of rows matching the query. Note that @COUNT(1)@ is supported as an alias.
 
 h4(#selectWhere). @<where-clause>@
@@ -966,6 +973,7 @@ CQL distinguishes between _reserved_ and _non-reserved_ keywords. Reserved keywo
 | @AND@          | yes |
 | @ANY@          | yes |
 | @APPLY@        | yes |
+| @AS@           | no  |
 | @ASC@          | yes |
 | @ASCII@        | no  |
 | @AUTHORIZE@    | yes |
@@ -1051,6 +1059,7 @@ The following describes the addition/changes brought for each version of CQL.
 h3. 3.1.0
 
 * "ALTER TABLE":#alterTableStmt @DROP@ option has been reenabled for CQL3 tables and has new semantics now: the space formerly used by dropped columns will now be eventually reclaimed (post-compaction). You should not readd previously dropped columns unless you use timestamps with microsecond precision (see "CASSANDRA-3919":https://issues.apache.org/jira/browse/CASSANDRA-3919 for more details).
+* SELECT statement now supports aliases in select clause. Aliases in WHERE and ORDER BY clauses are not supported. See the "section on select"#selectStmt for details.
 
 h3. 3.0.3
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/08df0b86/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index 8ef596c..8e913dc 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -758,9 +758,9 @@ syntax_rules += r'''
                              ")" ("=" | "<" | ">" | "<=" | ">=") <tokenDefinition>
              | [rel_lhs]=<cident> "IN" "(" <term> ( "," <term> )* ")"
              ;
-<selectClause> ::= <selector> ("," <selector>)*
+<selectClause> ::= <selector> ("AS" <cident>)? ("," <selector> ("AS" <cident>)?)*
                  | "*"
-                 | "COUNT" "(" star=( "*" | "1" ) ")"
+                 | "COUNT" "(" star=( "*" | "1" ) ")" ("AS" <cident>)?
                  ;
 <selector> ::= [colname]=<cident>
              | "WRITETIME" "(" [colname]=<cident> ")"

http://git-wip-us.apache.org/repos/asf/cassandra/blob/08df0b86/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
index 495915e..330eec0 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
@@ -20,15 +20,14 @@ package org.apache.cassandra.cql3;
 import java.util.Locale;
 import java.nio.ByteBuffer;
 
+import org.apache.cassandra.cql3.statements.Selectable;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-import org.apache.cassandra.cql3.statements.RawSelector;
-
 /**
  * Represents an identifer for a CQL column definition.
  */
-public class ColumnIdentifier implements RawSelector, Comparable<ColumnIdentifier>
+public class ColumnIdentifier implements Selectable, Comparable<ColumnIdentifier>
 {
     public final ByteBuffer key;
     private final String text;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/08df0b86/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Cql.g b/src/java/org/apache/cassandra/cql3/Cql.g
index 30be530..8d31de9 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -211,11 +211,13 @@ useStatement returns [UseStatement stmt]
 selectStatement returns [SelectStatement.RawStatement expr]
     @init {
         boolean isCount = false;
+        ColumnIdentifier countAlias = null;
         int limit = Integer.MAX_VALUE;
         Map<ColumnIdentifier, Boolean> orderings = new LinkedHashMap<ColumnIdentifier, Boolean>();
         boolean allowFiltering = false;
     }
-    : K_SELECT ( sclause=selectClause | (K_COUNT '(' sclause=selectCountClause ')' { isCount = true; }) )
+    : K_SELECT ( sclause=selectClause
+               | (K_COUNT '(' sclause=selectCountClause ')' { isCount = true; } (K_AS c=cident { countAlias = c; })?) )
       K_FROM cf=columnFamilyName
       ( K_WHERE wclause=whereClause )?
       ( K_ORDER K_BY orderByClause[orderings] ( ',' orderByClause[orderings] )* )?
@@ -225,6 +227,7 @@ selectStatement returns [SelectStatement.RawStatement expr]
           SelectStatement.Parameters params = new SelectStatement.Parameters(limit,
                                                                              orderings,
                                                                              isCount,
+                                                                             countAlias,
                                                                              allowFiltering);
           $expr = new SelectStatement.RawStatement(cf, params, sclause, wclause);
       }
@@ -235,18 +238,23 @@ selectClause returns [List<RawSelector> expr]
     | '\*' { $expr = Collections.<RawSelector>emptyList();}
     ;
 
-selectionFunctionArgs returns [List<RawSelector> a]
-    : '(' ')' { $a = Collections.emptyList(); }
-    | '(' s1=selector { List<RawSelector> args = new ArrayList<RawSelector>(); args.add(s1); }
-          ( ',' sn=selector { args.add(sn); } )*
-       ')' { $a = args; }
+selector returns [RawSelector s]
+    @init{ ColumnIdentifier alias = null; }
+    : us=unaliasedSelector (K_AS c=cident { alias = c; })? { $s = new RawSelector(us, alias); }
     ;
 
-selector returns [RawSelector s]
+unaliasedSelector returns [Selectable s]
     : c=cident                                  { $s = c; }
-    | K_WRITETIME '(' c=cident ')'              { $s = new RawSelector.WritetimeOrTTL(c, true); }
-    | K_TTL       '(' c=cident ')'              { $s = new RawSelector.WritetimeOrTTL(c, false); }
-    | f=functionName args=selectionFunctionArgs { $s = new RawSelector.WithFunction(f, args); }
+    | K_WRITETIME '(' c=cident ')'              { $s = new Selectable.WritetimeOrTTL(c, true); }
+    | K_TTL       '(' c=cident ')'              { $s = new Selectable.WritetimeOrTTL(c, false); }
+    | f=functionName args=selectionFunctionArgs { $s = new Selectable.WithFunction(f, args); }
+    ;
+
+selectionFunctionArgs returns [List<Selectable> a]
+    : '(' ')' { $a = Collections.emptyList(); }
+    | '(' s1=unaliasedSelector { List<Selectable> args = new ArrayList<Selectable>(); args.add(s1); }
+          ( ',' sn=unaliasedSelector { args.add(sn); } )*
+       ')' { $a = args; }
     ;
 
 selectCountClause returns [List<RawSelector> expr]
@@ -878,6 +886,7 @@ unreserved_keyword returns [String str]
 
 unreserved_function_keyword returns [String str]
     : k=( K_KEY
+        | K_AS
         | K_CLUSTERING
         | K_COMPACT
         | K_STORAGE
@@ -905,6 +914,7 @@ unreserved_function_keyword returns [String str]
 // Case-insensitive keywords
 K_SELECT:      S E L E C T;
 K_FROM:        F R O M;
+K_AS:          A S;
 K_WHERE:       W H E R E;
 K_AND:         A N D;
 K_KEY:         K E Y;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/08df0b86/src/java/org/apache/cassandra/cql3/ResultSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ResultSet.java b/src/java/org/apache/cassandra/cql3/ResultSet.java
index 1d914da..451efb2 100644
--- a/src/java/org/apache/cassandra/cql3/ResultSet.java
+++ b/src/java/org/apache/cassandra/cql3/ResultSet.java
@@ -92,13 +92,14 @@ public class ResultSet
         }
     }
 
-    public ResultSet makeCountResult()
+    public ResultSet makeCountResult(ColumnIdentifier alias)
     {
         String ksName = metadata.names.get(0).ksName;
         String cfName = metadata.names.get(0).cfName;
         long count = rows.size();
 
-        Metadata newMetadata = new Metadata(Collections.singletonList(new ColumnSpecification(ksName, cfName, COUNT_COLUMN, LongType.instance)));
+        ColumnSpecification spec = new ColumnSpecification(ksName, cfName, alias == null ? COUNT_COLUMN : alias, LongType.instance);
+        Metadata newMetadata = new Metadata(Collections.singletonList(spec));
         List<List<ByteBuffer>> newRows = Collections.singletonList(Collections.singletonList(ByteBufferUtil.bytes(count)));
 
         return new ResultSet(newMetadata, newRows);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/08df0b86/src/java/org/apache/cassandra/cql3/statements/RawSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/RawSelector.java b/src/java/org/apache/cassandra/cql3/statements/RawSelector.java
index 2a7ceff..01fe9e4 100644
--- a/src/java/org/apache/cassandra/cql3/statements/RawSelector.java
+++ b/src/java/org/apache/cassandra/cql3/statements/RawSelector.java
@@ -18,52 +18,16 @@
  */
 package org.apache.cassandra.cql3.statements;
 
-import java.util.List;
-
 import org.apache.cassandra.cql3.ColumnIdentifier;
 
-public interface RawSelector
+public class RawSelector
 {
-    public static class WritetimeOrTTL implements RawSelector
-    {
-        public final ColumnIdentifier id;
-        public final boolean isWritetime;
+    public final Selectable selectable;
+    public final ColumnIdentifier alias;
 
-        public WritetimeOrTTL(ColumnIdentifier id, boolean isWritetime)
-        {
-            this.id = id;
-            this.isWritetime = isWritetime;
-        }
-
-        @Override
-        public String toString()
-        {
-            return (isWritetime ? "writetime" : "ttl") + "(" + id + ")";
-        }
-    }
-
-    public static class WithFunction implements RawSelector
+    public RawSelector(Selectable selectable, ColumnIdentifier alias)
     {
-        public final String functionName;
-        public final List<RawSelector> args;
-
-        public WithFunction(String functionName, List<RawSelector> args)
-        {
-            this.functionName = functionName;
-            this.args = args;
-        }
-
-        @Override
-        public String toString()
-        {
-            StringBuilder sb = new StringBuilder();
-            sb.append(functionName).append("(");
-            for (int i = 0; i < args.size(); i++)
-            {
-                if (i > 0) sb.append(", ");
-                sb.append(args.get(i));
-            }
-            return sb.append(")").toString();
-        }
+        this.selectable = selectable;
+        this.alias = alias;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/08df0b86/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 93571fa..856982c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -21,7 +21,9 @@ import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.ExecutionException;
 
+import com.google.common.base.Predicate;
 import com.google.common.collect.AbstractIterator;
+import com.google.common.collect.Iterables;
 
 import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.cql3.*;
@@ -72,6 +74,8 @@ public class SelectStatement implements CQLStatement
     private boolean keyIsInRelation;
     private boolean usesSecondaryIndexing;
 
+    private Map<CFDefinition.Name, Integer> orderingIndexes;
+
     private static enum Bound
     {
         START(0), END(1);
@@ -87,7 +91,7 @@ public class SelectStatement implements CQLStatement
         {
             return b == START ? END : START;
         }
-    };
+    }
 
     public SelectStatement(CFDefinition cfDef, int boundTerms, Parameters parameters, Selection selection)
     {
@@ -132,14 +136,14 @@ public class SelectStatement implements CQLStatement
     {
         // Even for count, we need to process the result as it'll group some column together in sparse column families
         ResultSet rset = process(rows, variables);
-        rset = parameters.isCount ? rset.makeCountResult() : rset;
+        rset = parameters.isCount ? rset.makeCountResult(parameters.countAlias) : rset;
         return new ResultMessage.Rows(rset);
     }
 
     static List<Row> readLocally(String keyspace, List<ReadCommand> cmds)
     {
         Table table = Table.open(keyspace);
-        List<Row> rows = new ArrayList(cmds.size());
+        List<Row> rows = new ArrayList<Row>(cmds.size());
         for (ReadCommand cmd : cmds)
             rows.add(cmd.getRow(table));
         return rows;
@@ -538,7 +542,7 @@ public class SelectStatement implements CQLStatement
     private List<IndexExpression> getIndexExpressions(List<ByteBuffer> variables) throws InvalidRequestException
     {
         if (!usesSecondaryIndexing || restrictedNames.isEmpty())
-            return Collections.<IndexExpression>emptyList();
+            return Collections.emptyList();
 
         List<IndexExpression> expressions = new ArrayList<IndexExpression>();
         for (CFDefinition.Name name : restrictedNames)
@@ -640,15 +644,9 @@ public class SelectStatement implements CQLStatement
             if (row.cf == null)
                 continue;
 
-            ByteBuffer[] keyComponents = null;
-            if (cfDef.hasCompositeKey)
-            {
-                keyComponents = ((CompositeType)cfDef.cfm.getKeyValidator()).split(row.key.key);
-            }
-            else
-            {
-                keyComponents = new ByteBuffer[]{ row.key.key };
-            }
+            ByteBuffer[] keyComponents = cfDef.hasCompositeKey
+                                       ? ((CompositeType)cfDef.cfm.getKeyValidator()).split(row.key.key)
+                                       : new ByteBuffer[]{ row.key.key };
 
             if (cfDef.isCompact)
             {
@@ -715,7 +713,7 @@ public class SelectStatement implements CQLStatement
                 }
 
                 for (ColumnGroupMap group : builder.groups())
-                    handleGroup(selection, result, row.key.key, keyComponents, group);
+                    handleGroup(selection, result, keyComponents, group);
             }
             else
             {
@@ -759,13 +757,14 @@ public class SelectStatement implements CQLStatement
         if (cqlRows.size() == 0 || parameters.orderings.isEmpty() || isKeyRange || !keyIsInRelation)
             return;
 
+        assert orderingIndexes != null;
 
         // optimization when only *one* order condition was given
         // because there is no point of using composite comparator if there is only one order condition
         if (parameters.orderings.size() == 1)
         {
             CFDefinition.Name ordering = cfDef.get(parameters.orderings.keySet().iterator().next());
-            Collections.sort(cqlRows.rows, new SingleColumnComparator(getColumnPositionInResultSet(cqlRows, ordering), ordering.type));
+            Collections.sort(cqlRows.rows, new SingleColumnComparator(orderingIndexes.get(ordering), ordering.type));
             return;
         }
 
@@ -780,47 +779,13 @@ public class SelectStatement implements CQLStatement
         {
             CFDefinition.Name orderingColumn = cfDef.get(identifier);
             types.add(orderingColumn.type);
-            positions[idx++] = getColumnPositionInResultSet(cqlRows, orderingColumn);
+            positions[idx++] = orderingIndexes.get(orderingColumn);
         }
 
         Collections.sort(cqlRows.rows, new CompositeComparator(types, positions));
     }
 
-    // determine position of column in the select clause
-    private int getColumnPositionInResultSet(ResultSet rs, CFDefinition.Name columnName)
-    {
-        for (int i = 0; i < rs.metadata.names.size(); i++)
-        {
-            if (rs.metadata.names.get(i).name.equals(columnName.name))
-                return i;
-        }
-
-        throw new IllegalArgumentException(String.format("Column %s wasn't found in select clause.", columnName));
-    }
-
-    /**
-     * For sparse composite, returns wheter two columns belong to the same
-     * cqlRow base on the full list of component in the name.
-     * Two columns do belong together if they differ only by the last
-     * component.
-     */
-    private static boolean isSameRow(ByteBuffer[] c1, ByteBuffer[] c2)
-    {
-        // Cql don't allow to insert columns who doesn't have all component of
-        // the composite set for sparse composite. Someone coming from thrift
-        // could hit that though. But since we have no way to handle this
-        // correctly, better fail here and tell whomever may hit that (if
-        // someone ever do) to change the definition to a dense composite
-        assert c1.length == c2.length : "Sparse composite should not have partial column names";
-        for (int i = 0; i < c1.length - 1; i++)
-        {
-            if (!c1[i].equals(c2[i]))
-                return false;
-        }
-        return true;
-    }
-
-    private void handleGroup(Selection selection, Selection.ResultSetBuilder result, ByteBuffer key, ByteBuffer[] keyComponents, ColumnGroupMap columns) throws InvalidRequestException
+    private void handleGroup(Selection selection, Selection.ResultSetBuilder result, ByteBuffer[] keyComponents, ColumnGroupMap columns) throws InvalidRequestException
     {
         // Respect requested order
         result.newRow();
@@ -870,22 +835,6 @@ public class SelectStatement implements CQLStatement
         return true;
     }
 
-    private boolean hasIndexedColumnRestricted(List<ColumnDefinition> columns, Restriction[] restrictions)
-    {
-        assert columns.size() == restrictions.length;
-        for (int i = 0; i < columns.size(); ++i)
-        {
-            Restriction restriction = restrictions[i];
-            if (restriction == null)
-                continue;
-
-            ColumnDefinition def = columns.get(i);
-            if (def != null && def.isIndexed())
-                return true;
-        }
-        return false;
-    }
-
     public static class RawStatement extends CFStatement
     {
         private final Parameters parameters;
@@ -935,7 +884,12 @@ public class SelectStatement implements CQLStatement
             {
                 CFDefinition.Name name = cfDef.get(rel.getEntity());
                 if (name == null)
-                    throw new InvalidRequestException(String.format("Undefined name %s in where clause ('%s')", rel.getEntity(), rel));
+                {
+                    if (containsAlias(rel.getEntity()))
+                        throw new InvalidRequestException(String.format("Aliases aren't allowed in where clause ('%s')", rel));
+                    else
+                        throw new InvalidRequestException(String.format("Undefined name %s in where clause ('%s')", rel.getEntity(), rel));
+                }
 
                 ColumnDefinition def = cfDef.cfm.getColumnDefinition(name.name.key);
                 stmt.restrictedNames.add(name);
@@ -1142,24 +1096,47 @@ public class SelectStatement implements CQLStatement
                 // If we order an IN query, we'll have to do a manual sort post-query. Currently, this sorting requires that we
                 // have queried the column on which we sort (TODO: we should update it to add the column on which we sort to the one
                 // queried automatically, and then removing it from the resultSet afterwards if needed)
-                if (stmt.keyIsInRelation && !selectClause.isEmpty()) // empty means wildcard was used
+                if (stmt.keyIsInRelation)
                 {
+                    stmt.orderingIndexes = new HashMap<CFDefinition.Name, Integer>();
                     for (ColumnIdentifier column : stmt.parameters.orderings.keySet())
                     {
-                        CFDefinition.Name name = cfDef.get(column);
+                        final CFDefinition.Name name = cfDef.get(column);
+                        if (name == null)
+                        {
+                            if (containsAlias(column))
+                                throw new InvalidRequestException(String.format("Aliases are not allowed in order by clause ('%s')", column));
+                            else
+                                throw new InvalidRequestException(String.format("Order by on unknown column %s", column));
+                        }
 
-                        boolean hasColumn = false;
-                        for (RawSelector selector : selectClause)
+                        if (selectClause.isEmpty()) // wildcard
+                        {
+                            stmt.orderingIndexes.put(name, Iterables.indexOf(cfDef, new Predicate<CFDefinition.Name>()
+                                                                                    {
+                                                                                        public boolean apply(CFDefinition.Name n)
+                                                                                        {
+                                                                                            return name.equals(n);
+                                                                                        }
+                                                                                    }));
+                        }
+                        else
                         {
-                            if (name.name.equals(selector))
+                            boolean hasColumn = false;
+                            for (int i = 0; i < selectClause.size(); i++)
                             {
-                                hasColumn = true;
-                                break;
+                                RawSelector selector = selectClause.get(i);
+                                if (name.name.equals(selector.selectable))
+                                {
+                                    stmt.orderingIndexes.put(name, i);
+                                    hasColumn = true;
+                                    break;
+                                }
                             }
-                        }
 
-                        if (!hasColumn)
-                            throw new InvalidRequestException("ORDER BY could not be used on columns missing in select clause.");
+                            if (!hasColumn)
+                                throw new InvalidRequestException("ORDER BY could not be used on columns missing in select clause.");
+                        }
                     }
                 }
 
@@ -1172,7 +1149,12 @@ public class SelectStatement implements CQLStatement
 
                     CFDefinition.Name name = cfDef.get(column);
                     if (name == null)
-                        throw new InvalidRequestException(String.format("Order by on unknown column %s", column));
+                    {
+                        if (containsAlias(column))
+                            throw new InvalidRequestException(String.format("Aliases are not allowed in order by clause ('%s')", column));
+                        else
+                            throw new InvalidRequestException(String.format("Order by on unknown column %s", column));
+                    }
 
                     if (name.kind != CFDefinition.Name.Kind.COLUMN_ALIAS)
                         throw new InvalidRequestException(String.format("Order by is currently only supported on the clustered columns of the PRIMARY KEY, got %s", column));
@@ -1217,6 +1199,17 @@ public class SelectStatement implements CQLStatement
             return new ParsedStatement.Prepared(stmt, Arrays.<ColumnSpecification>asList(names));
         }
 
+        private boolean containsAlias(final ColumnIdentifier name)
+        {
+            return Iterables.any(selectClause, new Predicate<RawSelector>()
+                                               {
+                                                   public boolean apply(RawSelector raw)
+                                                   {
+                                                       return name.equals(raw.alias);
+                                                   }
+                                               });
+        }
+
         Restriction updateRestriction(CFDefinition.Name name, Restriction restriction, Relation newRel, ColumnSpecification[] boundNames) throws InvalidRequestException
         {
             ColumnSpecification receiver = name;
@@ -1327,16 +1320,6 @@ public class SelectStatement implements CQLStatement
             return eqValues != null;
         }
 
-        public void setBound(Bound b, Term t)
-        {
-            bounds[b.idx] = t;
-        }
-
-        public void setInclusive(Bound b)
-        {
-            boundInclusive[b.idx] = true;
-        }
-
         public Term bound(Bound b)
         {
             return bounds[b.idx];
@@ -1373,8 +1356,8 @@ public class SelectStatement implements CQLStatement
 
         public void setBound(ColumnIdentifier name, Relation.Type type, Term t) throws InvalidRequestException
         {
-            Bound b = null;
-            boolean inclusive = false;
+            Bound b;
+            boolean inclusive;
             switch (type)
             {
                 case GT:
@@ -1393,6 +1376,8 @@ public class SelectStatement implements CQLStatement
                     b = Bound.END;
                     inclusive = true;
                     break;
+                default:
+                    throw new AssertionError();
             }
 
             if (bounds == null)
@@ -1428,13 +1413,15 @@ public class SelectStatement implements CQLStatement
         private final int limit;
         private final Map<ColumnIdentifier, Boolean> orderings;
         private final boolean isCount;
+        private final ColumnIdentifier countAlias;
         private final boolean allowFiltering;
 
-        public Parameters(int limit, Map<ColumnIdentifier, Boolean> orderings, boolean isCount, boolean allowFiltering)
+        public Parameters(int limit, Map<ColumnIdentifier, Boolean> orderings, boolean isCount, ColumnIdentifier countAlias, boolean allowFiltering)
         {
             this.limit = limit;
             this.orderings = orderings;
             this.isCount = isCount;
+            this.countAlias = countAlias;
             this.allowFiltering = allowFiltering;
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/08df0b86/src/java/org/apache/cassandra/cql3/statements/Selectable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/Selectable.java b/src/java/org/apache/cassandra/cql3/statements/Selectable.java
new file mode 100644
index 0000000..9f25542
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/Selectable.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.cassandra.cql3.statements;
+
+import java.util.List;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+
+public interface Selectable
+{
+    public static class WritetimeOrTTL implements Selectable
+    {
+        public final ColumnIdentifier id;
+        public final boolean isWritetime;
+
+        public WritetimeOrTTL(ColumnIdentifier id, boolean isWritetime)
+        {
+            this.id = id;
+            this.isWritetime = isWritetime;
+        }
+
+        @Override
+        public String toString()
+        {
+            return (isWritetime ? "writetime" : "ttl") + "(" + id + ")";
+        }
+    }
+
+    public static class WithFunction implements Selectable
+    {
+        public final String functionName;
+        public final List<Selectable> args;
+
+        public WithFunction(String functionName, List<Selectable> args)
+        {
+            this.functionName = functionName;
+            this.args = args;
+        }
+
+        @Override
+        public String toString()
+        {
+            StringBuilder sb = new StringBuilder();
+            sb.append(functionName).append("(");
+            for (int i = 0; i < args.size(); i++)
+            {
+                if (i > 0) sb.append(", ");
+                sb.append(args.get(i));
+            }
+            return sb.append(")").toString();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/08df0b86/src/java/org/apache/cassandra/cql3/statements/Selection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/Selection.java b/src/java/org/apache/cassandra/cql3/statements/Selection.java
index 1ed92e9..d3018e5 100644
--- a/src/java/org/apache/cassandra/cql3/statements/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Selection.java
@@ -34,13 +34,8 @@ import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 public abstract class Selection
 {
-    private static final Logger logger = LoggerFactory.getLogger(Selection.class);
-
     private final List<CFDefinition.Name> columnsList;
     private final List<ColumnSpecification> metadata;
     private final boolean collectTimestamps;
@@ -66,7 +61,7 @@ public abstract class Selection
     {
         for (RawSelector rawSelector : rawSelectors)
         {
-            if (!(rawSelector instanceof ColumnIdentifier))
+            if (!(rawSelector.selectable instanceof ColumnIdentifier))
                 return true;
         }
         return false;
@@ -85,18 +80,18 @@ public abstract class Selection
 
     private static Selector makeSelector(CFDefinition cfDef, RawSelector raw, List<CFDefinition.Name> names, List<ColumnSpecification> metadata) throws InvalidRequestException
     {
-        if (raw instanceof ColumnIdentifier)
+        if (raw.selectable instanceof ColumnIdentifier)
         {
-            CFDefinition.Name name = cfDef.get((ColumnIdentifier)raw);
+            CFDefinition.Name name = cfDef.get((ColumnIdentifier)raw.selectable);
             if (name == null)
-                throw new InvalidRequestException(String.format("Undefined name %s in selection clause", raw));
+                throw new InvalidRequestException(String.format("Undefined name %s in selection clause", raw.selectable));
             if (metadata != null)
-                metadata.add(name);
+                metadata.add(raw.alias == null ? name : makeAliasSpec(cfDef, name.type, raw.alias));
             return new SimpleSelector(name.toString(), addAndGetIndex(name, names), name.type);
         }
-        else if (raw instanceof RawSelector.WritetimeOrTTL)
+        else if (raw.selectable instanceof Selectable.WritetimeOrTTL)
         {
-            RawSelector.WritetimeOrTTL tot = (RawSelector.WritetimeOrTTL)raw;
+            Selectable.WritetimeOrTTL tot = (Selectable.WritetimeOrTTL)raw.selectable;
             CFDefinition.Name name = cfDef.get(tot.id);
             if (name == null)
                 throw new InvalidRequestException(String.format("Undefined name %s in selection clause", tot.id));
@@ -106,20 +101,20 @@ public abstract class Selection
                 throw new InvalidRequestException(String.format("Cannot use selection function %s on collections", tot.isWritetime ? "writeTime" : "ttl"));
 
             if (metadata != null)
-                metadata.add(makeWritetimeOrTTLSpec(cfDef, tot));
+                metadata.add(makeWritetimeOrTTLSpec(cfDef, tot, raw.alias));
             return new WritetimeOrTTLSelector(name.toString(), addAndGetIndex(name, names), tot.isWritetime);
         }
         else
         {
-            RawSelector.WithFunction withFun = (RawSelector.WithFunction)raw;
+            Selectable.WithFunction withFun = (Selectable.WithFunction)raw.selectable;
             List<Selector> args = new ArrayList<Selector>(withFun.args.size());
-            for (RawSelector rawArg : withFun.args)
-                args.add(makeSelector(cfDef, rawArg, names, null));
+            for (Selectable rawArg : withFun.args)
+                args.add(makeSelector(cfDef, new RawSelector(rawArg, null), names, null));
 
             AbstractType<?> returnType = Functions.getReturnType(withFun.functionName, cfDef.cfm.ksName, cfDef.cfm.cfName);
             if (returnType == null)
                 throw new InvalidRequestException(String.format("Unknown function '%s'", withFun.functionName));
-            ColumnSpecification spec = makeFunctionSpec(cfDef, withFun, returnType);
+            ColumnSpecification spec = makeFunctionSpec(cfDef, withFun, returnType, raw.alias);
             Function fun = Functions.get(withFun.functionName, args, spec);
             if (metadata != null)
                 metadata.add(spec);
@@ -127,25 +122,33 @@ public abstract class Selection
         }
     }
 
-    private static ColumnSpecification makeWritetimeOrTTLSpec(CFDefinition cfDef, RawSelector.WritetimeOrTTL tot)
+    private static ColumnSpecification makeWritetimeOrTTLSpec(CFDefinition cfDef, Selectable.WritetimeOrTTL tot, ColumnIdentifier alias)
     {
         return new ColumnSpecification(cfDef.cfm.ksName,
                                        cfDef.cfm.cfName,
-                                       new ColumnIdentifier(tot.toString(), true),
+                                       alias == null ? new ColumnIdentifier(tot.toString(), true) : alias,
                                        tot.isWritetime ? LongType.instance : Int32Type.instance);
     }
 
-    private static ColumnSpecification makeFunctionSpec(CFDefinition cfDef, RawSelector.WithFunction fun, AbstractType<?> returnType) throws InvalidRequestException
+    private static ColumnSpecification makeFunctionSpec(CFDefinition cfDef,
+                                                        Selectable.WithFunction fun,
+                                                        AbstractType<?> returnType,
+                                                        ColumnIdentifier alias) throws InvalidRequestException
     {
         if (returnType == null)
             throw new InvalidRequestException(String.format("Unknown function %s called in selection clause", fun.functionName));
 
         return new ColumnSpecification(cfDef.cfm.ksName,
                                        cfDef.cfm.cfName,
-                                       new ColumnIdentifier(fun.toString(), true),
+                                       alias == null ? new ColumnIdentifier(fun.toString(), true) : alias,
                                        returnType);
     }
 
+    private static ColumnSpecification makeAliasSpec(CFDefinition cfDef, AbstractType<?> type, ColumnIdentifier alias)
+    {
+        return new ColumnSpecification(cfDef.cfm.ksName, cfDef.cfm.cfName, alias, type);
+    }
+
     public static Selection fromSelectors(CFDefinition cfDef, List<RawSelector> rawSelectors) throws InvalidRequestException
     {
         boolean usesFunction = isUsingFunction(rawSelectors);
@@ -172,15 +175,17 @@ public abstract class Selection
         else
         {
             List<CFDefinition.Name> names = new ArrayList<CFDefinition.Name>(rawSelectors.size());
+            List<ColumnSpecification> metadata = new ArrayList<ColumnSpecification>(rawSelectors.size());
             for (RawSelector rawSelector : rawSelectors)
             {
-                assert rawSelector instanceof ColumnIdentifier;
-                CFDefinition.Name name = cfDef.get((ColumnIdentifier)rawSelector);
+                assert rawSelector.selectable instanceof ColumnIdentifier;
+                CFDefinition.Name name = cfDef.get((ColumnIdentifier)rawSelector.selectable);
                 if (name == null)
-                    throw new InvalidRequestException(String.format("Undefined name %s in selection clause", rawSelector));
+                    throw new InvalidRequestException(String.format("Undefined name %s in selection clause", rawSelector.selectable));
                 names.add(name);
+                metadata.add(rawSelector.alias == null ? name : makeAliasSpec(cfDef, name.type, rawSelector.alias));
             }
-            return new SimpleSelection(names);
+            return new SimpleSelection(names, metadata);
         }
     }
 
@@ -259,7 +264,7 @@ public abstract class Selection
             {
                 int ttl = -1;
                 if (!isDead(c) && c instanceof ExpiringColumn)
-                    ttl = ((ExpiringColumn)c).getLocalDeletionTime() - (int) (System.currentTimeMillis() / 1000);
+                    ttl = c.getLocalDeletionTime() - (int) (System.currentTimeMillis() / 1000);
                 ttls[current.size() - 1] = ttl;
             }
         }
@@ -292,12 +297,17 @@ public abstract class Selection
     {
         public SimpleSelection(List<CFDefinition.Name> columnsList)
         {
+            this(columnsList, new ArrayList<ColumnSpecification>(columnsList));
+        }
+
+        public SimpleSelection(List<CFDefinition.Name> columnsList, List<ColumnSpecification> metadata)
+        {
             /*
              * In theory, even a simple selection could have multiple time the same column, so we
              * could filter those duplicate out of columnsList. But since we're very unlikely to
              * get much duplicate in practice, it's more efficient not to bother.
              */
-            super(columnsList, new ArrayList<ColumnSpecification>(columnsList), false, false);
+            super(columnsList, metadata, false, false);
         }
 
         protected List<ByteBuffer> handleRow(ResultSetBuilder rs)