You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2013/01/18 13:58:05 UTC

git commit: Fix interpreting dates as valid timeuuid and introduce new methods to work with timeuuid

Updated Branches:
  refs/heads/cassandra-1.2 d64dc2eb3 -> 8be7e5c0e


Fix interpreting dates as valid timeuuid and introduce new methods to work with timeuuid

patch by slebresne; reviewed by thobbs for CASSANDRA-4936


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

Branch: refs/heads/cassandra-1.2
Commit: 8be7e5c0eb6b0df2e28c5a1038ef0da0f7e0902e
Parents: d64dc2e
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Jan 18 13:56:32 2013 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Jan 18 13:56:32 2013 +0100

----------------------------------------------------------------------
 CHANGES.txt                                        |    1 +
 NEWS.txt                                           |   14 ++
 doc/cql3/CQL.textile                               |   71 +++++-----
 pylib/cqlshlib/formatting.py                       |    8 +-
 .../apache/cassandra/cql/AlterTableStatement.java  |    2 +-
 .../cassandra/cql/CreateColumnFamilyStatement.java |    2 +-
 src/java/org/apache/cassandra/cql/Term.java        |    2 +-
 src/java/org/apache/cassandra/cql3/Cql.g           |   16 ++-
 .../org/apache/cassandra/cql3/QueryProcessor.java  |    2 +-
 .../cassandra/cql3/statements/SelectStatement.java |  107 ++++++++------
 .../apache/cassandra/cql3/statements/Selector.java |   17 ++-
 .../apache/cassandra/db/marshal/AbstractType.java  |    6 +
 .../apache/cassandra/db/marshal/TimeUUIDType.java  |   65 +++++++++-
 src/java/org/apache/cassandra/utils/UUIDGen.java   |   61 ++++++++-
 14 files changed, 279 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8be7e5c0/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f684d6b..c254bf9 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -38,6 +38,7 @@
  * Don't remove tokens from System table for node we know (CASSANDRA-5121)
  * fix streaming progress report for compresed files (CASSANDRA-5130)
  * Coverage analysis for low-CL queries (CASSANDRA-4858)
+ * Stop interpreting dates as valid timeUUID value (CASSANDRA-4936)
 Merged from 1.1:
  * Simplify CompressedRandomAccessReader to work around JDK FD bug (CASSANDRA-5088)
  * Improve handling a changing target throttle rate mid-compaction (CASSANDRA-5087)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8be7e5c0/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 37d8748..3902d05 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -8,6 +8,20 @@ upgrade, just in case you need to roll back to the previous version.
 (Cassandra version X + 1 will always be able to read data files created
 by version X, but the inverse is not necessarily the case.)
 
