You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by bl...@apache.org on 2017/03/07 16:50:54 UTC

cassandra git commit: Add the currentTimestamp, currentDate, currentTime and currentTimeUUID functions

Repository: cassandra
Updated Branches:
  refs/heads/trunk ae0604f08 -> f3cd28d85


Add the currentTimestamp, currentDate, currentTime and currentTimeUUID functions

patch by Benjamin Lerer; reviewed by Alex Petrov for CASSANDRA-13132


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

Branch: refs/heads/trunk
Commit: f3cd28d8595774b70027997fff06d11b1b88c651
Parents: ae0604f
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Mar 7 17:49:14 2017 +0100
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Mar 7 17:49:14 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 NEWS.txt                                        |  2 ++
 doc/source/cql/changes.rst                      |  9 ++++++-
 doc/source/cql/functions.rst                    | 25 +++++++++++++++++++-
 .../apache/cassandra/cql3/QueryProcessor.java   |  2 +-
 .../cassandra/cql3/functions/TimeFcts.java      | 10 +++++---
 .../apache/cassandra/db/marshal/TimeType.java   |  9 +++++++
 .../org/apache/cassandra/schema/Schema.java     |  2 +-
 8 files changed, 53 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f3cd28d8/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 6cf571a..3c06d32 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0
+ * Add the currentTimestamp, currentDate, currentTime and currentTimeUUID functions (CASSANDRA-13132)
  * Remove config option index_interval (CASSANDRA-10671)
  * Reduce lock contention for collection types and serializers (CASSANDRA-13271)
  * Make it possible to override MessagingService.Verb ids (CASSANDRA-13283)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f3cd28d8/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 027786d7..cfb48e2 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -18,6 +18,8 @@ using the provided 'sstableupgrade' tool.
 
 New features
 ------------
+   - The currentTimestamp, currentDate, currentTime and currentTimeUUID functions have been added.
+     See CASSANDRA-13132
    - Support for arithmetic operations between `timestamp`/`date` and `duration` has been added.
      See CASSANDRA-11936
    - Support for arithmetic operations on number has been added. See CASSANDRA-11935

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f3cd28d8/doc/source/cql/changes.rst
----------------------------------------------------------------------
diff --git a/doc/source/cql/changes.rst b/doc/source/cql/changes.rst
index 08ee5d1..6691f15 100644
--- a/doc/source/cql/changes.rst
+++ b/doc/source/cql/changes.rst
@@ -21,6 +21,14 @@ Changes
 
 The following describes the changes in each version of CQL.
 
+3.4.5
+^^^^^
+
+- Adds support for arithmetic operators (:jira:`11935`)
+- Adds support for ``+`` and ``-`` operations on dates (:jira:`11936`)
+- Adds ``currentTimestamp``, ``currentDate``, ``currentTime`` and ``currentTimeUUID`` functions (:jira:`13132`)
+
+
 3.4.4
 ^^^^^
 
@@ -34,7 +42,6 @@ The following describes the changes in each version of CQL.
 - Support for ``GROUP BY`` (:jira:`10707`).
 - Adds a ``DEFAULT UNSET`` option for ``INSERT JSON`` to ignore omitted columns (:jira:`11424`).
 - Allows ``null`` as a legal value for TTL on insert and update. It will be treated as equivalent to inserting a 0 (:jira:`12216`).
-- Adds support for arithmetic operators (:jira:`11935`)
 
 3.4.2
 ^^^^^

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f3cd28d8/doc/source/cql/functions.rst
----------------------------------------------------------------------
diff --git a/doc/source/cql/functions.rst b/doc/source/cql/functions.rst
index 6ae7cbf..965125a 100644
--- a/doc/source/cql/functions.rst
+++ b/doc/source/cql/functions.rst
@@ -143,6 +143,8 @@ time the function is invoked. Note that this method is useful for insertion but
 
 will never return any result by design, since the value returned by ``now()`` is guaranteed to be unique.
 
+``currentTimeUUID`` is an alias of ``now``.
+
 ``minTimeuuid`` and ``maxTimeuuid``
 ###################################
 
