You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sn...@apache.org on 2015/06/11 10:20:26 UTC

[2/3] cassandra git commit: Add functions to convert timeuuid to date or time, deprecate dateOf and unixTimestampOf

Add functions to convert timeuuid to date or time, deprecate dateOf and unixTimestampOf

patch by Benjamin Lerer; reviewed by Robert Stupp for CASSANDRA-9229


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

Branch: refs/heads/trunk
Commit: c08aaabd95d4872593c29807de6ec1485cefa7fa
Parents: 6dfde0e
Author: Benjamin Lerer <be...@datastax.com>
Authored: Thu Jun 11 10:18:05 2015 +0200
Committer: Robert Stupp <sn...@snazy.de>
Committed: Thu Jun 11 10:18:05 2015 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |   7 +
 doc/cql3/CQL.textile                            |  17 +-
 .../cassandra/cql3/functions/Functions.java     |  17 +-
 .../cassandra/cql3/functions/TimeFcts.java      | 229 +++++++++++++++++++
 .../cassandra/cql3/functions/TimeuuidFcts.java  |  88 -------
 .../cassandra/db/marshal/SimpleDateType.java    |  10 +
 .../cassandra/db/marshal/TimestampType.java     |   5 +
 .../repair/SystemDistributedKeyspace.java       |  12 +-
 .../serializers/SimpleDateSerializer.java       |  18 +-
 .../apache/cassandra/cql3/AggregationTest.java  |  12 +-
 .../org/apache/cassandra/cql3/TypeTest.java     |   9 +-
 test/unit/org/apache/cassandra/cql3/UFTest.java |  10 +-
 .../cassandra/cql3/functions/TimeFctsTest.java  | 206 +++++++++++++++++
 14 files changed, 523 insertions(+), 118 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c08aaabd/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 355eefb..0a03e60 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -24,6 +24,7 @@
  * Revert CASSANDRA-7807 (tracing completion client notifications) (CASSANDRA-9429)
  * Add ability to stop compaction by ID (CASSANDRA-7207)
  * Let CassandraVersion handle SNAPSHOT version (CASSANDRA-9438)
+ * Add functions to convert timeuuid to date or time, deprecate dateOf and unixTimestampOf (CASSANDRA-9229)
 Merged from 2.1:
  * Make nodetool exit with non-0 status on failure (CASSANDRA-9569)
  * (cqlsh) Fix using COPY through SOURCE or -f (CASSANDRA-9083)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c08aaabd/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 9beb911..3c71310 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -88,6 +88,13 @@ New features
    - New `ShortType` (cql smallint). 2-byte signed integer
    - New `SimpleDateType` (cql date). 4-byte unsigned integer
    - New `TimeType` (cql time). 8-byte long
+   - The toDate(timeuuid), toTimestamp(timeuuid) and toUnixTimestamp(timeuuid) functions have been added to allow
+     to convert from timeuuid into date type, timestamp type and bigint raw value.
+     The functions unixTimestampOf(timeuuid) and dateOf(timeuuid) have been deprecated.
+   - The toDate(timestamp) and toUnixTimestamp(timestamp) functions have been added to allow
+     to convert from timestamp into date type and bigint raw value.
+   - The toTimestamp(date) and toUnixTimestamp(date) functions have been added to allow
+     to convert from date into timestamp type and bigint raw value.
 
 
 Upgrading

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c08aaabd/doc/cql3/CQL.textile
----------------------------------------------------------------------
diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile
index 9cf7b23..3755a2d 100644
--- a/doc/cql3/CQL.textile
+++ b/doc/cql3/CQL.textile
@@ -1819,9 +1819,20 @@ will select all rows where the @timeuuid@ column @t@ is strictly older than '201
 
 _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_.
 