+1.2.1
+=====
+
+Upgrading
+---------
+    - In CQL3, date string are no longer accepted as timeuuid value since a
+      date string is not a correct representation of a timeuuid. Instead, new
+      methods (minTimeuuid, maxTimeuuid, now, dateOf, unixTimestampOf) have been
+      introduced to make working on timeuuid from date string easy. cqlsh also
+      does not display timeuuid as date string (since this is a lossy
+      representation), but the new dateOf method can be used instead. Please
+      refer to the reference documentation (http://cassandra.apache.org/doc/cql3/CQL.html)
+      for more detail.
+
 
 1.2
 ===

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8be7e5c0/doc/cql3/CQL.textile
----------------------------------------------------------------------
diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile
index da4eb90..93122b9 100644
--- a/doc/cql3/CQL.textile
+++ b/doc/cql3/CQL.textile
@@ -1,6 +1,6 @@
 <link rel="StyleSheet" href="CQL.css" type="text/css" media="screen">
 
-h1. Cassandra Query Language (CQL) v3.0.0
+h1. Cassandra Query Language (CQL) v3.0.1
 
 
  <span id="tableOfContents">
@@ -13,7 +13,7 @@ h2. CQL Syntax
 
 h3. Preamble
 
-This document describes the Cassandra Query Language (CQL) version 3. CQL v3 is not backward compatible with CQL v2 and differs from it in numerous ways.
+This document describes the Cassandra Query Language (CQL) version 3. CQL v3 is not backward compatible with CQL v2 and differs from it in numerous ways. Note that this document describes the last version of the languages. However, the "changes":#changes section provides the diff between the different versions of CQL v3.
 
 CQL v3 offers a model very close to SQL in the sense that data is put in _tables_ containing _rows_ of _columns_. For that reason, when used in this document, these terms (tables, rows and columns) have the same definition than they have in SQL. But please note that as such, they do *not* refer to the concept of rows and columns found in the internal implementation of Cassandra and in the thrift and CQL v2 API.
 
@@ -697,16 +697,16 @@ p. The following table gives additional informations on the native data types:
 |@int@      |32-bit signed int|
 |@text@     |UTF8 encoded string|
 |@timestamp@|A timestamp. See "Working with dates":#usingdates below for more information.|
-|@timeuuid@ |Type 1 UUID. This is a "conflict-free" timestamp and as @timestamp@, it allows date notation: see "Working with dates":#usingdates below.|
+|@timeuuid@ |Type 1 UUID. This is generally used as a "conflict-free" timestamp. See "Working with @timeuuid@":#usingtimeuuid below.|
 |@uuid@     |Type 1 or type 4 UUID|
 |@varchar@  |UTF8 encoded string|
 |@varint@   |Arbitrary-precision integer|
 
 h3(#usingdates). Working with dates
 
-Values of 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. Values of the @timeuuid@ type also include such timestamp and sort accordingly to said timestamp.
+Values of 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 and timeuuid types can be input in CQL as simple long integers, giving the number of milliseconds since the epoch, as defined above.
+Timestamp can be input in CQL as simple long integers, giving the number of milliseconds since the epoch, as defined above.
 
 They can also be input as string literals in any of the following ISO 8601 formats, each representing the time and date Mar 2, 2011, at 04:05:00 AM, GMT.:
 
@@ -731,6 +731,31 @@ The time of day may also be omitted, if the date is the only piece that matters:
 
 In that case, the time of day will default to 00:00:00, in the specified or default time zone.
 
+h3(#usingtimeuuid). Working with @timeuuid@
+
+Values of the @timeuuid@ type are type 1 "UUID":http://en.wikipedia.org/wiki/Universally_unique_identifier, i.e. UUID that include the timestamp of their generation, and they sort accordingly to said timestamp. They thus serve as conflict-free timestamps.
+
+Valid @timeuuid@ values should be inputed using UUID constants described "here":#constants. However, a number of convenience method are provided to interact with @timeuuid@.
+
+First, the method @now@ generates a new unique timeuuid (at the time where the statement using it is executed). Note that this method is useful for insertion but is largely non-sensical in @WHERE@ clauses. For instance, a query of the form
+
+bc(sample). 
+SELECT * FROM myTable WHERE t = now()
+
+will never return any result by design, since the value returned by @now()@ is guaranteed to be unique.
+
+For querying, the method @minTimeuuid@ (resp. @maxTimeuuid@) takes a date @d@ in argument and returns a _fake_ @timeuuid@ corresponding to the _smallest_ (resp. _biggest_) possible @timeuuid@ having for date @d@. So for instance:
+
+bc(sample). 
+SELECT * FROM myTable WHERE t > maxTimeuuid('2013-01-01 00:05+0000') AND t < minTimeuuid('2013-02-02 10:00+0000')
+
+will select all rows where the @timeuuid@ column @t@ is strictly older than '2013-01-01 00:05+0000' but stricly younger than '2013-02-02 10:00+0000'.  Please note that @t >= maxTimeuuid('2013-01-01 00:05+0000')@ would still _not_ select a @timeuuid@ generated exactly at '2013-01-01 00:05+0000' and is essentially equivalent to @t > maxTimeuuid('2013-01-01 00:05+0000')@.
+
+_Warning_: We called the values generated by @minTimeuuid@ and @maxTimeuuid@ _fake_ UUID because they do no respect the Time-Based UUID generation process specified by the "RFC 4122":http://www.ietf.org/rfc/rfc4122.txt. In particular, the value returned by these 2 methods will not be unique. This means you should only use those methods for querying (as in the example above). Inserting the result of those methods is almost certainly _a bad idea_.
+
+Lastly, the @dateOf@ and @unixTimestampOf@ methods can used in @SELECT@ clauses to extract the timestamp of a @timeuuid@ column in a resultset. The difference between the @dateOf@ and @unixTimestampOf@ is that the former return the extract timestamp as a date, while the latter returns it as a raw timestamp (i.e. a 64 bits integer).
+
+
 h3(#counters). Counters
 
 The @counter@ type is used to define _counter columns_. A counter column is a column whose value is a 64-bit signed integer and on which 2 operations are supported: incrementation and decrementation (see "@UPDATE@":#updateStmt for syntax).  Note the value of a counter cannot be set. A counter doesn't exist until first incremented/decremented, and the first incrementation/decrementation is made as if the previous value was 0. Deletion of counter columns is supported but have some limitations (see the "Cassandra Wiki":http://wiki.apache.org/cassandra/Counters for more information).
@@ -932,6 +957,13 @@ CQL distinguishes between _reserved_ and _non-reserved_ keywords. Reserved keywo
 | @WRITETIME@    | no  |
 
 
+h2(#changes). Changes
+
+The following describes the addition/changes brought for each version of CQL.
+
+h3. 3.0.1
+
+- "Date strings":#usingdates (and timestamps) are no longer accepted as valid @timeuuid@ values. Doing so was a bug in the sense that date string are not valid @timeuuid@, and it was thus resulting in "confusing behaviors":https://issues.apache.org/jira/browse/CASSANDRA-4936.  However, the following new methods have been added to help working with @timeuuid@: @now@, @minTimeuuid@, @maxTimeuuid@ , @dateOf@ and @unixTimestampOf@. See the "section dedicated to these methods":#usingtimeuuid for more detail.
 
 
 h2. Versioning
@@ -942,32 +974,3 @@ Versioning of the CQL language adheres to the "Semantic Versioning":http://semve
 |Major     |The major version _must_ be bumped when backward incompatible changes are introduced. This should rarely occur.|
 |Minor     |Minor version increments occur when new, but backward compatible, functionality is introduced.|
 |Patch     |The patch version is incremented when bugs are fixed.|
-
-h2. Changes
-
-pre.. 
-Tue, 06 Nov 2012 15:03:12 +0200 - Eric Evans and Sylvain Lebresne
- * Update for 1.2 changes to CQL 3
-
-Tue, 24 Apr 2012 15:12:36 +0200 - Sylvain Lebresne
- * Rework whole doc to target CQL 3
-
-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
-
-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
-
-Wed, 10 Aug 2011 11:22:00 -0500 - Eric Evans
- * Improved INSERT vs. UPDATE wording.
- * Documented counter column incr/descr.
-
-Sat, 01 Jun 2011 15:58:00 -0600 - Pavel Yaskevich
- * Updated to support ALTER (CASSANDRA-1709)
-
-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/8be7e5c0/pylib/cqlshlib/formatting.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/formatting.py b/pylib/cqlshlib/formatting.py
index 793a1d0..6322caf 100644
--- a/pylib/cqlshlib/formatting.py
+++ b/pylib/cqlshlib/formatting.py
@@ -122,6 +122,8 @@ def format_value_decimal(val, colormap, **_):
 def format_value_uuid(val, colormap, **_):
     return format_python_formatted_type(val, colormap, 'uuid')
 
+formatter_for('timeuuid')(format_value_uuid)
+
 @formatter_for('inet')
 def formatter_value_inet(val, colormap, **_):
     return format_python_formatted_type(val, colormap, 'inet')
@@ -152,12 +154,6 @@ def format_value_timestamp(val, colormap, time_format, **_):
     bval = strftime(time_format, val)
     return colorme(bval, colormap, 'timestamp')
 
-@formatter_for('timeuuid')
-def format_value_timeuuid(val, colormap, time_format, **_):
-    utime = cqltypes.unix_time_from_uuid1(val)
-    bval = strftime(time_format, utime)
-    return colorme(bval, colormap, 'timestamp')
-
 def strftime(time_format, seconds):
     local = time.localtime(seconds)
     formatted = time.strftime(time_format, local)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8be7e5c0/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
index 7d5acce..a0ebc30 100644
--- a/src/java/org/apache/cassandra/cql/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql/AlterTableStatement.java
@@ -69,7 +69,7 @@ public class AlterTableStatement
         CFMetaData cfm = meta.clone();
 
         ByteBuffer columnName = this.oType == OperationType.OPTS ? null
-                                                                 : meta.comparator.fromString(this.columnName);
+                                                                 : meta.comparator.fromStringCQL2(this.columnName);
 
         switch (oType)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8be7e5c0/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
index d286a8e..a84f93e 100644
--- a/src/java/org/apache/cassandra/cql/CreateColumnFamilyStatement.java
+++ b/src/java/org/apache/cassandra/cql/CreateColumnFamilyStatement.java
@@ -131,7 +131,7 @@ public class CreateColumnFamilyStatement
         {
             try
             {
-                ByteBuffer columnName = comparator.fromString(col.getKey().getText());
+                ByteBuffer columnName = comparator.fromStringCQL2(col.getKey().getText());
                 String validatorClassName = CFPropDefs.comparators.containsKey(col.getValue())
                                           ? CFPropDefs.comparators.get(col.getValue())
                                           : col.getValue();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8be7e5c0/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
index 8cd312b..e2f53d8 100644
--- a/src/java/org/apache/cassandra/cql/Term.java
+++ b/src/java/org/apache/cassandra/cql/Term.java
@@ -90,7 +90,7 @@ public class Term
     {
         try
         {
-            if (!isBindMarker()) return validator.fromString(text);
+            if (!isBindMarker()) return validator.fromStringCQL2(text);
 
             // must be a marker term so check for a CqlBindValue stored in the term
             if (bindIndex == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8be7e5c0/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 146b09f..b866bd1 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -221,8 +221,10 @@ selectClause returns [List<Selector> expr]
 
 selector returns [Selector s]
     : c=cident             { $s = c; }
-    | K_WRITETIME '(' c=cident ')' { $s = new Selector.WithFunction(c, Selector.Function.WRITE_TIME); }
-    | K_TTL '(' c=cident ')'       { $s = new Selector.WithFunction(c, Selector.Function.TTL); }
+    | K_WRITETIME        '(' c=cident ')' { $s = new Selector.WithFunction(c, Selector.Function.WRITE_TIME); }
+    | K_TTL              '(' c=cident ')' { $s = new Selector.WithFunction(c, Selector.Function.TTL); }
+    | K_DATE_OF          '(' c=cident ')' { $s = new Selector.WithFunction(c, Selector.Function.DATE_OF); }
+    | K_UNIXTIMESTAMP_OF '(' c=cident ')' { $s = new Selector.WithFunction(c, Selector.Function.UNIXTIMESTAMP_OF); }
     ;
 
 selectCountClause returns [List<Selector> expr]
@@ -661,6 +663,7 @@ map_literal returns [Map<Term, Term> value]
 
 finalTerm returns [Term term]
     : t=(STRING_LITERAL | UUID | INTEGER | FLOAT | K_TRUE | K_FALSE ) { $term = new Term($t.text, $t.type); }
+    | f=(K_MIN_TIMEUUID | K_MAX_TIMEUUID | K_NOW) '(' (v=(STRING_LITERAL | INTEGER))? ')' { $term = new Term($f.text + "(" + ($v == null ? "" : $v.text) + ")", UUID); }
     ;
 
 term returns [Term term]
@@ -847,6 +850,9 @@ unreserved_keyword returns [String str]
         | K_SUPERUSER
         | K_NOSUPERUSER
         | K_PASSWORD
+        | K_MIN_TIMEUUID
+        | K_MAX_TIMEUUID
+        | K_NOW
         ) { $str = $k.text; }
     | t=native_type { $str = t.toString(); }
     ;
@@ -942,6 +948,12 @@ K_LIST:        L I S T;
 K_TRUE:        T R U E;
 K_FALSE:       F A L S E;
 
+K_MIN_TIMEUUID:     M I N T I M E U U I D;
+K_MAX_TIMEUUID:     M A X T I M E U U I D;
+K_NOW:              N O W;
+K_DATE_OF:          D A T E O F;
+K_UNIXTIMESTAMP_OF: U N I X T I M E S T A M P O F;
+
 // Case-insensitive alpha characters
 fragment A: ('a'|'A');
 fragment B: ('b'|'B');

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8be7e5c0/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 fec45e6..f18cdcc 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -41,7 +41,7 @@ import org.apache.cassandra.utils.SemanticVersion;
 
 public class QueryProcessor
 {
-    public static final SemanticVersion CQL_VERSION = new SemanticVersion("3.0.0");
+    public static final SemanticVersion CQL_VERSION = new SemanticVersion("3.0.1");
 
     private static final Logger logger = LoggerFactory.getLogger(QueryProcessor.class);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8be7e5c0/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 e88784c..1cf6d01 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -52,6 +52,7 @@ import org.apache.cassandra.thrift.ThriftValidation;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.UUIDGen;
 
 /**
  * Encapsulates a completely parsed SELECT query, including the target
@@ -627,7 +628,7 @@ public class SelectStatement implements CQLStatement
             {
                 case WRITE_TIME:
                     cqlRows.addColumnValue(ByteBufferUtil.bytes(c.timestamp()));
-                    break;
+                    return;
                 case TTL:
                     if (c instanceof ExpiringColumn)
                     {
@@ -638,13 +639,31 @@ public class SelectStatement implements CQLStatement
                     {
                         cqlRows.addColumnValue(null);
                     }
-                    break;
+                    return;
             }
         }
-        else
+
+        addReturnValue(cqlRows, s, value(c));
+    }
+
+    private void addReturnValue(ResultSet cqlRows, Selector s, ByteBuffer value)
+    {
+        if (value != null && s.hasFunction())
         {
-            cqlRows.addColumnValue(value(c));
+            switch (s.function())
+            {
+                case DATE_OF:
+                    value = DateType.instance.decompose(new Date(UUIDGen.unixTimestamp(UUIDGen.getUUID(value))));
+                    break;
+                case UNIXTIMESTAMP_OF:
+                    value = ByteBufferUtil.bytes(UUIDGen.unixTimestamp(UUIDGen.getUUID(value)));
+                    break;
+                case WRITE_TIME:
+                case TTL:
+                    throw new AssertionError("Cannot return the timestamp or ttl of a value");
+            }
         }
+        cqlRows.addColumnValue(value);
     }
 
     private ResultSet createResult(List<Pair<CFDefinition.Name, Selector>> selection)
@@ -652,22 +671,9 @@ public class SelectStatement implements CQLStatement
         List<ColumnSpecification> names = new ArrayList<ColumnSpecification>(selection.size());
         for (Pair<CFDefinition.Name, Selector> p : selection)
         {
-            if (p.right.hasFunction())
-            {
-                switch (p.right.function())
-                {
-                    case WRITE_TIME:
-                        names.add(new ColumnSpecification(p.left.ksName, p.left.cfName, new ColumnIdentifier(p.right.toString(), true), LongType.instance));
-                        break;
-                    case TTL:
-                        names.add(new ColumnSpecification(p.left.ksName, p.left.cfName, new ColumnIdentifier(p.right.toString(), true), Int32Type.instance));
-                        break;
-                }
-            }
-            else
-            {
-                names.add(p.left);
-            }
+            names.add(p.right.hasFunction()
+                      ? new ColumnSpecification(p.left.ksName, p.left.cfName, new ColumnIdentifier(p.right.toString(), true), p.right.function().resultType)
+                      : p.left);
         }
         return new ResultSet(names);
     }
@@ -763,20 +769,13 @@ public class SelectStatement implements CQLStatement
                         switch (name.kind)
                         {
                             case KEY_ALIAS:
-                                cqlRows.addColumnValue(keyComponents[name.position]);
+                                addReturnValue(cqlRows, selector, keyComponents[name.position]);
                                 break;
                             case COLUMN_ALIAS:
-                                if (cfDef.isComposite)
-                                {
-                                    if (name.position < components.length)
-                                        cqlRows.addColumnValue(components[name.position]);
-                                    else
-                                        cqlRows.addColumnValue(null);
-                                }
-                                else
-                                {
-                                    cqlRows.addColumnValue(c.name());
-                                }
+                                ByteBuffer val = cfDef.isComposite
+                                               ? (name.position < components.length ? components[name.position] : null)
+                                               : c.name();
+                                addReturnValue(cqlRows, selector, val);
                                 break;
                             case VALUE_ALIAS:
                                 addReturnValue(cqlRows, selector, c);
@@ -821,7 +820,7 @@ public class SelectStatement implements CQLStatement
                     Selector selector = p.right;
                     if (name.kind == CFDefinition.Name.Kind.KEY_ALIAS)
                     {
-                        cqlRows.addColumnValue(keyComponents[name.position]);
+                        addReturnValue(cqlRows, selector, keyComponents[name.position]);
                         continue;
                     }
 
@@ -925,10 +924,10 @@ public class SelectStatement implements CQLStatement
             switch (name.kind)
             {
                 case KEY_ALIAS:
-                    cqlRows.addColumnValue(keyComponents[name.position]);
+                    addReturnValue(cqlRows, selector, keyComponents[name.position]);
                     break;
                 case COLUMN_ALIAS:
-                    cqlRows.addColumnValue(columns.getKeyComponent(name.position));
+                    addReturnValue(cqlRows, selector, columns.getKeyComponent(name.position));
                     break;
                 case VALUE_ALIAS:
                     // This should not happen for SPARSE
@@ -937,14 +936,16 @@ public class SelectStatement implements CQLStatement
                     if (name.type.isCollection())
                     {
                         List<Pair<ByteBuffer, IColumn>> collection = columns.getCollection(name.name.key);
-                        if (collection == null)
-                            cqlRows.addColumnValue(null);
-                        else
-                            cqlRows.addColumnValue(((CollectionType)name.type).serialize(collection));
-                        break;
+                        ByteBuffer value = collection == null
+                                         ? null
+                                         : ((CollectionType)name.type).serialize(collection);
+                        addReturnValue(cqlRows, selector, value);
+                    }
+                    else
+                    {
+                        IColumn c = columns.getSimple(name.name.key);
+                        addReturnValue(cqlRows, selector, c);
                     }
-                    IColumn c = columns.getSimple(name.name.key);
-                    addReturnValue(cqlRows, selector, c);
                     break;
                 default:
                     throw new AssertionError();
@@ -1006,10 +1007,24 @@ public class SelectStatement implements CQLStatement
                     CFDefinition.Name name = cfDef.get(t.id());
                     if (name == null)
                         throw new InvalidRequestException(String.format("Undefined name %s in selection clause", t.id()));
-                    if (t.hasFunction() && name.kind != CFDefinition.Name.Kind.COLUMN_METADATA && name.kind != CFDefinition.Name.Kind.VALUE_ALIAS)
-                        throw new InvalidRequestException(String.format("Cannot use function %s on PRIMARY KEY part %s", t.function(), name));
-                    if (t.hasFunction() && name.type.isCollection())
-                        throw new InvalidRequestException(String.format("Function %s is not supported on collections", t.function()));
+                    if (t.hasFunction())
+                    {
+                        if (name.type.isCollection())
+                            throw new InvalidRequestException(String.format("Function %s is not supported on collections", t.function()));
+                        switch (t.function())
+                        {
+                            case WRITE_TIME:
+                            case TTL:
+                                if (name.kind != CFDefinition.Name.Kind.COLUMN_METADATA && name.kind != CFDefinition.Name.Kind.VALUE_ALIAS)
+                                    throw new InvalidRequestException(String.format("Cannot use function %s on PRIMARY KEY part %s", t.function(), name));
+                                break;
+                            case DATE_OF:
+                            case UNIXTIMESTAMP_OF:
+                                if (!(name.type instanceof TimeUUIDType))
+                                    throw new InvalidRequestException(String.format("Function %s is only allowed on timeuuid columns", t.function()));
+                                break;
+                        }
+                    }
 
                     stmt.selectedNames.add(Pair.create(name, t));
                 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8be7e5c0/src/java/org/apache/cassandra/cql3/statements/Selector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/Selector.java b/src/java/org/apache/cassandra/cql3/statements/Selector.java
index 5847b1c..4f1db00 100644
--- a/src/java/org/apache/cassandra/cql3/statements/Selector.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Selector.java
@@ -22,12 +22,23 @@ import com.google.common.base.Objects;
 
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.db.marshal.*;
 
 public abstract class Selector
 {
     public enum Function
     {
-        WRITE_TIME, TTL;
+        WRITE_TIME      (LongType.instance),
+        TTL             (Int32Type.instance),
+        DATE_OF         (DateType.instance),
+        UNIXTIMESTAMP_OF(LongType.instance);
+
+        public final AbstractType<?> resultType;
+
+        private Function(AbstractType<?> resultType)
+        {
+            this.resultType = resultType;
+        }
 
         @Override
         public String toString()
@@ -38,6 +49,10 @@ public abstract class Selector
                     return "writetime";
                 case TTL:
                     return "ttl";
+                case DATE_OF:
+                    return "dateof";
+                case UNIXTIMESTAMP_OF:
+                    return "unixtimestampof";
             }
             throw new AssertionError();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8be7e5c0/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 5393c0c..828926d 100644
--- a/src/java/org/apache/cassandra/db/marshal/AbstractType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
@@ -138,6 +138,12 @@ 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 abstract void validate(ByteBuffer bytes) throws MarshalException;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8be7e5c0/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 4b7e151..c114775 100644
--- a/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
 import java.util.UUID;
+import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.cassandra.cql.jdbc.JdbcTimeUUID;
@@ -30,6 +31,7 @@ public class TimeUUIDType extends AbstractType<UUID>
     public static final TimeUUIDType instance = new TimeUUIDType();
 
     static final Pattern regexPattern = Pattern.compile("[A-Fa-f0-9]{8}\\-[A-Fa-f0-9]{4}\\-[A-Fa-f0-9]{4}\\-[A-Fa-f0-9]{4}\\-[A-Fa-f0-9]{12}");
+    static final Pattern functionPattern = Pattern.compile("(\\w+)\\((.*)\\)");
 
     TimeUUIDType() {} // singleton
 
@@ -100,7 +102,10 @@ public class TimeUUIDType extends AbstractType<UUID>
         }
     }
 
-    public ByteBuffer fromString(String source) throws MarshalException
+    // 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())
@@ -133,6 +138,64 @@ public class TimeUUIDType extends AbstractType<UUID>
         return idBytes;
     }
 
+    public ByteBuffer fromString(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
+        {
+            Matcher m = functionPattern.matcher(source);
+            if (!m.matches())
+                throw new MarshalException(String.format("Unable to make a time-based UUID from '%s'", source));
+
+            String fct = m.group(1);
+            String arg = m.group(2);
+
+            if (fct.equalsIgnoreCase("minTimeUUID"))
+            {
+                idBytes = decompose(UUIDGen.minTimeUUID(DateType.dateStringToTimestamp(arg)));
+            }
+            else if (fct.equalsIgnoreCase("maxTimeUUID"))
+            {
+                idBytes = decompose(UUIDGen.maxTimeUUID(DateType.dateStringToTimestamp(arg)));
+            }
+            else if (fct.equalsIgnoreCase("now"))
+            {
+                if (!arg.trim().isEmpty())
+                    throw new MarshalException(String.format("The 'now' timeuuid method takes no argument ('%s' provided)", arg));
+
+                idBytes = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes());
+            }
+            else
+            {
+                throw new MarshalException(String.format("Unknown timeuuid method '%s'", fct));
+            }
+        }
+
+        return idBytes;
+    }
+
     public void validate(ByteBuffer bytes) throws MarshalException
     {
         if (bytes.remaining() != 16 && bytes.remaining() != 0)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8be7e5c0/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 7da9b5b..38f7f0e 100644
--- a/src/java/org/apache/cassandra/utils/UUIDGen.java
+++ b/src/java/org/apache/cassandra/utils/UUIDGen.java
@@ -39,6 +39,20 @@ public class UUIDGen
     private static final long START_EPOCH = -12219292800000L;
     private static final long clockSeqAndNode = makeClockSeqAndNode();
 
+    /*
+     * The min and max possible lsb for a UUID.
+     * Note that his is not 0 and all 1's because Cassandra TimeUUIDType
+     * compares the lsb parts as a signed byte array comparison. So the min
+     * value is 8 times -128 and the max is 8 times +127.
+     *
+     * Note that we ignore the uuid variant (namely, MIN_CLOCK_SEQ_AND_NODE
+     * have variant 2 as it should, but MAX_CLOCK_SEQ_AND_NODE have variant 0).
+     * I don't think that has any practical consequence and is more robust in
+     * case someone provides a UUID with a broken variant.
+     */
+    private static final long MIN_CLOCK_SEQ_AND_NODE = 0x8080808080808080L;
+    private static final long MAX_CLOCK_SEQ_AND_NODE = 0x7f7f7f7f7f7f7f7fL;
+
     // placement of this singleton is important.  It needs to be instantiated *AFTER* the other statics.
     private static final UUIDGen instance = new UUIDGen();
 
@@ -92,9 +106,54 @@ public class UUIDGen
     }
 
     /**
+     * Returns the smaller possible type 1 UUID having the provided timestamp.
+     *
+     * <b>Warning:</b> this method should only be used for querying as this
+     * doesn't at all guarantee the uniqueness of the resulting UUID.
+     */
+    public static UUID minTimeUUID(long timestamp)
+    {
+        return new UUID(createTime(fromUnixTimestamp(timestamp)), MIN_CLOCK_SEQ_AND_NODE);
+    }
+
+    /**
+     * Returns the biggest possible type 1 UUID having the provided timestamp.
+     *
+     * <b>Warning:</b> this method should only be used for querying as this
+     * doesn't at all guarantee the uniqueness of the resulting UUID.
+     */
+    public static UUID maxTimeUUID(long timestamp)
+    {
+        // unix timestamp are milliseconds precision, uuid timestamp are 100's
+        // nanoseconds precision. If we ask for the biggest uuid have unix
+        // timestamp 1ms, then we should not extend 100's nanoseconds
+        // precision by taking 10000, but rather 19999.
+        long uuidTstamp = fromUnixTimestamp(timestamp + 1) - 1;
+        return new UUID(createTime(uuidTstamp), MAX_CLOCK_SEQ_AND_NODE);
+    }
+
+    public static long unixTimestamp(UUID uuid) {
+        if (uuid.version() != 1)
+            throw new IllegalArgumentException(String.format("Can only retrieve the unix timestamp for version 1 uuid (provided version %d)", uuid.version()));
+
+        long timestamp = uuid.timestamp();
+        return (timestamp / 10000) + START_EPOCH;
+    }
+
+    private static long fromUnixTimestamp(long tstamp) {
+        return (tstamp - START_EPOCH) * 10000;
+    }
+
+    /**
      * 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>
      *
@@ -188,7 +247,7 @@ public class UUIDGen
         return createTime(nanosSince);
     }
 
-    private long createTime(long nanosSince)
+    private static long createTime(long nanosSince)
     {
         long msb = 0L;
         msb |= (0x00000000ffffffffL & nanosSince) << 32;