@@ -164,8 +166,29 @@ maxTimeuuid('2013-01-01 00:05+0000')``.
    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*.
 
+Datetime functions
+``````````````````
+
+Retrieving the current date/time
+################################
+
+The following functions can be used to retrieve the date/time at the time where the function is invoked:
+
+===================== ===============
+ Function name         Output type
+===================== ===============
+ ``currentTimestamp``  ``timestamp``
+ ``currentDate``       ``date``
+ ``currentTime``       ``time``
+ ``currentTimeUUID``   ``timeUUID``
+===================== ===============
+
+For example the last 2 days of data can be retrieved using::
+
+    SELECT * FROM myTable WHERE date >= currentDate() - 2d
+
 Time conversion functions
-`````````````````````````
+#########################
 
 A number of functions are provided to \u201cconvert\u201d a ``timeuuid``, a ``timestamp`` or a ``date`` into another ``native``
 type.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f3cd28d8/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 cf0e777..4aa2026 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -63,7 +63,7 @@ import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
 
 public class QueryProcessor implements QueryHandler
 {
-    public static final CassandraVersion CQL_VERSION = new CassandraVersion("3.4.4");
+    public static final CassandraVersion CQL_VERSION = new CassandraVersion("3.4.5");
 
     public static final QueryProcessor instance = new QueryProcessor();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f3cd28d8/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
index 944f183..881f636 100644
--- a/src/java/org/apache/cassandra/cql3/functions/TimeFcts.java
+++ b/src/java/org/apache/cassandra/cql3/functions/TimeFcts.java
@@ -36,7 +36,11 @@ public abstract class TimeFcts
 
     public static Collection<Function> all()
     {
-        return ImmutableList.of(now(TimeUUIDType.instance),
+        return ImmutableList.of(now("now", TimeUUIDType.instance),
+                                now("currenttimeuuid", TimeUUIDType.instance),
+                                now("currenttimestamp", TimestampType.instance),
+                                now("currentdate", SimpleDateType.instance),
+                                now("currenttime", TimeType.instance),
                                 minTimeuuidFct,
                                 maxTimeuuidFct,
                                 dateOfFct,
@@ -50,9 +54,9 @@ public abstract class TimeFcts
                                 toTimestamp(SimpleDateType.instance));
     }
 
-    public static final Function now(final TemporalType<?> type)
+    public static final Function now(final String name, final TemporalType<?> type)
     {
-        return new NativeScalarFunction("now", type)
+        return new NativeScalarFunction(name, type)
         {
             @Override
             public ByteBuffer execute(ProtocolVersion protocolVersion, List<ByteBuffer> parameters)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f3cd28d8/src/java/org/apache/cassandra/db/marshal/TimeType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/TimeType.java b/src/java/org/apache/cassandra/db/marshal/TimeType.java
index 4b9d3a1..be20ba7 100644
--- a/src/java/org/apache/cassandra/db/marshal/TimeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TimeType.java
@@ -18,6 +18,9 @@
 package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
 
 import org.apache.cassandra.cql3.Constants;
 import org.apache.cassandra.cql3.Term;
@@ -75,4 +78,10 @@ public class TimeType extends TemporalType<Long>
     {
         return TimeSerializer.instance;
     }
+
+    @Override
+    public ByteBuffer now()
+    {
+        return decompose(LocalTime.now(ZoneOffset.UTC).toNanoOfDay());
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f3cd28d8/src/java/org/apache/cassandra/schema/Schema.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/Schema.java b/src/java/org/apache/cassandra/schema/Schema.java
index 5786804..6dc63aa 100644
--- a/src/java/org/apache/cassandra/schema/Schema.java
+++ b/src/java/org/apache/cassandra/schema/Schema.java
@@ -486,7 +486,7 @@ public final class Schema
     public Collection<Function> getFunctions(FunctionName name)
     {
         if (!name.hasKeyspace())
-            throw new IllegalArgumentException(String.format("Function name must be fully quallified: got %s", name));
+            throw new IllegalArgumentException(String.format("Function name must be fully qualified: got %s", name));
 
         KeyspaceMetadata ksm = getKeyspaceMetadata(name.keyspace);
         return ksm == null