-h4. @dateOf@ and @unixTimestampOf@
-
-The @dateOf@ and @unixTimestampOf@ functions take a @timeuuid@ argument and extract the embedded timestamp. However, while the @dateof@ function return it with the @timestamp@ type (that most client, including cqlsh, interpret as a date), the @unixTimestampOf@ function returns it as a @bigint@ raw value.
+h3(#timeFun). Time conversion functions
+
+A number of functions are provided to "convert" a @timeuuid@, a @timestamp@ or a @date@ into another @native@ type.
+
+|_. function name    |_. input type   |_. description|
+|@toDate@            |@timeuuid@      |Converts the @timeuuid@ argument into a @date@ type|
+|@toDate@            |@timestamp@     |Converts the @timestamp@ argument into a @date@ type|
+|@toTimestamp@       |@timeuuid@      |Converts the @timeuuid@ argument into a @timestamp@ type|
+|@toTimestamp@       |@date@          |Converts the @date@ argument into a @timestamp@ type|
+|@toUnixTimestamp@   |@timeuuid@      |Converts the @timeuuid@ argument into a @bigInt@ raw value|
+|@toUnixTimestamp@   |@timestamp@     |Converts the @timestamp@ argument into a @bigInt@ raw value|
+|@toUnixTimestamp@   |@date@          |Converts the @date@ argument into a @bigInt@ raw value|
+|@dateOf@            |@timeuuid@      |Similar to @toTimestamp(timeuuid)@ (DEPRECATED)|
+|@unixTimestampOf@   |@timeuuid@      |Similar to @toUnixTimestamp(timeuuid)@ (DEPRECATED)|
 
 h3(#blobFun). Blob conversion functions
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c08aaabd/src/java/org/apache/cassandra/cql3/functions/Functions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/Functions.java b/src/java/org/apache/cassandra/cql3/functions/Functions.java
index 7ac8039..c940787 100644
--- a/src/java/org/apache/cassandra/cql3/functions/Functions.java
+++ b/src/java/org/apache/cassandra/cql3/functions/Functions.java
@@ -46,11 +46,18 @@ public abstract class Functions
     static
     {
         declare(AggregateFcts.countRowsFunction);
-        declare(TimeuuidFcts.nowFct);
-        declare(TimeuuidFcts.minTimeuuidFct);
-        declare(TimeuuidFcts.maxTimeuuidFct);
-        declare(TimeuuidFcts.dateOfFct);
-        declare(TimeuuidFcts.unixTimestampOfFct);
+        declare(TimeFcts.nowFct);
+        declare(TimeFcts.minTimeuuidFct);
+        declare(TimeFcts.maxTimeuuidFct);
+        declare(TimeFcts.dateOfFct);
+        declare(TimeFcts.unixTimestampOfFct);
+        declare(TimeFcts.timeUuidtoDate);
+        declare(TimeFcts.timeUuidToTimestamp);
+        declare(TimeFcts.timeUuidToUnixTimestamp);
+        declare(TimeFcts.timestampToDate);
+        declare(TimeFcts.timestampToUnixTimestamp);
+        declare(TimeFcts.dateToTimestamp);
+        declare(TimeFcts.dateToUnixTimestamp);
         declare(UuidFcts.uuidFct);
 
         for (CQL3Type type : CQL3Type.Native.values())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c08aaabd/src/java/org/apache/cassandra/cql3/functions/TimeFcts.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/TimeFcts.java b/src/java/org/apache/cassandra/cql3/functions/TimeFcts.java
new file mode 100644
index 0000000..a4623cd
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/TimeFcts.java
@@ -0,0 +1,229 @@
+/*
+ * 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.functions;
+
+import java.nio.ByteBuffer;
+import java.util.Date;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.serializers.TimestampSerializer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.UUIDGen;
+
+public abstract class TimeFcts
+{
+    public static Logger logger = LoggerFactory.getLogger(TimeFcts.class);
+
+    public static final Function nowFct = new NativeScalarFunction("now", TimeUUIDType.instance)
+    {
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+        {
+            return ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes());
+        }
+    };
+
+    public static final Function minTimeuuidFct = new NativeScalarFunction("mintimeuuid", TimeUUIDType.instance, TimestampType.instance)
+    {
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+        {
+            ByteBuffer bb = parameters.get(0);
+            if (bb == null)
+                return null;
+
+            return ByteBuffer.wrap(UUIDGen.decompose(UUIDGen.minTimeUUID(TimestampType.instance.compose(bb).getTime())));
+        }
+    };
+
+    public static final Function maxTimeuuidFct = new NativeScalarFunction("maxtimeuuid", TimeUUIDType.instance, TimestampType.instance)
+    {
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+        {
+            ByteBuffer bb = parameters.get(0);
+            if (bb == null)
+                return null;
+
+            return ByteBuffer.wrap(UUIDGen.decompose(UUIDGen.maxTimeUUID(TimestampType.instance.compose(bb).getTime())));
+        }
+    };
+
+    /**
+     * Function that convert a value of <code>TIMEUUID</code> into a value of type <code>TIMESTAMP</code>.
+     * @deprecated Replaced by the {@link #timeUuidToTimestamp} function
+     */
+    public static final Function dateOfFct = new NativeScalarFunction("dateof", TimestampType.instance, TimeUUIDType.instance)
+    {
+        private volatile boolean hasLoggedDeprecationWarning;
+
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+        {
+            if (!hasLoggedDeprecationWarning)
+            {
+                hasLoggedDeprecationWarning = true;
+                logger.warn("The function 'dateof' is deprecated." +
+                            " Use the function 'toTimestamp' instead.");
+            }
+
+            ByteBuffer bb = parameters.get(0);
+            if (bb == null)
+                return null;
+
+            long timeInMillis = UUIDGen.unixTimestamp(UUIDGen.getUUID(bb));
+            return ByteBufferUtil.bytes(timeInMillis);
+        }
+    };
+
+    /**
+     * Function that convert a value of type <code>TIMEUUID</code> into an UNIX timestamp.
+     * @deprecated Replaced by the {@link #timeUuidToUnixTimestamp} function
+     */
+    public static final Function unixTimestampOfFct = new NativeScalarFunction("unixtimestampof", LongType.instance, TimeUUIDType.instance)
+    {
+        private volatile boolean hasLoggedDeprecationWarning;
+
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+        {
+            if (!hasLoggedDeprecationWarning)
+            {
+                hasLoggedDeprecationWarning = true;
+                logger.warn("The function 'unixtimestampof' is deprecated." +
+                            " Use the function 'toUnixTimestamp' instead.");
+            }
+
+            ByteBuffer bb = parameters.get(0);
+            if (bb == null)
+                return null;
+
+            return ByteBufferUtil.bytes(UUIDGen.unixTimestamp(UUIDGen.getUUID(bb)));
+        }
+    };
+
+    /**
+     * Function that convert a value of <code>TIMEUUID</code> into a value of type <code>DATE</code>.
+     */
+    public static final Function timeUuidtoDate = new NativeScalarFunction("todate", SimpleDateType.instance, TimeUUIDType.instance)
+    {
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+        {
+            ByteBuffer bb = parameters.get(0);
+            if (bb == null)
+                return null;
+
+            long timeInMillis = UUIDGen.unixTimestamp(UUIDGen.getUUID(bb));
+            return SimpleDateType.instance.fromTimeInMillis(timeInMillis);
+        }
+    };
+
+    /**
+     * Function that convert a value of type <code>TIMEUUID</code> into a value of type <code>TIMESTAMP</code>.
+     */
+    public static final Function timeUuidToTimestamp = new NativeScalarFunction("totimestamp", TimestampType.instance, TimeUUIDType.instance)
+    {
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+        {
+            ByteBuffer bb = parameters.get(0);
+            if (bb == null)
+                return null;
+
+            long timeInMillis = UUIDGen.unixTimestamp(UUIDGen.getUUID(bb));
+            return TimestampType.instance.fromTimeInMillis(timeInMillis);
+        }
+    };
+
+    /**
+     * Function that convert a value of type <code>TIMEUUID</code> into an UNIX timestamp.
+     */
+    public static final Function timeUuidToUnixTimestamp = new NativeScalarFunction("tounixtimestamp", LongType.instance, TimeUUIDType.instance)
+    {
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+        {
+            ByteBuffer bb = parameters.get(0);
+            if (bb == null)
+                return null;
+
+            return ByteBufferUtil.bytes(UUIDGen.unixTimestamp(UUIDGen.getUUID(bb)));
+        }
+    };
+
+    /**
+     * Function that convert a value of type <code>TIMESTAMP</code> into an UNIX timestamp.
+     */
+    public static final Function timestampToUnixTimestamp = new NativeScalarFunction("tounixtimestamp", LongType.instance, TimestampType.instance)
+    {
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+        {
+            ByteBuffer bb = parameters.get(0);
+            if (bb == null)
+                return null;
+
+            Date date = TimestampType.instance.compose(bb);
+            return date == null ? null : ByteBufferUtil.bytes(date.getTime());
+        }
+    };
+
+   /**
+    * Function that convert a value of type <code>TIMESTAMP</code> into a <code>DATE</code>.
+    */
+   public static final Function timestampToDate = new NativeScalarFunction("todate", SimpleDateType.instance, TimestampType.instance)
+   {
+       public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+       {
+           ByteBuffer bb = parameters.get(0);
+           if (bb == null)
+               return null;
+
+           Date date = TimestampType.instance.compose(bb);
+           return date == null ? null : SimpleDateType.instance.fromTimeInMillis(date.getTime());
+       }
+   };
+
+   /**
+    * Function that convert a value of type <code>TIMESTAMP</code> into a <code>DATE</code>.
+    */
+   public static final Function dateToTimestamp = new NativeScalarFunction("totimestamp", TimestampType.instance, SimpleDateType.instance)
+   {
+       public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+       {
+           ByteBuffer bb = parameters.get(0);
+           if (bb == null)
+               return null;
+
+           long millis = SimpleDateType.instance.toTimeInMillis(bb);
+           return TimestampType.instance.fromTimeInMillis(millis);
+       }
+   };
+
+   /**
+    * Function that convert a value of type <code>DATE</code> into an UNIX timestamp.
+    */
+   public static final Function dateToUnixTimestamp = new NativeScalarFunction("tounixtimestamp", LongType.instance, SimpleDateType.instance)
+   {
+       public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+       {
+           ByteBuffer bb = parameters.get(0);
+           if (bb == null)
+               return null;
+
+           return ByteBufferUtil.bytes(SimpleDateType.instance.toTimeInMillis(bb));
+       }
+   };
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c08aaabd/src/java/org/apache/cassandra/cql3/functions/TimeuuidFcts.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/TimeuuidFcts.java b/src/java/org/apache/cassandra/cql3/functions/TimeuuidFcts.java
deleted file mode 100644
index d24572b..0000000
--- a/src/java/org/apache/cassandra/cql3/functions/TimeuuidFcts.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cql3.functions;
-
-import java.nio.ByteBuffer;
-import java.util.Date;
-import java.util.List;
-
-import org.apache.cassandra.db.marshal.TimestampType;
-import org.apache.cassandra.db.marshal.TimeUUIDType;
-import org.apache.cassandra.db.marshal.LongType;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.UUIDGen;
-
-public abstract class TimeuuidFcts
-{
-    public static final Function nowFct = new NativeScalarFunction("now", TimeUUIDType.instance)
-    {
-        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
-        {
-            return ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes());
-        }
-    };
-
-    public static final Function minTimeuuidFct = new NativeScalarFunction("mintimeuuid", TimeUUIDType.instance, TimestampType.instance)
-    {
-        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
-        {
-            ByteBuffer bb = parameters.get(0);
-            if (bb == null)
-                return null;
-
-            return ByteBuffer.wrap(UUIDGen.decompose(UUIDGen.minTimeUUID(TimestampType.instance.compose(bb).getTime())));
-        }
-    };
-
-    public static final Function maxTimeuuidFct = new NativeScalarFunction("maxtimeuuid", TimeUUIDType.instance, TimestampType.instance)
-    {
-        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
-        {
-            ByteBuffer bb = parameters.get(0);
-            if (bb == null)
-                return null;
-
-            return ByteBuffer.wrap(UUIDGen.decompose(UUIDGen.maxTimeUUID(TimestampType.instance.compose(bb).getTime())));
-        }
-    };
-
-    public static final Function dateOfFct = new NativeScalarFunction("dateof", TimestampType.instance, TimeUUIDType.instance)
-    {
-        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
-        {
-            ByteBuffer bb = parameters.get(0);
-            if (bb == null)
-                return null;
-
-            return TimestampType.instance.decompose(new Date(UUIDGen.unixTimestamp(UUIDGen.getUUID(bb))));
-        }
-    };
-
-    public static final Function unixTimestampOfFct = new NativeScalarFunction("unixtimestampof", LongType.instance, TimeUUIDType.instance)
-    {
-        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
-        {
-            ByteBuffer bb = parameters.get(0);
-            if (bb == null)
-                return null;
-
-            return ByteBufferUtil.bytes(UUIDGen.unixTimestamp(UUIDGen.getUUID(bb)));
-        }
-    };
-}
-

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c08aaabd/src/java/org/apache/cassandra/db/marshal/SimpleDateType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/SimpleDateType.java b/src/java/org/apache/cassandra/db/marshal/SimpleDateType.java
index 225b9cc..747709e 100644
--- a/src/java/org/apache/cassandra/db/marshal/SimpleDateType.java
+++ b/src/java/org/apache/cassandra/db/marshal/SimpleDateType.java
@@ -50,6 +50,16 @@ public class SimpleDateType extends AbstractType<Integer>
         return ByteBufferUtil.bytes(SimpleDateSerializer.dateStringToDays(source));
     }
 
+    public ByteBuffer fromTimeInMillis(long millis) throws MarshalException
+    {
+        return ByteBufferUtil.bytes(SimpleDateSerializer.timeInMillisToDay(millis));
+    }
+
+    public long toTimeInMillis(ByteBuffer buffer) throws MarshalException
+    {
+        return SimpleDateSerializer.dayToTimeInMillis(ByteBufferUtil.toInt(buffer));
+    }
+
     @Override
     public boolean isCompatibleWith(AbstractType<?> previous)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c08aaabd/src/java/org/apache/cassandra/db/marshal/TimestampType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/TimestampType.java b/src/java/org/apache/cassandra/db/marshal/TimestampType.java
index 38e0296..b01651d 100644
--- a/src/java/org/apache/cassandra/db/marshal/TimestampType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TimestampType.java
@@ -64,6 +64,11 @@ public class TimestampType extends AbstractType<Date>
       return ByteBufferUtil.bytes(TimestampSerializer.dateStringToTimestamp(source));
     }
 
+    public ByteBuffer fromTimeInMillis(long millis) throws MarshalException
+    {
+        return ByteBufferUtil.bytes(millis);
+    }
+
     @Override
     public Term fromJSONObject(Object parsed) throws MarshalException
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c08aaabd/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java b/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
index 0f4bde8..2509597 100644
--- a/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
+++ b/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
@@ -105,14 +105,14 @@ public final class SystemDistributedKeyspace
     {
 
         String query = "INSERT INTO %s.%s (parent_id, keyspace_name, columnfamily_names, requested_ranges, started_at)"+
-                                 " VALUES (%s,        '%s',          { '%s' },           { '%s' },          dateOf(now()))";
+                                 " VALUES (%s,        '%s',          { '%s' },           { '%s' },          toTimestamp(now()))";
         String fmtQry = String.format(query, NAME, PARENT_REPAIR_HISTORY, parent_id.toString(), keyspaceName, Joiner.on("','").join(cfnames), Joiner.on("','").join(ranges));
         processSilent(fmtQry);
     }
 
     public static void failParentRepair(UUID parent_id, Throwable t)
     {
-        String query = "UPDATE %s.%s SET finished_at = dateOf(now()), exception_message=?, exception_stacktrace=? WHERE parent_id=%s";
+        String query = "UPDATE %s.%s SET finished_at = toTimestamp(now()), exception_message=?, exception_stacktrace=? WHERE parent_id=%s";
 
         StringWriter sw = new StringWriter();
         PrintWriter pw = new PrintWriter(sw);
@@ -123,7 +123,7 @@ public final class SystemDistributedKeyspace
 
     public static void successfulParentRepair(UUID parent_id, Collection<Range<Token>> successfulRanges)
     {
-        String query = "UPDATE %s.%s SET finished_at = dateOf(now()), successful_ranges = {'%s'} WHERE parent_id=%s";
+        String query = "UPDATE %s.%s SET finished_at = toTimestamp(now()), successful_ranges = {'%s'} WHERE parent_id=%s";
         String fmtQuery = String.format(query, NAME, PARENT_REPAIR_HISTORY, Joiner.on("','").join(successfulRanges), parent_id.toString());
         processSilent(fmtQuery);
     }
@@ -138,7 +138,7 @@ public final class SystemDistributedKeyspace
 
         String query =
                 "INSERT INTO %s.%s (keyspace_name, columnfamily_name, id, parent_id, range_begin, range_end, coordinator, participants, status, started_at) " +
-                        "VALUES (   '%s',          '%s',              %s, %s,        '%s',        '%s',      '%s',        { '%s' },     '%s',   dateOf(now()))";
+                        "VALUES (   '%s',          '%s',              %s, %s,        '%s',        '%s',      '%s',        { '%s' },     '%s',   toTimestamp(now()))";
 
         for (String cfname : cfnames)
         {
@@ -164,7 +164,7 @@ public final class SystemDistributedKeyspace
 
     public static void successfulRepairJob(UUID id, String keyspaceName, String cfname)
     {
-        String query = "UPDATE %s.%s SET status = '%s', finished_at = dateOf(now()) WHERE keyspace_name = '%s' AND columnfamily_name = '%s' AND id = %s";
+        String query = "UPDATE %s.%s SET status = '%s', finished_at = toTimestamp(now()) WHERE keyspace_name = '%s' AND columnfamily_name = '%s' AND id = %s";
         String fmtQuery = String.format(query, NAME, REPAIR_HISTORY,
                                         RepairState.SUCCESS.toString(),
                                         keyspaceName,
@@ -175,7 +175,7 @@ public final class SystemDistributedKeyspace
 
     public static void failedRepairJob(UUID id, String keyspaceName, String cfname, Throwable t)
     {
-        String query = "UPDATE %s.%s SET status = '%s', finished_at = dateOf(now()), exception_message=?, exception_stacktrace=? WHERE keyspace_name = '%s' AND columnfamily_name = '%s' AND id = %s";
+        String query = "UPDATE %s.%s SET status = '%s', finished_at = toTimestamp(now()), exception_message=?, exception_stacktrace=? WHERE keyspace_name = '%s' AND columnfamily_name = '%s' AND id = %s";
         StringWriter sw = new StringWriter();
         PrintWriter pw = new PrintWriter(sw);
         t.printStackTrace(pw);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c08aaabd/src/java/org/apache/cassandra/serializers/SimpleDateSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/serializers/SimpleDateSerializer.java b/src/java/org/apache/cassandra/serializers/SimpleDateSerializer.java
index 82cedc0..075094c 100644
--- a/src/java/org/apache/cassandra/serializers/SimpleDateSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/SimpleDateSerializer.java
@@ -86,9 +86,7 @@ public class SimpleDateSerializer implements TypeSerializer<Integer>
             if (millis > maxSupportedDateMillis)
                 throw new MarshalException(String.format("Input date %s is greater than max supported date %s", source, new LocalDate(maxSupportedDateMillis).toString()));
 
-            Integer result = (int)TimeUnit.MILLISECONDS.toDays(millis);
-            result -= Integer.MIN_VALUE;
-            return result;
+            return timeInMillisToDay(millis);
         }
         catch (IllegalArgumentException e1)
         {
@@ -96,6 +94,18 @@ public class SimpleDateSerializer implements TypeSerializer<Integer>
         }
     }
 
+    public static int timeInMillisToDay(long millis)
+    {
+        Integer result = (int) TimeUnit.MILLISECONDS.toDays(millis);
+        result -= Integer.MIN_VALUE;
+        return result;
+    }
+
+    public static long dayToTimeInMillis(int days)
+    {
+        return TimeUnit.DAYS.toMillis(days - Integer.MIN_VALUE);
+    }
+
     public void validate(ByteBuffer bytes) throws MarshalException
     {
         if (bytes.remaining() != 4)
@@ -107,7 +117,7 @@ public class SimpleDateSerializer implements TypeSerializer<Integer>
         if (value == null)
             return "";
 
-        return formatter.print(new LocalDate(TimeUnit.DAYS.toMillis(value - Integer.MIN_VALUE), DateTimeZone.UTC));
+        return formatter.print(new LocalDate(dayToTimeInMillis(value), DateTimeZone.UTC));
     }
 
     public Class<Integer> getType()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c08aaabd/test/unit/org/apache/cassandra/cql3/AggregationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/AggregationTest.java b/test/unit/org/apache/cassandra/cql3/AggregationTest.java
index af68ddc..768fdc3 100644
--- a/test/unit/org/apache/cassandra/cql3/AggregationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/AggregationTest.java
@@ -111,10 +111,10 @@ public class AggregationTest extends CQLTester
                                          "LANGUAGE JAVA " +
                                          "AS 'return Double.valueOf(Math.copySign(magnitude, sign));';");
 
-        assertColumnNames(execute("SELECT max(a), max(unixTimestampOf(b)) FROM %s"), "system.max(a)", "system.max(system.unixtimestampof(b))");
-        assertRows(execute("SELECT max(a), max(unixTimestampOf(b)) FROM %s"), row(null, null));
-        assertColumnNames(execute("SELECT max(a), unixTimestampOf(max(b)) FROM %s"), "system.max(a)", "system.unixtimestampof(system.max(b))");
-        assertRows(execute("SELECT max(a), unixTimestampOf(max(b)) FROM %s"), row(null, null));
+        assertColumnNames(execute("SELECT max(a), max(toUnixTimestamp(b)) FROM %s"), "system.max(a)", "system.max(system.tounixtimestamp(b))");
+        assertRows(execute("SELECT max(a), max(toUnixTimestamp(b)) FROM %s"), row(null, null));
+        assertColumnNames(execute("SELECT max(a), toUnixTimestamp(max(b)) FROM %s"), "system.max(a)", "system.tounixtimestamp(system.max(b))");
+        assertRows(execute("SELECT max(a), toUnixTimestamp(max(b)) FROM %s"), row(null, null));
 
         assertColumnNames(execute("SELECT max(" + copySign + "(c, d)) FROM %s"), "system.max(" + copySign + "(c, d))");
         assertRows(execute("SELECT max(" + copySign + "(c, d)) FROM %s"), row((Object) null));
@@ -128,8 +128,8 @@ public class AggregationTest extends CQLTester
         Date date = format.parse("2011-02-03 04:10:00");
         date = DateUtils.truncate(date, Calendar.MILLISECOND);
 
-        assertRows(execute("SELECT max(a), max(unixTimestampOf(b)) FROM %s"), row(3, date.getTime()));
-        assertRows(execute("SELECT max(a), unixTimestampOf(max(b)) FROM %s"), row(3, date.getTime()));
+        assertRows(execute("SELECT max(a), max(toUnixTimestamp(b)) FROM %s"), row(3, date.getTime()));
+        assertRows(execute("SELECT max(a), toUnixTimestamp(max(b)) FROM %s"), row(3, date.getTime()));
 
         assertRows(execute("SELECT " + copySign + "(max(c), min(c)) FROM %s"), row(-1.4));
         assertRows(execute("SELECT " + copySign + "(c, d) FROM %s"), row(1.2), row(-1.3), row(1.4));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c08aaabd/test/unit/org/apache/cassandra/cql3/TypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/TypeTest.java b/test/unit/org/apache/cassandra/cql3/TypeTest.java
index 7e12f9f..0605554 100644
--- a/test/unit/org/apache/cassandra/cql3/TypeTest.java
+++ b/test/unit/org/apache/cassandra/cql3/TypeTest.java
@@ -36,8 +36,13 @@ public class TypeTest extends CQLTester
     public void testDateCompatibility() throws Throwable
     {
         createTable("CREATE TABLE %s (a int, b timestamp, c bigint, d varint, PRIMARY KEY (a, b, c, d))");
-        execute("INSERT INTO %s (a, b, c, d) VALUES (0, unixTimestampOf(now()), dateOf(now()), dateOf(now()))");
-        UntypedResultSet results = execute("SELECT * FROM %s WHERE a=0 AND b < unixTimestampOf(now())");
+
+        execute("INSERT INTO %s (a, b, c, d) VALUES (0, toUnixTimestamp(now()), toTimestamp(now()), toTimestamp(now()))");
+        UntypedResultSet results = execute("SELECT * FROM %s WHERE a=0 AND b < toUnixTimestamp(now())");
+        assertEquals(1, results.size());
+
+        execute("INSERT INTO %s (a, b, c, d) VALUES (1, unixTimestampOf(now()), dateOf(now()), dateOf(now()))");
+        results = execute("SELECT * FROM %s WHERE a=1 AND b < toUnixTimestamp(now())");
         assertEquals(1, results.size());
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c08aaabd/test/unit/org/apache/cassandra/cql3/UFTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/UFTest.java b/test/unit/org/apache/cassandra/cql3/UFTest.java
index db94a4c..5a20c18 100644
--- a/test/unit/org/apache/cassandra/cql3/UFTest.java
+++ b/test/unit/org/apache/cassandra/cql3/UFTest.java
@@ -388,7 +388,7 @@ public class UFTest extends CQLTester
         execute("DROP FUNCTION IF EXISTS " + fSin);
 
         // can't drop native functions
-        assertInvalidMessage("system keyspace is not user-modifiable", "DROP FUNCTION dateof");
+        assertInvalidMessage("system keyspace is not user-modifiable", "DROP FUNCTION totimestamp");
         assertInvalidMessage("system keyspace is not user-modifiable", "DROP FUNCTION uuid");
 
         // sin() no longer exists
@@ -732,10 +732,11 @@ public class UFTest extends CQLTester
     @Test
     public void testFunctionInSystemKS() throws Throwable
     {
-        execute("CREATE OR REPLACE FUNCTION " + KEYSPACE + ".dateof(val timeuuid) " +
+        execute("CREATE OR REPLACE FUNCTION " + KEYSPACE + ".totimestamp(val timeuuid) " +
                 "RETURNS NULL ON NULL INPUT " +
                 "RETURNS timestamp " +
                 "LANGUAGE JAVA\n" +
+
                 "AS 'return null;';");
 
         assertInvalidMessage("system keyspace is not user-modifiable",
@@ -745,10 +746,11 @@ public class UFTest extends CQLTester
                              "LANGUAGE JAVA\n" +
                              "AS 'return null;';");
         assertInvalidMessage("system keyspace is not user-modifiable",
-                             "CREATE OR REPLACE FUNCTION system.dateof(val timeuuid) " +
+                             "CREATE OR REPLACE FUNCTION system.totimestamp(val timeuuid) " +
                              "RETURNS NULL ON NULL INPUT " +
                              "RETURNS timestamp " +
                              "LANGUAGE JAVA\n" +
+
                              "AS 'return null;';");
         assertInvalidMessage("system keyspace is not user-modifiable",
                              "DROP FUNCTION system.now");
@@ -761,7 +763,7 @@ public class UFTest extends CQLTester
                              "LANGUAGE JAVA\n" +
                              "AS 'return null;';");
         assertInvalidMessage("system keyspace is not user-modifiable",
-                             "CREATE OR REPLACE FUNCTION dateof(val timeuuid) " +
+                             "CREATE OR REPLACE FUNCTION totimestamp(val timeuuid) " +
                              "RETURNS NULL ON NULL INPUT " +
                              "RETURNS timestamp " +
                              "LANGUAGE JAVA\n" +

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c08aaabd/test/unit/org/apache/cassandra/cql3/functions/TimeFctsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/functions/TimeFctsTest.java b/test/unit/org/apache/cassandra/cql3/functions/TimeFctsTest.java
new file mode 100644
index 0000000..f6fca20
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/functions/TimeFctsTest.java
@@ -0,0 +1,206 @@
+/*
+ * 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.functions;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+import org.junit.Test;
+
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.SimpleDateType;
+import org.apache.cassandra.db.marshal.TimeUUIDType;
+import org.apache.cassandra.db.marshal.TimestampType;
+import org.apache.cassandra.transport.Server;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.UUIDGen;
+import org.joda.time.DateTime;
+import org.joda.time.DateTimeZone;
+import org.joda.time.format.DateTimeFormat;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class TimeFctsTest
+{
+    @Test
+    public void testMinTimeUuid()
+    {
+        DateTime dateTime = DateTimeFormat.forPattern("yyyy-MM-dd hh:mm:ss")
+                .withZone(DateTimeZone.UTC)
+                .parseDateTime("2015-05-21 11:03:02");
+
+        long timeInMillis = dateTime.getMillis();
+        ByteBuffer input = TimestampType.instance.fromString("2015-05-21 11:03:02+00");
+        ByteBuffer output = executeFunction(TimeFcts.minTimeuuidFct, input);
+        assertEquals(UUIDGen.minTimeUUID(timeInMillis), TimeUUIDType.instance.compose(output));
+    }
+
+    @Test
+    public void testMaxTimeUuid()
+    {
+        DateTime dateTime = DateTimeFormat.forPattern("yyyy-MM-dd hh:mm:ss")
+                .withZone(DateTimeZone.UTC)
+                .parseDateTime("2015-05-21 11:03:02");
+
+        long timeInMillis = dateTime.getMillis();
+        ByteBuffer input = TimestampType.instance.fromString("2015-05-21 11:03:02+00");
+        ByteBuffer output = executeFunction(TimeFcts.maxTimeuuidFct, input);
+        assertEquals(UUIDGen.maxTimeUUID(timeInMillis), TimeUUIDType.instance.compose(output));
+    }
+
+    @Test
+    public void testDateOf()
+    {
+        DateTime dateTime = DateTimeFormat.forPattern("yyyy-MM-dd hh:mm:ss")
+                .withZone(DateTimeZone.UTC)
+                .parseDateTime("2015-05-21 11:03:02");
+
+        long timeInMillis = dateTime.getMillis();
+        ByteBuffer input = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes(timeInMillis, 0));
+        ByteBuffer output = executeFunction(TimeFcts.dateOfFct, input);
+        assertEquals(dateTime.toDate(), TimestampType.instance.compose(output));
+    }
+
+    @Test
+    public void testTimeUuidToTimestamp()
+    {
+        DateTime dateTime = DateTimeFormat.forPattern("yyyy-MM-dd hh:mm:ss")
+                .withZone(DateTimeZone.UTC)
+                .parseDateTime("2015-05-21 11:03:02");
+
+        long timeInMillis = dateTime.getMillis();
+        ByteBuffer input = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes(timeInMillis, 0));
+        ByteBuffer output = executeFunction(TimeFcts.timeUuidToTimestamp, input);
+        assertEquals(dateTime.toDate(), TimestampType.instance.compose(output));
+    }
+
+    @Test
+    public void testUnixTimestampOfFct()
+    {
+        DateTime dateTime = DateTimeFormat.forPattern("yyyy-MM-dd hh:mm:ss")
+                .withZone(DateTimeZone.UTC)
+                .parseDateTime("2015-05-21 11:03:02");
+
+        long timeInMillis = dateTime.getMillis();
+        ByteBuffer input = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes(timeInMillis, 0));
+        ByteBuffer output = executeFunction(TimeFcts.unixTimestampOfFct, input);
+        assertEquals(timeInMillis, LongType.instance.compose(output).longValue());
+    }
+
+    @Test
+    public void testTimeUuidToUnixTimestamp()
+    {
+        DateTime dateTime = DateTimeFormat.forPattern("yyyy-MM-dd hh:mm:ss")
+                .withZone(DateTimeZone.UTC)
+                .parseDateTime("2015-05-21 11:03:02");
+
+        long timeInMillis = dateTime.getMillis();
+        ByteBuffer input = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes(timeInMillis, 0));
+        ByteBuffer output = executeFunction(TimeFcts.timeUuidToUnixTimestamp, input);
+        assertEquals(timeInMillis, LongType.instance.compose(output).longValue());
+    }
+
+    @Test
+    public void testTimeUuidToDate()
+    {
+        DateTime dateTime = DateTimeFormat.forPattern("yyyy-MM-dd hh:mm:ss")
+                .withZone(DateTimeZone.UTC)
+                .parseDateTime("2015-05-21 11:03:02");
+
+        long timeInMillis = dateTime.getMillis();
+        ByteBuffer input = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes(timeInMillis, 0));
+        ByteBuffer output = executeFunction(TimeFcts.timeUuidtoDate, input);
+
+        long expectedTime = DateTimeFormat.forPattern("yyyy-MM-dd")
+                                          .withZone(DateTimeZone.UTC)
+                                          .parseDateTime("2015-05-21")
+                                          .getMillis();
+
+        assertEquals(expectedTime, SimpleDateType.instance.toTimeInMillis(output));
+    }
+
+    @Test
+    public void testDateToTimestamp()
+    {
+        DateTime dateTime = DateTimeFormat.forPattern("yyyy-MM-dd")
+                                          .withZone(DateTimeZone.UTC)
+                                          .parseDateTime("2015-05-21");
+
+        ByteBuffer input = SimpleDateType.instance.fromString("2015-05-21");
+        ByteBuffer output = executeFunction(TimeFcts.dateToTimestamp, input);
+        assertEquals(dateTime.toDate(), TimestampType.instance.compose(output));
+    }
+
+    @Test
+    public void testDateToUnixTimestamp()
+    {
+        DateTime dateTime = DateTimeFormat.forPattern("yyyy-MM-dd")
+                                          .withZone(DateTimeZone.UTC)
+                                          .parseDateTime("2015-05-21");
+
+        ByteBuffer input = SimpleDateType.instance.fromString("2015-05-21");
+        ByteBuffer output = executeFunction(TimeFcts.dateToUnixTimestamp, input);
+        assertEquals(dateTime.getMillis(), LongType.instance.compose(output).longValue());
+    }
+
+    @Test
+    public void testTimestampToDate()
+    {
+        DateTime dateTime = DateTimeFormat.forPattern("yyyy-MM-dd")
+                                          .withZone(DateTimeZone.UTC)
+                                          .parseDateTime("2015-05-21");
+
+        ByteBuffer input = TimestampType.instance.fromString("2015-05-21 11:03:02+00");
+        ByteBuffer output = executeFunction(TimeFcts.timestampToDate, input);
+        assertEquals(dateTime.getMillis(), SimpleDateType.instance.toTimeInMillis(output));
+    }
+
+    @Test
+    public void testTimestampToDateWithEmptyInput()
+    {
+        ByteBuffer output = executeFunction(TimeFcts.timestampToDate, ByteBufferUtil.EMPTY_BYTE_BUFFER);
+        assertNull(output);
+    }
+
+    @Test
+    public void testTimestampToUnixTimestamp()
+    {
+        DateTime dateTime = DateTimeFormat.forPattern("yyyy-MM-dd hh:mm:ss")
+                                          .withZone(DateTimeZone.UTC)
+                                          .parseDateTime("2015-05-21 11:03:02");
+
+        ByteBuffer input = TimestampType.instance.decompose(dateTime.toDate());
+        ByteBuffer output = executeFunction(TimeFcts.timestampToUnixTimestamp, input);
+        assertEquals(dateTime.getMillis(), LongType.instance.compose(output).longValue());
+    }
+
+    @Test
+    public void testTimestampToUnixTimestampWithEmptyInput()
+    {
+        ByteBuffer output = executeFunction(TimeFcts.timestampToUnixTimestamp, ByteBufferUtil.EMPTY_BYTE_BUFFER);
+        assertNull(output);
+    }
+
+    private static ByteBuffer executeFunction(Function function, ByteBuffer input)
+    {
+        List<ByteBuffer> params = Arrays.asList(input);
+        return ((ScalarFunction) function).execute(Server.CURRENT_VERSION, params);
+    }
+}