You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jm...@apache.org on 2015/03/09 18:28:36 UTC

[1/3] cassandra git commit: Add date and time types

Repository: cassandra
Updated Branches:
  refs/heads/trunk 66a5d3bbe -> 3d11ec5dd


http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/test/unit/org/apache/cassandra/serializers/SimpleDateSerializerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/serializers/SimpleDateSerializerTest.java b/test/unit/org/apache/cassandra/serializers/SimpleDateSerializerTest.java
new file mode 100644
index 0000000..4c0751f
--- /dev/null
+++ b/test/unit/org/apache/cassandra/serializers/SimpleDateSerializerTest.java
@@ -0,0 +1,155 @@
+/**
+ * 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.serializers;
+
+import org.apache.cassandra.db.marshal.SimpleDateType;
+import org.apache.cassandra.utils.Pair;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.sql.Timestamp;
+import java.text.SimpleDateFormat;
+import java.util.*;
+
+public class SimpleDateSerializerTest
+{
+    private static final long millisPerDay = 1000 * 60 * 60 * 24;
+
+    private String dates[] = new String[]
+    {
+            "1970-01-01",
+            "1970-01-02",
+            "1969-12-31",
+            "-0001-01-02",
+            "-5877521-01-02",
+            "2014-01-01",
+            "5881580-01-10",
+            "1920-12-01",
+            "1582-10-19"
+    };
+
+    private static GregorianCalendar testCalendar = new GregorianCalendar();
+    private static SimpleDateFormat dateFormatUTC = new SimpleDateFormat("yyyy-MM-dd");
+
+    {
+        testCalendar.setGregorianChange(new Date(Long.MIN_VALUE));
+        testCalendar.setTimeZone(TimeZone.getTimeZone("UTC"));
+        dateFormatUTC.setCalendar(testCalendar);
+        dateFormatUTC.setLenient(false);
+    }
+
+    @Test
+    public void testDateStringToTimestamp()
+    {
+        List<String> unparsedDates = new ArrayList<>();
+        List<String> badParseResults = new ArrayList<>();
+        for (String date : dates)
+        {
+            try
+            {
+                Integer days = SimpleDateSerializer.dateStringToDays(date);
+                ByteBuffer value = SimpleDateSerializer.instance.serialize(days);
+                Integer deserialized = SimpleDateSerializer.instance.deserialize(value);
+
+                String toStringValue = SimpleDateSerializer.instance.toString(deserialized);
+                if (!date.equals(toStringValue)) {
+                    badParseResults.add(String.format("Failed to parse date correctly.  Expected %s, got %s\n", date, toStringValue));
+                }
+            }
+            catch (MarshalException e)
+            {
+                System.err.println("Got an exception: " + e);
+                unparsedDates.add(date);
+            }
+        }
+        assert unparsedDates.isEmpty() : "Unable to parse: " + unparsedDates;
+        assert badParseResults.isEmpty() : "Incorrect parse results: " + badParseResults;
+    }
+
+    @Test
+    public void testDaysStringToInt()
+    {
+        Integer value = SimpleDateSerializer.dateStringToDays("12345");
+        assert value.compareTo(12345) == 0 : String.format("Failed to parse integer based date.  Expected %s, got %s",
+                12345,
+                value);
+    }
+
+    @Test
+    public void testProlepticRange()
+    {
+        for (int i = 1; i < 31; ++i)
+        {
+            String date = "1582-10-";
+            if (i < 10) date += "0";
+            date += i;
+
+            Integer days = SimpleDateSerializer.dateStringToDays(date);
+
+            ByteBuffer value = SimpleDateType.instance.fromString(days.toString());
+            Integer deserialized = SimpleDateSerializer.instance.deserialize(value);
+
+            // Serialized values are unsigned int, unwrap bits w/overflow
+            deserialized -= Integer.MIN_VALUE;
+
+            Timestamp ts = new Timestamp(deserialized * millisPerDay);
+            testCalendar.setTime(ts);
+
+            Date newDate = testCalendar.getTime();
+            assert (dateFormatUTC.format(newDate)).equals(date) :
+                    String.format("Expected [%s], got [%s]", date, dateFormatUTC.format(newDate).toString());
+        }
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testOutOfBoundsLow()
+    {
+        Integer days = SimpleDateSerializer.dateStringToDays(new Date(Integer.MIN_VALUE * millisPerDay - millisPerDay).toString());
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testOutOfBoundsHigh()
+    {
+        Integer days = SimpleDateSerializer.dateStringToDays(new Date(Integer.MAX_VALUE * millisPerDay + millisPerDay).toString());
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadInput()
+    {
+        Integer days = SimpleDateSerializer.dateStringToDays("12A-01-01");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadMonth()
+    {
+        Integer days = SimpleDateSerializer.dateStringToDays("1000-13-01");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadDay()
+    {
+        Integer days = SimpleDateSerializer.dateStringToDays("1000-12-32");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadDayToMonth()
+    {
+        Integer days = SimpleDateSerializer.dateStringToDays("1000-09-31");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/test/unit/org/apache/cassandra/serializers/TimeSerializerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/serializers/TimeSerializerTest.java b/test/unit/org/apache/cassandra/serializers/TimeSerializerTest.java
new file mode 100644
index 0000000..f8af48c
--- /dev/null
+++ b/test/unit/org/apache/cassandra/serializers/TimeSerializerTest.java
@@ -0,0 +1,242 @@
+/**
+ * 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.serializers;
+
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.TimeUnit;
+
+public class TimeSerializerTest
+{
+    @Test
+    public void testSerializerFromString()
+    {
+        // nano
+        long expected = 5;
+        Long time = TimeSerializer.timeStringToLong("00:00:00.000000005");
+        assert time == expected : String.format("Failed nano conversion.  Expected %s, got %s", expected, time);
+
+        // usec
+        expected = TimeUnit.MICROSECONDS.toNanos(123);
+        time = TimeSerializer.timeStringToLong("00:00:00.000123000");
+        assert time == expected : String.format("Failed usec conversion.  Expected %s, got %s", expected, time);
+
+        // milli
+        expected = TimeUnit.MILLISECONDS.toNanos(123);
+        time = TimeSerializer.timeStringToLong("00:00:00.123000");
+        assert time == expected : String.format("Failed milli conversion.  Expected %s, got %s", expected, time);
+
+        // sec
+        expected = TimeUnit.SECONDS.toNanos(15);
+        time = TimeSerializer.timeStringToLong("00:00:15.000");
+        assert time == expected : String.format("Failed sec conversion.  Expected %s, got %s", expected, time);
+
+        // min
+        expected = TimeUnit.MINUTES.toNanos(13);
+        time = TimeSerializer.timeStringToLong("00:13:00.000");
+        assert time == expected : String.format("Failed min conversion.  Expected %s, got %s", expected, time);
+
+        // hour
+        expected = TimeUnit.HOURS.toNanos(2);
+        time = TimeSerializer.timeStringToLong("02:0:00.000");
+        assert time == expected : String.format("Failed min conversion.  Expected %s, got %s", expected, time);
+
+        // complex
+        expected = buildExpected(4, 31, 12, 123, 456, 789);
+        time = TimeSerializer.timeStringToLong("4:31:12.123456789");
+        assert time == expected : String.format("Failed complex conversion.  Expected %s, got %s", expected, time);
+
+        // upper bound
+        expected = buildExpected(23, 59, 59, 999, 999, 999);
+        time = TimeSerializer.timeStringToLong("23:59:59.999999999");
+        assert time == expected : String.format("Failed upper bounds conversion.  Expected %s, got %s", expected, time);
+
+        // Test partial nano
+        expected = buildExpected(12, 13, 14, 123, 654, 120);
+        time = TimeSerializer.timeStringToLong("12:13:14.12365412");
+        assert time == expected : String.format("Failed partial nano timestring.  Expected %s, got %s", expected, time);
+
+        // Test raw long value
+        expected = 10;
+        time = TimeSerializer.timeStringToLong("10");
+        assert time == expected : String.format("Failed long conversion.  Expected %s, got %s", expected, time);
+
+        // Test 0 long
+        expected = 0;
+        time = TimeSerializer.timeStringToLong("0");
+        assert time == expected : String.format("Failed long conversion.  Expected %s, got %s", expected, time);
+    }
+
+    private long buildExpected(int hour, int minute, int second, int milli, int micro, int nano)
+    {
+        return  TimeUnit.HOURS.toNanos(hour) +
+                TimeUnit.MINUTES.toNanos(minute) +
+                TimeUnit.SECONDS.toNanos(second) +
+                TimeUnit.MILLISECONDS.toNanos(milli) +
+                TimeUnit.MICROSECONDS.toNanos(micro) +
+                nano;
+    }
+
+    @Test
+    public void testSerializerToString()
+    {
+        String source = "00:00:00.000000011";
+        Long time = TimeSerializer.timeStringToLong(source);
+        assert(source.equals(TimeSerializer.instance.toString(time)));
+
+        source = "00:00:00.000012311";
+        time = TimeSerializer.timeStringToLong(source);
+        assert(source.equals(TimeSerializer.instance.toString(time)));
+
+        source = "00:00:00.123000000";
+        time = TimeSerializer.timeStringToLong(source);
+        assert(source.equals(TimeSerializer.instance.toString(time)));
+
+        source = "00:00:12.123450000";
+        time = TimeSerializer.timeStringToLong(source);
+        assert(source.equals(TimeSerializer.instance.toString(time)));
+
+        source = "00:34:12.123450000";
+        time = TimeSerializer.timeStringToLong(source);
+        assert(source.equals(TimeSerializer.instance.toString(time)));
+
+        source = "15:00:12.123450000";
+        time = TimeSerializer.timeStringToLong(source);
+        assert(source.equals(TimeSerializer.instance.toString(time)));
+
+        // boundaries
+        source = "00:00:00.000000000";
+        time = TimeSerializer.timeStringToLong(source);
+        assert(source.equals(TimeSerializer.instance.toString(time)));
+
+        source = "23:59:59.999999999";
+        time = TimeSerializer.timeStringToLong(source);
+        assert(source.equals(TimeSerializer.instance.toString(time)));
+
+        // truncated
+        source = "01:14:18.12";
+        time = TimeSerializer.timeStringToLong(source);
+        String result = TimeSerializer.instance.toString(time);
+        assert(result.equals("01:14:18.120000000"));
+
+        source = "01:14:18.1201";
+        time = TimeSerializer.timeStringToLong(source);
+        result = TimeSerializer.instance.toString(time);
+        assert(result.equals("01:14:18.120100000"));
+
+        source = "01:14:18.1201098";
+        time = TimeSerializer.timeStringToLong(source);
+        result = TimeSerializer.instance.toString(time);
+        assert(result.equals("01:14:18.120109800"));
+    }
+
+    @Test public void testSerialization()
+    {
+        String source = "01:01:01.123123123";
+        Long nt = TimeSerializer.timeStringToLong(source);
+
+        ByteBuffer buf = TimeSerializer.instance.serialize(nt);
+        TimeSerializer.instance.validate(buf);
+
+        Long result = TimeSerializer.instance.deserialize(buf);
+        String strResult = TimeSerializer.instance.toString(result);
+
+        assert(strResult.equals(source));
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadHourLow()
+    {
+        Long time = TimeSerializer.timeStringToLong("-1:0:0.123456789");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadHourHigh()
+    {
+        Long time = TimeSerializer.timeStringToLong("24:0:0.123456789");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadMinuteLow()
+    {
+        Long time = TimeSerializer.timeStringToLong("23:-1:0.123456789");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadMinuteHigh()
+    {
+        Long time = TimeSerializer.timeStringToLong("23:60:0.123456789");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testEmpty()
+    {
+        Long time = TimeSerializer.timeStringToLong("");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadSecondLow()
+    {
+        Long time = TimeSerializer.timeStringToLong("23:59:-1.123456789");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadSecondHigh()
+    {
+        Long time = TimeSerializer.timeStringToLong("23:59:60.123456789");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadSecondHighNoMilli()
+    {
+        Long time = TimeSerializer.timeStringToLong("23:59:60");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadNanoLow()
+    {
+        Long time = TimeSerializer.timeStringToLong("23:59:59.-123456789");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadNanoHigh()
+    {
+        Long time = TimeSerializer.timeStringToLong("23:59:59.1234567899");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadNanoCharacter()
+    {
+        Long time = TimeSerializer.timeStringToLong("23:59:59.12345A789");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testNegativeLongTime()
+    {
+        Long time = TimeSerializer.timeStringToLong("-10");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testRawLongOverflow()
+    {
+        Long input = TimeUnit.DAYS.toNanos(1) + 1;
+        Long time = TimeSerializer.timeStringToLong(input.toString());
+    }
+}


[3/3] cassandra git commit: Merge branch 'cassandra-2.1' into trunk

Posted by jm...@apache.org.
Merge branch 'cassandra-2.1' into trunk

Conflicts:
	pylib/cqlshlib/cql3handling.py


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

Branch: refs/heads/trunk
Commit: 3d11ec5dd3c2bb39150e2f1c634ffcfc89492a47
Parents: 66a5d3b 107545b
Author: Joshua McKenzie <jm...@apache.org>
Authored: Mon Mar 9 12:26:24 2015 -0500
Committer: Joshua McKenzie <jm...@apache.org>
Committed: Mon Mar 9 12:26:24 2015 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |   5 +
 bin/cqlsh                                       |  47 +++-
 doc/cql3/CQL.textile                            |  41 +++-
 doc/native_protocol_v4.spec                     |   2 +
 lib/joda-time-2.4.jar                           | Bin 0 -> 586201 bytes
 lib/licenses/joda-time-2.4.txt                  | 201 +++++++++++++++
 pylib/cqlshlib/cql3handling.py                  |   6 +-
 pylib/cqlshlib/displaying.py                    |   2 +
 pylib/cqlshlib/formatting.py                    |  65 +++--
 pylib/cqlshlib/helptopics.py                    |  23 ++
 pylib/cqlshlib/test/test_cqlsh_output.py        |  17 +-
 pylib/cqlshlib/test/test_keyspace_init.cql      |  41 ++--
 .../org/apache/cassandra/cql3/CQL3Type.java     |  34 +--
 .../org/apache/cassandra/cql3/Constants.java    |   5 +-
 src/java/org/apache/cassandra/cql3/Cql.g        |   4 +
 .../cassandra/db/marshal/IntegerType.java       |   5 +
 .../cassandra/db/marshal/SimpleDateType.java    |  72 ++++++
 .../apache/cassandra/db/marshal/TimeType.java   |  72 ++++++
 .../serializers/SimpleDateSerializer.java       | 117 +++++++++
 .../cassandra/serializers/TimeSerializer.java   | 198 +++++++++++++++
 .../cassandra/serializers/TypeSerializer.java   |   1 +
 .../apache/cassandra/transport/DataType.java    |   2 +
 .../db/marshal/SimpleDateTypeTest.java          | 153 ++++++++++++
 .../cassandra/db/marshal/TimeTypeTest.java      |  61 +++++
 .../serializers/SimpleDateSerializerTest.java   | 155 ++++++++++++
 .../serializers/TimeSerializerTest.java         | 242 +++++++++++++++++++
 27 files changed, 1485 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/3d11ec5d/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 45cab3e,31828fc..34037f4
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,67 -1,5 +1,68 @@@
 +3.0
 + * Make CRC32Ex into a separate maven dependency (CASSANDRA-8836)
 + * Use preloaded jemalloc w/ Unsafe (CASSANDRA-8714)
 + * Add role based access control (CASSANDRA-7653, 8650, 7216, 8760, 8849, 8761)
 + * Avoid accessing partitioner through StorageProxy (CASSANDRA-8244, 8268)
 + * Upgrade Metrics library and remove depricated metrics (CASSANDRA-5657)
 + * Serializing Row cache alternative, fully off heap (CASSANDRA-7438)
 + * Duplicate rows returned when in clause has repeated values (CASSANDRA-6707)
 + * Make CassandraException unchecked, extend RuntimeException (CASSANDRA-8560)
 + * Support direct buffer decompression for reads (CASSANDRA-8464)
 + * DirectByteBuffer compatible LZ4 methods (CASSANDRA-7039)
 + * Group sstables for anticompaction correctly (CASSANDRA-8578)
 + * Add ReadFailureException to native protocol, respond
 +   immediately when replicas encounter errors while handling
 +   a read request (CASSANDRA-7886)
 + * Switch CommitLogSegment from RandomAccessFile to nio (CASSANDRA-8308)
 + * Allow mixing token and partition key restrictions (CASSANDRA-7016)
 + * Support index key/value entries on map collections (CASSANDRA-8473)
 + * Modernize schema tables (CASSANDRA-8261)
 + * Support for user-defined aggregation functions (CASSANDRA-8053)
 + * Fix NPE in SelectStatement with empty IN values (CASSANDRA-8419)
 + * Refactor SelectStatement, return IN results in natural order instead
 +   of IN value list order and ignore duplicate values in partition key IN restrictions (CASSANDRA-7981)
 + * Support UDTs, tuples, and collections in user-defined
 +   functions (CASSANDRA-7563)
 + * Fix aggregate fn results on empty selection, result column name,
 +   and cqlsh parsing (CASSANDRA-8229)
 + * Mark sstables as repaired after full repair (CASSANDRA-7586)
 + * Extend Descriptor to include a format value and refactor reader/writer
 +   APIs (CASSANDRA-7443)
 + * Integrate JMH for microbenchmarks (CASSANDRA-8151)
 + * Keep sstable levels when bootstrapping (CASSANDRA-7460)
 + * Add Sigar library and perform basic OS settings check on startup (CASSANDRA-7838)
 + * Support for aggregation functions (CASSANDRA-4914)
 + * Remove cassandra-cli (CASSANDRA-7920)
 + * Accept dollar quoted strings in CQL (CASSANDRA-7769)
 + * Make assassinate a first class command (CASSANDRA-7935)
 + * Support IN clause on any partition key column (CASSANDRA-7855)
 + * Support IN clause on any clustering column (CASSANDRA-4762)
 + * Improve compaction logging (CASSANDRA-7818)
 + * Remove YamlFileNetworkTopologySnitch (CASSANDRA-7917)
 + * Do anticompaction in groups (CASSANDRA-6851)
 + * Support user-defined functions (CASSANDRA-7395, 7526, 7562, 7740, 7781, 7929,
 +   7924, 7812, 8063, 7813, 7708)
 + * Permit configurable timestamps with cassandra-stress (CASSANDRA-7416)
 + * Move sstable RandomAccessReader to nio2, which allows using the
 +   FILE_SHARE_DELETE flag on Windows (CASSANDRA-4050)
 + * Remove CQL2 (CASSANDRA-5918)
 + * Add Thrift get_multi_slice call (CASSANDRA-6757)
 + * Optimize fetching multiple cells by name (CASSANDRA-6933)
 + * Allow compilation in java 8 (CASSANDRA-7028)
 + * Make incremental repair default (CASSANDRA-7250)
 + * Enable code coverage thru JaCoCo (CASSANDRA-7226)
 + * Switch external naming of 'column families' to 'tables' (CASSANDRA-4369) 
 + * Shorten SSTable path (CASSANDRA-6962)
 + * Use unsafe mutations for most unit tests (CASSANDRA-6969)
 + * Fix race condition during calculation of pending ranges (CASSANDRA-7390)
 + * Fail on very large batch sizes (CASSANDRA-8011)
 + * Improve concurrency of repair (CASSANDRA-6455, 8208)
 + * Select optimal CRC32 implementation at runtime (CASSANDRA-8614)
 + * Evaluate MurmurHash of Token once per query (CASSANDRA-7096)
 +
 +
  2.1.4
+  * Add SimpleDate (cql date) and Time (cql time) types (CASSANDRA-7523)
   * Use long for key count in cfstats (CASSANDRA-8913)
   * Make SSTableRewriter.abort() more robust to failure (CASSANDRA-8832)
   * Remove cold_reads_to_omit from STCS (CASSANDRA-8860)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3d11ec5d/NEWS.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3d11ec5d/bin/cqlsh
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3d11ec5d/doc/cql3/CQL.textile
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3d11ec5d/doc/native_protocol_v4.spec
----------------------------------------------------------------------
diff --cc doc/native_protocol_v4.spec
index 5032bb5,0000000..a3a3fd3
mode 100644,000000..100644
--- a/doc/native_protocol_v4.spec
+++ b/doc/native_protocol_v4.spec
@@@ -1,1042 -1,0 +1,1044 @@@
 +
 +                             CQL BINARY PROTOCOL v4
 +
 +
 +Table of Contents
 +
 +  1. Overview
 +  2. Frame header
 +    2.1. version
 +    2.2. flags
 +    2.3. stream
 +    2.4. opcode
 +    2.5. length
 +  3. Notations
 +  4. Messages
 +    4.1. Requests
 +      4.1.1. STARTUP
 +      4.1.2. AUTH_RESPONSE
 +      4.1.3. OPTIONS
 +      4.1.4. QUERY
 +      4.1.5. PREPARE
 +      4.1.6. EXECUTE
 +      4.1.7. BATCH
 +      4.1.8. REGISTER
 +    4.2. Responses
 +      4.2.1. ERROR
 +      4.2.2. READY
 +      4.2.3. AUTHENTICATE
 +      4.2.4. SUPPORTED
 +      4.2.5. RESULT
 +        4.2.5.1. Void
 +        4.2.5.2. Rows
 +        4.2.5.3. Set_keyspace
 +        4.2.5.4. Prepared
 +        4.2.5.5. Schema_change
 +      4.2.6. EVENT
 +      4.2.7. AUTH_CHALLENGE
 +      4.2.8. AUTH_SUCCESS
 +  5. Compression
 +  6. Data Type Serialization Formats
 +  7. User Defined Type Serialization
 +  8. Result paging
 +  9. Error codes
 +  10. Changes from v3
 +
 +
 +1. Overview
 +
 +  The CQL binary protocol is a frame based protocol. Frames are defined as:
 +
 +      0         8        16        24        32         40
 +      +---------+---------+---------+---------+---------+
 +      | version |  flags  |      stream       | opcode  |
 +      +---------+---------+---------+---------+---------+
 +      |                length                 |
 +      +---------+---------+---------+---------+
 +      |                                       |
 +      .            ...  body ...              .
 +      .                                       .
 +      .                                       .
 +      +----------------------------------------
 +
 +  The protocol is big-endian (network byte order).
 +
 +  Each frame contains a fixed size header (9 bytes) followed by a variable size
 +  body. The header is described in Section 2. The content of the body depends
 +  on the header opcode value (the body can in particular be empty for some
 +  opcode values). The list of allowed opcode is defined Section 2.3 and the
 +  details of each corresponding message is described Section 4.
 +
 +  The protocol distinguishes 2 types of frames: requests and responses. Requests
 +  are those frame sent by the clients to the server, response are the ones sent
 +  by the server. Note however that the protocol supports server pushes (events)
 +  so responses does not necessarily come right after a client request.
 +
 +  Note to client implementors: clients library should always assume that the
 +  body of a given frame may contain more data than what is described in this
 +  document. It will however always be safe to ignore the remaining of the frame
 +  body in such cases. The reason is that this may allow to sometimes extend the
 +  protocol with optional features without needing to change the protocol
 +  version.
 +
 +
 +
 +2. Frame header
 +
 +2.1. version
 +
 +  The version is a single byte that indicate both the direction of the message
 +  (request or response) and the version of the protocol in use. The up-most bit
 +  of version is used to define the direction of the message: 0 indicates a
 +  request, 1 indicates a responses. This can be useful for protocol analyzers to
 +  distinguish the nature of the packet from the direction which it is moving.
 +  The rest of that byte is the protocol version (4 for the protocol defined in
 +  this document). In other words, for this version of the protocol, version will
 +  have one of:
 +    0x04    Request frame for this protocol version
 +    0x84    Response frame for this protocol version
 +
 +  Please note that the while every message ship with the version, only one version
 +  of messages is accepted on a given connection. In other words, the first message
 +  exchanged (STARTUP) sets the version for the connection for the lifetime of this
 +  connection.
 +
 +  This document describe the version 3 of the protocol. For the changes made since
 +  version 3, see Section 10.
 +
 +
 +2.2. flags
 +
 +  Flags applying to this frame. The flags have the following meaning (described
 +  by the mask that allow to select them):
 +    0x01: Compression flag. If set, the frame body is compressed. The actual
 +          compression to use should have been set up beforehand through the
 +          Startup message (which thus cannot be compressed; Section 4.1.1).
 +    0x02: Tracing flag. For a request frame, this indicate the client requires
 +          tracing of the request. Note that not all requests support tracing.
 +          Currently, only QUERY, PREPARE and EXECUTE queries support tracing.
 +          Other requests will simply ignore the tracing flag if set. If a
 +          request support tracing and the tracing flag was set, the response to
 +          this request will have the tracing flag set and contain tracing
 +          information.
 +          If a response frame has the tracing flag set, its body contains
 +          a tracing ID. The tracing ID is a [uuid] and is the first thing in
 +          the frame body. The rest of the body will then be the usual body
 +          corresponding to the response opcode.
 +
 +  The rest of the flags is currently unused and ignored.
 +
 +2.3. stream
 +
 +  A frame has a stream id (a [short] value). When sending request messages, this
 +  stream id must be set by the client to a non-negative value (negative stream id
 +  are reserved for streams initiated by the server; currently all EVENT messages
 +  (section 4.2.6) have a streamId of -1). If a client sends a request message
 +  with the stream id X, it is guaranteed that the stream id of the response to
 +  that message will be X.
 +
 +  This allow to deal with the asynchronous nature of the protocol. If a client
 +  sends multiple messages simultaneously (without waiting for responses), there
 +  is no guarantee on the order of the responses. For instance, if the client
 +  writes REQ_1, REQ_2, REQ_3 on the wire (in that order), the server might
 +  respond to REQ_3 (or REQ_2) first. Assigning different stream id to these 3
 +  requests allows the client to distinguish to which request an received answer
 +  respond to. As there can only be 32768 different simultaneous streams, it is up
 +  to the client to reuse stream id.
 +
 +  Note that clients are free to use the protocol synchronously (i.e. wait for
 +  the response to REQ_N before sending REQ_N+1). In that case, the stream id
 +  can be safely set to 0. Clients should also feel free to use only a subset of
 +  the 32768 maximum possible stream ids if it is simpler for those
 +  implementation.
 +
 +2.4. opcode
 +
 +  An integer byte that distinguish the actual message:
 +    0x00    ERROR
 +    0x01    STARTUP
 +    0x02    READY
 +    0x03    AUTHENTICATE
 +    0x05    OPTIONS
 +    0x06    SUPPORTED
 +    0x07    QUERY
 +    0x08    RESULT
 +    0x09    PREPARE
 +    0x0A    EXECUTE
 +    0x0B    REGISTER
 +    0x0C    EVENT
 +    0x0D    BATCH
 +    0x0E    AUTH_CHALLENGE
 +    0x0F    AUTH_RESPONSE
 +    0x10    AUTH_SUCCESS
 +
 +  Messages are described in Section 4.
 +
 +  (Note that there is no 0x04 message in this version of the protocol)
 +
 +
 +2.5. length
 +
 +  A 4 byte integer representing the length of the body of the frame (note:
 +  currently a frame is limited to 256MB in length).
 +
 +
 +3. Notations
 +
 +  To describe the layout of the frame body for the messages in Section 4, we
 +  define the following:
 +
 +    [int]          A 4 bytes integer
 +    [long]         A 8 bytes integer
 +    [short]        A 2 bytes unsigned integer
 +    [string]       A [short] n, followed by n bytes representing an UTF-8
 +                   string.
 +    [long string]  An [int] n, followed by n bytes representing an UTF-8 string.
 +    [uuid]         A 16 bytes long uuid.
 +    [string list]  A [short] n, followed by n [string].
 +    [bytes]        A [int] n, followed by n bytes if n >= 0. If n < 0,
 +                   no byte should follow and the value represented is `null`.
 +    [short bytes]  A [short] n, followed by n bytes if n >= 0.
 +
 +    [option]       A pair of <id><value> where <id> is a [short] representing
 +                   the option id and <value> depends on that option (and can be
 +                   of size 0). The supported id (and the corresponding <value>)
 +                   will be described when this is used.
 +    [option list]  A [short] n, followed by n [option].
 +    [inet]         An address (ip and port) to a node. It consists of one
 +                   [byte] n, that represents the address size, followed by n
 +                   [byte] representing the IP address (in practice n can only be
 +                   either 4 (IPv4) or 16 (IPv6)), following by one [int]
 +                   representing the port.
 +    [consistency]  A consistency level specification. This is a [short]
 +                   representing a consistency level with the following
 +                   correspondance:
 +                     0x0000    ANY
 +                     0x0001    ONE
 +                     0x0002    TWO
 +                     0x0003    THREE
 +                     0x0004    QUORUM
 +                     0x0005    ALL
 +                     0x0006    LOCAL_QUORUM
 +                     0x0007    EACH_QUORUM
 +                     0x0008    SERIAL
 +                     0x0009    LOCAL_SERIAL
 +                     0x000A    LOCAL_ONE
 +
 +    [string map]      A [short] n, followed by n pair <k><v> where <k> and <v>
 +                      are [string].
 +    [string multimap] A [short] n, followed by n pair <k><v> where <k> is a
 +                      [string] and <v> is a [string list].
 +
 +
 +4. Messages
 +
 +4.1. Requests
 +
 +  Note that outside of their normal responses (described below), all requests
 +  can get an ERROR message (Section 4.2.1) as response.
 +
 +4.1.1. STARTUP
 +
 +  Initialize the connection. The server will respond by either a READY message
 +  (in which case the connection is ready for queries) or an AUTHENTICATE message
 +  (in which case credentials will need to be provided using AUTH_RESPONSE).
 +
 +  This must be the first message of the connection, except for OPTIONS that can
 +  be sent before to find out the options supported by the server. Once the
 +  connection has been initialized, a client should not send any more STARTUP
 +  message.
 +
 +  The body is a [string map] of options. Possible options are:
 +    - "CQL_VERSION": the version of CQL to use. This option is mandatory and
 +      currenty, the only version supported is "3.0.0". Note that this is
 +      different from the protocol version.
 +    - "COMPRESSION": the compression algorithm to use for frames (See section 5).
 +      This is optional, if not specified no compression will be used.
 +
 +
 +4.1.2. AUTH_RESPONSE
 +
 +  Answers a server authentication challenge.
 +
 +  Authentication in the protocol is SASL based. The server sends authentication
 +  challenges (a bytes token) to which the client answer with this message. Those
 +  exchanges continue until the server accepts the authentication by sending a
 +  AUTH_SUCCESS message after a client AUTH_RESPONSE. It is however that client that
 +  initiate the exchange by sending an initial AUTH_RESPONSE in response to a
 +  server AUTHENTICATE request.
 +
 +  The body of this message is a single [bytes] token. The details of what this
 +  token contains (and when it can be null/empty, if ever) depends on the actual
 +  authenticator used.
 +
 +  The response to a AUTH_RESPONSE is either a follow-up AUTH_CHALLENGE message,
 +  an AUTH_SUCCESS message or an ERROR message.
 +
 +
 +4.1.3. OPTIONS
 +
 +  Asks the server to return what STARTUP options are supported. The body of an
 +  OPTIONS message should be empty and the server will respond with a SUPPORTED
 +  message.
 +
 +
 +4.1.4. QUERY
 +
 +  Performs a CQL query. The body of the message must be:
 +    <query><query_parameters>
 +  where <query> is a [long string] representing the query and
 +  <query_parameters> must be
 +    <consistency><flags>[<n>[name_1]<value_1>...[name_n]<value_n>][<result_page_size>][<paging_state>][<serial_consistency>][<timestamp>]
 +  where:
 +    - <consistency> is the [consistency] level for the operation.
 +    - <flags> is a [byte] whose bits define the options for this query and
 +      in particular influence what the remainder of the message contains.
 +      A flag is set if the bit corresponding to its `mask` is set. Supported
 +      flags are, given there mask:
 +        0x01: Values. In that case, a [short] <n> followed by <n> [bytes]
 +              values are provided. Those value are used for bound variables in
 +              the query. Optionally, if the 0x40 flag is present, each value
 +              will be preceded by a [string] name, representing the name of
 +              the marker the value must be binded to. This is optional, and
 +              if not present, values will be binded by position.
 +        0x02: Skip_metadata. If present, the Result Set returned as a response
 +              to that query (if any) will have the NO_METADATA flag (see
 +              Section 4.2.5.2).
 +        0x04: Page_size. In that case, <result_page_size> is an [int]
 +              controlling the desired page size of the result (in CQL3 rows).
 +              See the section on paging (Section 8) for more details.
 +        0x08: With_paging_state. If present, <paging_state> should be present.
 +              <paging_state> is a [bytes] value that should have been returned
 +              in a result set (Section 4.2.5.2). If provided, the query will be
 +              executed but starting from a given paging state. This also to
 +              continue paging on a different node from the one it has been
 +              started (See Section 8 for more details).
 +        0x10: With serial consistency. If present, <serial_consistency> should be
 +              present. <serial_consistency> is the [consistency] level for the
 +              serial phase of conditional updates. That consitency can only be
 +              either SERIAL or LOCAL_SERIAL and if not present, it defaults to
 +              SERIAL. This option will be ignored for anything else that a
 +              conditional update/insert.
 +        0x20: With default timestamp. If present, <timestamp> should be present.
 +              <timestamp> is a [long] representing the default timestamp for the query
 +              in microseconds (negative values are forbidden). If provided, this will
 +              replace the server side assigned timestamp as default timestamp.
 +              Note that a timestamp in the query itself will still override
 +              this timestamp. This is entirely optional.
 +        0x40: With names for values. This only makes sense if the 0x01 flag is set and
 +              is ignored otherwise. If present, the values from the 0x01 flag will
 +              be preceded by a name (see above). Note that this is only useful for
 +              QUERY requests where named bind markers are used; for EXECUTE statements,
 +              since the names for the expected values was returned during preparation,
 +              a client can always provide values in the right order without any names
 +              and using this flag, while supported, is almost surely inefficient.
 +
 +  Note that the consistency is ignored by some queries (USE, CREATE, ALTER,
 +  TRUNCATE, ...).
 +
 +  The server will respond to a QUERY message with a RESULT message, the content
 +  of which depends on the query.
 +
 +
 +4.1.5. PREPARE
 +
 +  Prepare a query for later execution (through EXECUTE). The body consists of
 +  the CQL query to prepare as a [long string].
 +
 +  The server will respond with a RESULT message with a `prepared` kind (0x0004,
 +  see Section 4.2.5).
 +
 +
 +4.1.6. EXECUTE
 +
 +  Executes a prepared query. The body of the message must be:
 +    <id><query_parameters>
 +  where <id> is the prepared query ID. It's the [short bytes] returned as a
 +  response to a PREPARE message. As for <query_parameters>, it has the exact
 +  same definition than in QUERY (see Section 4.1.4).
 +
 +  The response from the server will be a RESULT message.
 +
 +
 +4.1.7. BATCH
 +
 +  Allows executing a list of queries (prepared or not) as a batch (note that
 +  only DML statements are accepted in a batch). The body of the message must
 +  be:
 +    <type><n><query_1>...<query_n><consistency><flags>[<serial_consistency>][<timestamp>]
 +  where:
 +    - <type> is a [byte] indicating the type of batch to use:
 +        - If <type> == 0, the batch will be "logged". This is equivalent to a
 +          normal CQL3 batch statement.
 +        - If <type> == 1, the batch will be "unlogged".
 +        - If <type> == 2, the batch will be a "counter" batch (and non-counter
 +          statements will be rejected).
 +    - <flags> is a [byte] whose bits define the options for this query and
 +      in particular influence the remainder of the message contains. It is similar
 +      to the <flags> from QUERY and EXECUTE methods, except that the 4 rightmost
 +      bits must always be 0 as their corresponding option do not make sense for
 +      Batch. A flag is set if the bit corresponding to its `mask` is set. Supported
 +      flags are, given there mask:
 +        0x10: With serial consistency. If present, <serial_consistency> should be
 +              present. <serial_consistency> is the [consistency] level for the
 +              serial phase of conditional updates. That consitency can only be
 +              either SERIAL or LOCAL_SERIAL and if not present, it defaults to
 +              SERIAL. This option will be ignored for anything else that a
 +              conditional update/insert.
 +        0x20: With default timestamp. If present, <timestamp> should be present.
 +              <timestamp> is a [long] representing the default timestamp for the query
 +              in microseconds. If provided, this will replace the server side assigned
 +              timestamp as default timestamp. Note that a timestamp in the query itself
 +              will still override this timestamp. This is entirely optional.
 +        0x40: With names for values. If set, then all values for all <query_i> must be
 +              preceded by a [string] <name_i> that have the same meaning as in QUERY
 +              requests.
 +    - <n> is a [short] indicating the number of following queries.
 +    - <query_1>...<query_n> are the queries to execute. A <query_i> must be of the
 +      form:
 +        <kind><string_or_id><n>[<name_1>]<value_1>...[<name_n>]<value_n>
 +      where:
 +       - <kind> is a [byte] indicating whether the following query is a prepared
 +         one or not. <kind> value must be either 0 or 1.
 +       - <string_or_id> depends on the value of <kind>. If <kind> == 0, it should be
 +         a [long string] query string (as in QUERY, the query string might contain
 +         bind markers). Otherwise (that is, if <kind> == 1), it should be a
 +         [short bytes] representing a prepared query ID.
 +       - <n> is a [short] indicating the number (possibly 0) of following values.
 +       - <name_i> is the optional name of the following <value_i>. It must be present
 +         if and only if the 0x40 flag is provided for the batch.
 +       - <value_i> is the [bytes] to use for bound variable i (of bound variable <name_i>
 +         if the 0x40 flag is used).
 +    - <consistency> is the [consistency] level for the operation.
 +    - <serial_consistency> is only present if the 0x10 flag is set. In that case,
 +      <serial_consistency> is the [consistency] level for the serial phase of
 +      conditional updates. That consitency can only be either SERIAL or
 +      LOCAL_SERIAL and if not present will defaults to SERIAL. This option will
 +      be ignored for anything else that a conditional update/insert.
 +
 +  The server will respond with a RESULT message.
 +
 +
 +4.1.8. REGISTER
 +
 +  Register this connection to receive some type of events. The body of the
 +  message is a [string list] representing the event types to register to. See
 +  section 4.2.6 for the list of valid event types.
 +
 +  The response to a REGISTER message will be a READY message.
 +
 +  Please note that if a client driver maintains multiple connections to a
 +  Cassandra node and/or connections to multiple nodes, it is advised to
 +  dedicate a handful of connections to receive events, but to *not* register
 +  for events on all connections, as this would only result in receiving
 +  multiple times the same event messages, wasting bandwidth.
 +
 +
 +4.2. Responses
 +
 +  This section describes the content of the frame body for the different
 +  responses. Please note that to make room for future evolution, clients should
 +  support extra informations (that they should simply discard) to the one
 +  described in this document at the end of the frame body.
 +
 +4.2.1. ERROR
 +
 +  Indicates an error processing a request. The body of the message will be an
 +  error code ([int]) followed by a [string] error message. Then, depending on
 +  the exception, more content may follow. The error codes are defined in
 +  Section 9, along with their additional content if any.
 +
 +
 +4.2.2. READY
 +
 +  Indicates that the server is ready to process queries. This message will be
 +  sent by the server either after a STARTUP message if no authentication is
 +  required, or after a successful CREDENTIALS message.
 +
 +  The body of a READY message is empty.
 +
 +
 +4.2.3. AUTHENTICATE
 +
 +  Indicates that the server require authentication, and which authentication
 +  mechanism to use.
 +
 +  The authentication is SASL based and thus consists on a number of server
 +  challenges (AUTH_CHALLENGE, Section 4.2.7) followed by client responses
 +  (AUTH_RESPONSE, Section 4.1.2). The Initial exchange is however boostrapped
 +  by an initial client response. The details of that exchange (including how
 +  much challenge-response pair are required) are specific to the authenticator
 +  in use. The exchange ends when the server sends an AUTH_SUCCESS message or
 +  an ERROR message.
 +
 +  This message will be sent following a STARTUP message if authentication is
 +  required and must be answered by a AUTH_RESPONSE message from the client.
 +
 +  The body consists of a single [string] indicating the full class name of the
 +  IAuthenticator in use.
 +
 +
 +4.2.4. SUPPORTED
 +
 +  Indicates which startup options are supported by the server. This message
 +  comes as a response to an OPTIONS message.
 +
 +  The body of a SUPPORTED message is a [string multimap]. This multimap gives
 +  for each of the supported STARTUP options, the list of supported values.
 +
 +
 +4.2.5. RESULT
 +
 +  The result to a query (QUERY, PREPARE, EXECUTE or BATCH messages).
 +
 +  The first element of the body of a RESULT message is an [int] representing the
 +  `kind` of result. The rest of the body depends on the kind. The kind can be
 +  one of:
 +    0x0001    Void: for results carrying no information.
 +    0x0002    Rows: for results to select queries, returning a set of rows.
 +    0x0003    Set_keyspace: the result to a `use` query.
 +    0x0004    Prepared: result to a PREPARE message.
 +    0x0005    Schema_change: the result to a schema altering query.
 +
 +  The body for each kind (after the [int] kind) is defined below.
 +
 +
 +4.2.5.1. Void
 +
 +  The rest of the body for a Void result is empty. It indicates that a query was
 +  successful without providing more information.
 +
 +
 +4.2.5.2. Rows
 +
 +  Indicates a set of rows. The rest of body of a Rows result is:
 +    <metadata><rows_count><rows_content>
 +  where:
 +    - <metadata> is composed of:
 +        <flags><columns_count>[<paging_state>][<global_table_spec>?<col_spec_1>...<col_spec_n>]
 +      where:
 +        - <flags> is an [int]. The bits of <flags> provides information on the
 +          formatting of the remaining informations. A flag is set if the bit
 +          corresponding to its `mask` is set. Supported flags are, given there
 +          mask:
 +            0x0001    Global_tables_spec: if set, only one table spec (keyspace
 +                      and table name) is provided as <global_table_spec>. If not
 +                      set, <global_table_spec> is not present.
 +            0x0002    Has_more_pages: indicates whether this is not the last
 +                      page of results and more should be retrieve. If set, the
 +                      <paging_state> will be present. The <paging_state> is a
 +                      [bytes] value that should be used in QUERY/EXECUTE to
 +                      continue paging and retrieve the remained of the result for
 +                      this query (See Section 8 for more details).
 +            0x0004    No_metadata: if set, the <metadata> is only composed of
 +                      these <flags>, the <column_count> and optionally the
 +                      <paging_state> (depending on the Has_more_pages flage) but
 +                      no other information (so no <global_table_spec> nor <col_spec_i>).
 +                      This will only ever be the case if this was requested
 +                      during the query (see QUERY and RESULT messages).
 +        - <columns_count> is an [int] representing the number of columns selected
 +          by the query this result is of. It defines the number of <col_spec_i>
 +          elements in and the number of element for each row in <rows_content>.
 +        - <global_table_spec> is present if the Global_tables_spec is set in
 +          <flags>. If present, it is composed of two [string] representing the
 +          (unique) keyspace name and table name the columns return are of.
 +        - <col_spec_i> specifies the columns returned in the query. There is
 +          <column_count> such column specifications that are composed of:
 +            (<ksname><tablename>)?<name><type>
 +          The initial <ksname> and <tablename> are two [string] are only present
 +          if the Global_tables_spec flag is not set. The <column_name> is a
 +          [string] and <type> is an [option] that correspond to the description
 +          (what this description is depends a bit on the context: in results to
 +          selects, this will be either the user chosen alias or the selection used
 +          (often a colum name, but it can be a function call too). In results to
 +          a PREPARE, this will be either the name of the bind variable corresponding
 +          or the column name for the variable if it is "anonymous") and type of
 +          the corresponding result. The option for <type> is either a native
 +          type (see below), in which case the option has no value, or a
 +          'custom' type, in which case the value is a [string] representing
 +          the full qualified class name of the type represented. Valid option
 +          ids are:
 +            0x0000    Custom: the value is a [string], see above.
 +            0x0001    Ascii
 +            0x0002    Bigint
 +            0x0003    Blob
 +            0x0004    Boolean
 +            0x0005    Counter
 +            0x0006    Decimal
 +            0x0007    Double
 +            0x0008    Float
 +            0x0009    Int
 +            0x000B    Timestamp
 +            0x000C    Uuid
 +            0x000D    Varchar
 +            0x000E    Varint
 +            0x000F    Timeuuid
 +            0x0010    Inet
++            0x0011    Date
++            0x0012    Time
 +            0x0020    List: the value is an [option], representing the type
 +                            of the elements of the list.
 +            0x0021    Map: the value is two [option], representing the types of the
 +                           keys and values of the map
 +            0x0022    Set: the value is an [option], representing the type
 +                            of the elements of the set
 +            0x0030    UDT: the value is <ks><udt_name><n><name_1><type_1>...<name_n><type_n>
 +                           where:
 +                              - <ks> is a [string] representing the keyspace name this
 +                                UDT is part of.
 +                              - <udt_name> is a [string] representing the UDT name.
 +                              - <n> is a [short] reprensenting the number of fields of
 +                                the UDT, and thus the number of <name_i><type_i> pair
 +                                following
 +                              - <name_i> is a [string] representing the name of the
 +                                i_th field of the UDT.
 +                              - <type_i> is an [option] representing the type of the
 +                                i_th field of the UDT.
 +            0x0031    Tuple: the value is <n><type_1>...<type_n> where <n> is a [short]
 +                             representing the number of value in the type, and <type_i>
 +                             are [option] representing the type of the i_th component
 +                             of the tuple
 +
 +    - <rows_count> is an [int] representing the number of rows present in this
 +      result. Those rows are serialized in the <rows_content> part.
 +    - <rows_content> is composed of <row_1>...<row_m> where m is <rows_count>.
 +      Each <row_i> is composed of <value_1>...<value_n> where n is
 +      <columns_count> and where <value_j> is a [bytes] representing the value
 +      returned for the jth column of the ith row. In other words, <rows_content>
 +      is composed of (<rows_count> * <columns_count>) [bytes].
 +
 +
 +4.2.5.3. Set_keyspace
 +
 +  The result to a `use` query. The body (after the kind [int]) is a single
 +  [string] indicating the name of the keyspace that has been set.
 +
 +
 +4.2.5.4. Prepared
 +
 +  The result to a PREPARE message. The rest of the body of a Prepared result is:
 +    <id><metadata><result_metadata>
 +  where:
 +    - <id> is [short bytes] representing the prepared query ID.
 +    - <metadata> is defined exactly as for a Rows RESULT (See section 4.2.5.2; you
 +      can however assume that the Has_more_pages flag is always off) and
 +      is the specification for the variable bound in this prepare statement.
 +    - <result_metadata> is defined exactly as <metadata> but correspond to the
 +      metadata for the resultSet that execute this query will yield. Note that
 +      <result_metadata> may be empty (have the No_metadata flag and 0 columns, See
 +      section 4.2.5.2) and will be for any query that is not a Select. There is
 +      in fact never a guarantee that this will non-empty so client should protect
 +      themselves accordingly. The presence of this information is an
 +      optimization that allows to later execute the statement that has been
 +      prepared without requesting the metadata (Skip_metadata flag in EXECUTE).
 +      Clients can safely discard this metadata if they do not want to take
 +      advantage of that optimization.
 +
 +  Note that prepared query ID return is global to the node on which the query
 +  has been prepared. It can be used on any connection to that node and this
 +  until the node is restarted (after which the query must be reprepared).
 +
 +4.2.5.5. Schema_change
 +
 +  The result to a schema altering query (creation/update/drop of a
 +  keyspace/table/index). The body (after the kind [int]) is the same
 +  as the body for a "SCHEMA_CHANGE" event, so 3 strings:
 +    <change_type><target><options>
 +  Please refer to the section 4.2.6 below for the meaning of those fields.
 +
 +  Note that queries to create and drop an index are considered as change
 +  updating the table the index is on.
 +
 +
 +4.2.6. EVENT
 +
 +  And event pushed by the server. A client will only receive events for the
 +  type it has REGISTER to. The body of an EVENT message will start by a
 +  [string] representing the event type. The rest of the message depends on the
 +  event type. The valid event types are:
 +    - "TOPOLOGY_CHANGE": events related to change in the cluster topology.
 +      Currently, events are sent when new nodes are added to the cluster, and
 +      when nodes are removed. The body of the message (after the event type)
 +      consists of a [string] and an [inet], corresponding respectively to the
 +      type of change ("NEW_NODE" or "REMOVED_NODE") followed by the address of
 +      the new/removed node.
 +    - "STATUS_CHANGE": events related to change of node status. Currently,
 +      up/down events are sent. The body of the message (after the event type)
 +      consists of a [string] and an [inet], corresponding respectively to the
 +      type of status change ("UP" or "DOWN") followed by the address of the
 +      concerned node.
 +    - "SCHEMA_CHANGE": events related to schema change. After the event type,
 +      the rest of the message will be <change_type><target><options> where:
 +        - <change_type> is a [string] representing the type of changed involved.
 +          It will be one of "CREATED", "UPDATED" or "DROPPED".
 +        - <target> is a [string] that can be one of "KEYSPACE", "TABLE", "TYPE",
 +          "FUNCTION" or "AGGREGATE" and describes what has been modified
 +          ("TYPE" stands for modifications related to user types, "FUNCTION"
 +          for modifications related to user defined functions, "AGGREGATE"
 +          for modifications related to user defined aggregates).
 +        - <options> depends on the preceding <target>:
 +          - If <target> is "KEYSPACE", then <options> will be a single [string]
 +            representing the keyspace changed.
 +          - If <target> is "TABLE" or "TYPE", then
 +            <options> will be 2 [string]: the first one will be the keyspace
 +            containing the affected object, and the second one will be the name
 +            of said affected object (either the table, user type, function, or
 +            aggregate name).
 +          - If <target> is "FUNCTION" or "AGGREGATE", multiple arguments follow:
 +            - [string] keyspace containing the user defined function / aggregate
 +            - [string] the function/aggregate name
 +            - [string list] one string for each argument type (as CQL type)
 +
 +  All EVENT messages have a streamId of -1 (Section 2.3).
 +
 +  Please note that "NEW_NODE" and "UP" events are sent based on internal Gossip
 +  communication and as such may be sent a short delay before the binary
 +  protocol server on the newly up node is fully started. Clients are thus
 +  advise to wait a short time before trying to connect to the node (1 seconds
 +  should be enough), otherwise they may experience a connection refusal at
 +  first.
 +
 +4.2.7. AUTH_CHALLENGE
 +
 +  A server authentication challenge (see AUTH_RESPONSE (Section 4.1.2) for more
 +  details).
 +
 +  The body of this message is a single [bytes] token. The details of what this
 +  token contains (and when it can be null/empty, if ever) depends on the actual
 +  authenticator used.
 +
 +  Clients are expected to answer the server challenge by an AUTH_RESPONSE
 +  message.
 +
 +4.2.7. AUTH_SUCCESS
 +
 +  Indicate the success of the authentication phase. See Section 4.2.3 for more
 +  details.
 +
 +  The body of this message is a single [bytes] token holding final information
 +  from the server that the client may require to finish the authentication
 +  process. What that token contains and whether it can be null depends on the
 +  actual authenticator used.
 +
 +
 +5. Compression
 +
 +  Frame compression is supported by the protocol, but then only the frame body
 +  is compressed (the frame header should never be compressed).
 +
 +  Before being used, client and server must agree on a compression algorithm to
 +  use, which is done in the STARTUP message. As a consequence, a STARTUP message
 +  must never be compressed.  However, once the STARTUP frame has been received
 +  by the server can be compressed (including the response to the STARTUP
 +  request). Frame do not have to be compressed however, even if compression has
 +  been agreed upon (a server may only compress frame above a certain size at its
 +  discretion). A frame body should be compressed if and only if the compressed
 +  flag (see Section 2.2) is set.
 +
 +  As of this version 2 of the protocol, the following compressions are available:
 +    - lz4 (https://code.google.com/p/lz4/). In that, note that the 4 first bytes
 +      of the body will be the uncompressed length (followed by the compressed
 +      bytes).
 +    - snappy (https://code.google.com/p/snappy/). This compression might not be
 +      available as it depends on a native lib (server-side) that might not be
 +      avaivable on some installation.
 +
 +
 +6. Data Type Serialization Formats
 +
 +  This sections describes the serialization formats for all CQL data types
 +  supported by Cassandra through the native protocol.  These serialization
 +  formats should be used by client drivers to encode values for EXECUTE
 +  messages.  Cassandra will use these formats when returning values in
 +  RESULT messages.
 +
 +  All values are represented as [bytes] in EXECUTE and RESULT messages.
 +  The [bytes] format includes an int prefix denoting the length of the value.
 +  For that reason, the serialization formats described here will not include
 +  a length component.
 +
 +  For legacy compatibility reasons, note that most non-string types support
 +  "empty" values (i.e. a value with zero length).  An empty value is distinct
 +  from NULL, which is encoded with a negative length.
 +
 +  As with the rest of the native protocol, all encodings are big-endian.
 +
 +6.1. ascii
 +
 +  A sequence of bytes in the ASCII range [0, 127].  Bytes with values outside of
 +  this range will result in a validation error.
 +
 +6.2 bigint
 +
 +  An eight-byte two's complement integer.
 +
 +6.3 blob
 +
 +  Any sequence of bytes.
 +
 +6.4 boolean
 +
 +  A single byte.  A value of 0 denotes "false"; any other value denotes "true".
 +  (However, it is recommended that a value of 1 be used to represent "true".)
 +
 +6.5 decimal
 +
 +  The decimal format represents an arbitrary-precision number.  It contains an
 +  [int] "scale" component followed by a varint encoding (see section 6.17)
 +  of the unscaled value.  The encoded value represents "<unscaled>E<-scale>".
 +  In other words, "<unscaled> * 10 ^ (-1 * <scale>)".
 +
 +6.6 double
 +
 +  An eight-byte floating point number in the IEEE 754 binary64 format.
 +
 +6.7 float
 +
 +  An four-byte floating point number in the IEEE 754 binary32 format.
 +
 +6.8 inet
 +
 +  A 4 byte or 16 byte sequence denoting an IPv4 or IPv6 address, respectively.
 +
 +6.9 int
 +
 +  A four-byte two's complement integer.
 +
 +6.10 list
 +
 +  A [int] n indicating the number of elements in the list, followed by n
 +  elements.  Each element is [bytes] representing the serialized value.
 +
 +6.11 map
 +
 +  A [int] n indicating the number of key/value pairs in the map, followed by
 +  n entries.  Each entry is composed of two [bytes] representing the key
 +  and value.
 +
 +6.12 set
 +
 +  A [int] n indicating the number of elements in the set, followed by n
 +  elements.  Each element is [bytes] representing the serialized value.
 +
 +6.13 text
 +
 +  A sequence of bytes conforming to the UTF-8 specifications.
 +
 +6.14 timestamp
 +
 +  An eight-byte two's complement integer representing a millisecond-precision
 +  offset from the unix epoch (00:00:00, January 1st, 1970).  Negative values
 +  represent a negative offset from the epoch.
 +
 +6.15 uuid
 +
 +  A 16 byte sequence representing any valid UUID as defined by RFC 4122.
 +
 +6.16 varchar
 +
 +  An alias of the "text" type.
 +
 +6.17 varint
 +
 +  A variable-length two's complement encoding of a signed integer.
 +
 +  The following examples may help implementors of this spec:
 +
 +  Value | Encoding
 +  ------|---------
 +      0 |     0x00
 +      1 |     0x01
 +    127 |     0x7F
 +    128 |   0x0080
 +    129 |   0x0081
 +     -1 |     0xFF
 +   -128 |     0x80
 +   -129 |   0xFF7F
 +
 +  Note that positive numbers must use a most-significant byte with a value
 +  less than 0x80, because a most-significant bit of 1 indicates a negative
 +  value.  Implementors should pad positive values that have a MSB >= 0x80
 +  with a leading 0x00 byte.
 +
 +6.18 timeuuid
 +
 +  A 16 byte sequence representing a version 1 UUID as defined by RFC 4122.
 +
 +6.19 tuple
 +
 +  A sequence of [bytes] values representing the items in a tuple.  The encoding
 +  of each element depends on the data type for that position in the tuple.
 +  Null values may be represented by using length -1 for the [bytes]
 +  representation of an element.
 +
 +  Within a tuple, all data types should use the v3 protocol serialization format.
 +
 +
 +7. User Defined Types
 +
 +  This section describes the serialization format for User defined types (UDT),
 +  as described in section 4.2.5.2.
 +
 +  A UDT value is composed of successive [bytes] values, one for each field of the UDT
 +  value (in the order defined by the type). A UDT value will generally have one value
 +  for each field of the type it represents, but it is allowed to have less values than
 +  the type has fields.
 +
 +  Within a user-defined type value, all data types should use the v3 protocol
 +  serialization format.
 +
 +
 +8. Result paging
 +
 +  The protocol allows for paging the result of queries. For that, the QUERY and
 +  EXECUTE messages have a <result_page_size> value that indicate the desired
 +  page size in CQL3 rows.
 +
 +  If a positive value is provided for <result_page_size>, the result set of the
 +  RESULT message returned for the query will contain at most the
 +  <result_page_size> first rows of the query result. If that first page of result
 +  contains the full result set for the query, the RESULT message (of kind `Rows`)
 +  will have the Has_more_pages flag *not* set. However, if some results are not
 +  part of the first response, the Has_more_pages flag will be set and the result
 +  will contain a <paging_state> value. In that case, the <paging_state> value
 +  should be used in a QUERY or EXECUTE message (that has the *same* query than
 +  the original one or the behavior is undefined) to retrieve the next page of
 +  results.
 +
 +  Only CQL3 queries that return a result set (RESULT message with a Rows `kind`)
 +  support paging. For other type of queries, the <result_page_size> value is
 +  ignored.
 +
 +  Note to client implementors:
 +  - While <result_page_size> can be as low as 1, it will likely be detrimental
 +    to performance to pick a value too low. A value below 100 is probably too
 +    low for most use cases.
 +  - Clients should not rely on the actual size of the result set returned to
 +    decide if there is more result to fetch or not. Instead, they should always
 +    check the Has_more_pages flag (unless they did not enabled paging for the query
 +    obviously). Clients should also not assert that no result will have more than
 +    <result_page_size> results. While the current implementation always respect
 +    the exact value of <result_page_size>, we reserve ourselves the right to return
 +    slightly smaller or bigger pages in the future for performance reasons.
 +
 +
 +9. Error codes
 +
 +  The supported error codes are described below:
 +    0x0000    Server error: something unexpected happened. This indicates a
 +              server-side bug.
 +    0x000A    Protocol error: some client message triggered a protocol
 +              violation (for instance a QUERY message is sent before a STARTUP
 +              one has been sent)
 +    0x0100    Bad credentials: CREDENTIALS request failed because Cassandra
 +              did not accept the provided credentials.
 +
 +    0x1000    Unavailable exception. The rest of the ERROR message body will be
 +                <cl><required><alive>
 +              where:
 +                <cl> is the [consistency] level of the query having triggered
 +                     the exception.
 +                <required> is an [int] representing the number of node that
 +                           should be alive to respect <cl>
 +                <alive> is an [int] representing the number of replica that
 +                        were known to be alive when the request has been
 +                        processed (since an unavailable exception has been
 +                        triggered, there will be <alive> < <required>)
 +    0x1001    Overloaded: the request cannot be processed because the
 +              coordinator node is overloaded
 +    0x1002    Is_bootstrapping: the request was a read request but the
 +              coordinator node is bootstrapping
 +    0x1003    Truncate_error: error during a truncation error.
 +    0x1100    Write_timeout: Timeout exception during a write request. The rest
 +              of the ERROR message body will be
 +                <cl><received><blockfor><writeType>
 +              where:
 +                <cl> is the [consistency] level of the query having triggered
 +                     the exception.
 +                <received> is an [int] representing the number of nodes having
 +                           acknowledged the request.
 +                <blockfor> is the number of replica whose acknowledgement is
 +                           required to achieve <cl>.
 +                <writeType> is a [string] that describe the type of the write
 +                            that timeouted. The value of that string can be one
 +                            of:
 +                             - "SIMPLE": the write was a non-batched
 +                               non-counter write.
 +                             - "BATCH": the write was a (logged) batch write.
 +                               If this type is received, it means the batch log
 +                               has been successfully written (otherwise a
 +                               "BATCH_LOG" type would have been send instead).
 +                             - "UNLOGGED_BATCH": the write was an unlogged
 +                               batch. Not batch log write has been attempted.
 +                             - "COUNTER": the write was a counter write
 +                               (batched or not).
 +                             - "BATCH_LOG": the timeout occured during the
 +                               write to the batch log when a (logged) batch
 +                               write was requested.
 +    0x1200    Read_timeout: Timeout exception during a read request. The rest
 +              of the ERROR message body will be
 +                <cl><received><blockfor><data_present>
 +              where:
 +                <cl> is the [consistency] level of the query having triggered
 +                     the exception.
 +                <received> is an [int] representing the number of nodes having
 +                           answered the request.
 +                <blockfor> is the number of replica whose response is
 +                           required to achieve <cl>. Please note that it is
 +                           possible to have <received> >= <blockfor> if
 +                           <data_present> is false. And also in the (unlikely)
 +                           case were <cl> is achieved but the coordinator node
 +                           timeout while waiting for read-repair
 +                           acknowledgement.
 +                <data_present> is a single byte. If its value is 0, it means
 +                               the replica that was asked for data has not
 +                               responded. Otherwise, the value is != 0.
 +    0x1300    Read_failure: A non-timeout exception during a read request. The rest
 +              of the ERROR message body will be
 +                <cl><received><blockfor><numfailures><data_present>
 +              where:
 +                <cl> is the [consistency] level of the query having triggered
 +                     the exception.
 +                <received> is an [int] representing the number of nodes having
 +                           answered the request.
 +                <blockfor> is the number of replicas whose response is
 +                           required to achieve <cl>.
 +                <numfailures> is an [int] representing the number of nodes that
 +                              experience a failure while executing the request.
 +                <data_present> is a single byte. If its value is 0, it means
 +                               the replica that was asked for data had not
 +                               responded. Otherwise, the value is != 0.
 +    0x1400    Function_failure: A (user defined) function failed during execution.
 +              The rest of the ERROR message body will be
 +                <keyspace><function><arg_types>
 +              where:
 +                <keyspace> is the keyspace [string] of the failed function
 +                <function> is the name [string] of the failed function
 +                <arg_types> [string list] one string for each argument type (as CQL type) of the failed function
 +
 +    0x2000    Syntax_error: The submitted query has a syntax error.
 +    0x2100    Unauthorized: The logged user doesn't have the right to perform
 +              the query.
 +    0x2200    Invalid: The query is syntactically correct but invalid.
 +    0x2300    Config_error: The query is invalid because of some configuration issue
 +    0x2400    Already_exists: The query attempted to create a keyspace or a
 +              table that was already existing. The rest of the ERROR message
 +              body will be <ks><table> where:
 +                <ks> is a [string] representing either the keyspace that
 +                     already exists, or the keyspace in which the table that
 +                     already exists is.
 +                <table> is a [string] representing the name of the table that
 +                        already exists. If the query was attempting to create a
 +                        keyspace, <table> will be present but will be the empty
 +                        string.
 +    0x2500    Unprepared: Can be thrown while a prepared statement tries to be
 +              executed if the provide prepared statement ID is not known by
 +              this host. The rest of the ERROR message body will be [short
 +              bytes] representing the unknown ID.
 +
 +10. Changes from v3
 +
 +  * The format of "SCHEMA_CHANGE" events (Section 4.2.6) (and implicitly "Schema_change" results (Section 4.2.5.5))
 +    has been modified, and now includes changes related to user defined functions and user defined aggregates.
 +  * Read_failure error code was added.
 +  * Function_failure error code was added.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3d11ec5d/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --cc pylib/cqlshlib/cql3handling.py
index fc53967,0da6a7c..0eb7a3b
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@@ -39,10 -39,10 +39,10 @@@ NONALTERBALE_KEYSPACES = ('system'
  class Cql3ParsingRuleSet(CqlParsingRuleSet):
      keywords = set((
          'select', 'from', 'where', 'and', 'key', 'insert', 'update', 'with',
 -        'limit', 'using', 'use', 'count', 'set',
 +        'limit', 'using', 'use', 'set',
          'begin', 'apply', 'batch', 'truncate', 'delete', 'in', 'create',
 -        'keyspace', 'schema', 'columnfamily', 'table', 'index', 'on', 'drop',
 +        'function', 'aggregate', 'keyspace', 'schema', 'columnfamily', 'table', 'index', 'on', 'drop',
-         'primary', 'into', 'values', 'timestamp', 'ttl', 'alter', 'add', 'type',
+         'primary', 'into', 'values', 'date', 'time', 'timestamp', 'ttl', 'alter', 'add', 'type',
          'compact', 'storage', 'order', 'by', 'asc', 'desc', 'clustering',
          'token', 'writetime', 'map', 'list', 'to', 'custom', 'if', 'not'
      ))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3d11ec5d/pylib/cqlshlib/helptopics.py
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3d11ec5d/src/java/org/apache/cassandra/cql3/CQL3Type.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3d11ec5d/src/java/org/apache/cassandra/cql3/Constants.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Constants.java
index 1a257d1,2d5fdaa..69693bf
--- a/src/java/org/apache/cassandra/cql3/Constants.java
+++ b/src/java/org/apache/cassandra/cql3/Constants.java
@@@ -181,10 -179,12 +181,12 @@@ public abstract class Constant
                          case TEXT:
                          case INET:
                          case VARCHAR:
+                         case DATE:
+                         case TIME:
                          case TIMESTAMP:
 -                            return true;
 +                            return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
                      }
 -                    return false;
 +                    break;
                  case INTEGER:
                      switch (nt)
                      {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3d11ec5d/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------


[2/3] cassandra git commit: Add date and time types

Posted by jm...@apache.org.
Add date and time types

Patch by jmckenzie; reviewed by thobbs and cyeksigian for CASSANDRA-7523


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

Branch: refs/heads/trunk
Commit: 107545b3929c32c61e7d00c6ca448c10046db792
Parents: 2e3a287
Author: Joshua McKenzie <jm...@apache.org>
Authored: Mon Mar 9 12:20:28 2015 -0500
Committer: Joshua McKenzie <jm...@apache.org>
Committed: Mon Mar 9 12:20:28 2015 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |   5 +
 bin/cqlsh                                       |  47 +++-
 doc/cql3/CQL.textile                            |  41 +++-
 lib/joda-time-2.4.jar                           | Bin 0 -> 586201 bytes
 lib/licenses/joda-time-2.4.txt                  | 201 +++++++++++++++
 pylib/cqlshlib/cql3handling.py                  |   6 +-
 pylib/cqlshlib/displaying.py                    |   2 +
 pylib/cqlshlib/formatting.py                    |  65 +++--
 pylib/cqlshlib/helptopics.py                    |  23 ++
 pylib/cqlshlib/test/test_cqlsh_output.py        |  17 +-
 pylib/cqlshlib/test/test_keyspace_init.cql      |  41 ++--
 .../org/apache/cassandra/cql3/CQL3Type.java     |  34 +--
 .../org/apache/cassandra/cql3/Constants.java    |   5 +-
 src/java/org/apache/cassandra/cql3/Cql.g        |   4 +
 .../cassandra/db/marshal/IntegerType.java       |   5 +
 .../cassandra/db/marshal/SimpleDateType.java    |  72 ++++++
 .../apache/cassandra/db/marshal/TimeType.java   |  72 ++++++
 .../serializers/SimpleDateSerializer.java       | 117 +++++++++
 .../cassandra/serializers/TimeSerializer.java   | 198 +++++++++++++++
 .../cassandra/serializers/TypeSerializer.java   |   1 +
 .../apache/cassandra/transport/DataType.java    |   2 +
 .../db/marshal/SimpleDateTypeTest.java          | 153 ++++++++++++
 .../cassandra/db/marshal/TimeTypeTest.java      |  61 +++++
 .../serializers/SimpleDateSerializerTest.java   | 155 ++++++++++++
 .../serializers/TimeSerializerTest.java         | 242 +++++++++++++++++++
 26 files changed, 1483 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 5acc288..31828fc 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.4
+ * Add SimpleDate (cql date) and Time (cql time) types (CASSANDRA-7523)
  * Use long for key count in cfstats (CASSANDRA-8913)
  * Make SSTableRewriter.abort() more robust to failure (CASSANDRA-8832)
  * Remove cold_reads_to_omit from STCS (CASSANDRA-8860)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 06013b8..4ead1ea 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -21,6 +21,11 @@ Upgrading
       removed - it is almost always better to use date tiered compaction for
       workloads that have cold data. 
 
+New features
+------------
+    - New `SimpleDateType` (cql date). 4-byte unsigned integer w/out timestamp
+    - New `TimeType` (cql time). 8-byte long nano-second resolution timestamp
+
 2.1.3
 =====
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/bin/cqlsh
----------------------------------------------------------------------
diff --git a/bin/cqlsh b/bin/cqlsh
index 31dc080..3ec9457 100755
--- a/bin/cqlsh
+++ b/bin/cqlsh
@@ -122,6 +122,10 @@ from cqlshlib.displaying import (RED, BLUE, CYAN, ANSI_RESET, COLUMN_NAME_COLORS
                                  FormattedValue, colorme)
 from cqlshlib.formatting import format_by_type, formatter_for, format_value_utype
 from cqlshlib.util import trim_if_present, get_file_encoding_bomsize
+from cqlshlib.formatting import DateTimeFormat
+from cqlshlib.formatting import DEFAULT_TIMESTAMP_FORMAT
+from cqlshlib.formatting import DEFAULT_DATE_FORMAT
+from cqlshlib.formatting import DEFAULT_NANOTIME_FORMAT
 from cqlshlib.tracing import print_trace_session, print_trace
 
 DEFAULT_HOST = '127.0.0.1'
@@ -129,7 +133,6 @@ DEFAULT_PORT = 9042
 DEFAULT_CQLVER = '3.2.0'
 DEFAULT_PROTOCOL_VERSION = 3
 
-DEFAULT_TIME_FORMAT = '%Y-%m-%d %H:%M:%S%z'
 DEFAULT_FLOAT_PRECISION = 5
 DEFAULT_MAX_TRACE_WAIT = 10
 
@@ -431,7 +434,7 @@ def full_cql_version(ver):
     vertuple = tuple(map(int, ver_parts[0].split('.')) + [ver_parts[1]])
     return ver, vertuple
 
-def format_value(val, output_encoding, addcolor=False, time_format=None,
+def format_value(val, output_encoding, addcolor=False, date_time_format=None,
                  float_precision=None, colormap=None, nullval=None):
     if isinstance(val, DecodeError):
         if addcolor:
@@ -439,7 +442,7 @@ def format_value(val, output_encoding, addcolor=False, time_format=None,
         else:
             return FormattedValue(repr(val.thebytes))
     return format_by_type(type(val), val, output_encoding, colormap=colormap,
-                          addcolor=addcolor, nullval=nullval, time_format=time_format,
+                          addcolor=addcolor, nullval=nullval, date_time_format=date_time_format,
                           float_precision=float_precision)
 
 def show_warning_without_quoting_line(message, category, filename, lineno, file=None, line=None):
@@ -529,7 +532,9 @@ class Shell(cmd.Cmd):
                  completekey=DEFAULT_COMPLETEKEY, use_conn=None,
                  cqlver=DEFAULT_CQLVER, keyspace=None,
                  tracing_enabled=False, expand_enabled=False,
-                 display_time_format=DEFAULT_TIME_FORMAT,
+                 display_nanotime_format=DEFAULT_NANOTIME_FORMAT,
+                 display_timestamp_format=DEFAULT_TIMESTAMP_FORMAT,
+                 display_date_format=DEFAULT_DATE_FORMAT,
                  display_float_precision=DEFAULT_FLOAT_PRECISION,
                  max_trace_wait=DEFAULT_MAX_TRACE_WAIT,
                  ssl=False,
@@ -564,7 +569,11 @@ class Shell(cmd.Cmd):
             self.session = self.conn.connect()
 
         self.color = color
-        self.display_time_format = display_time_format
+
+        self.display_nanotime_format = display_nanotime_format
+        self.display_timestamp_format = display_timestamp_format
+        self.display_date_format = display_date_format
+
         self.display_float_precision = display_float_precision
 
         # Workaround for CASSANDRA-8521 until PYTHON-205 is resolved.
@@ -588,6 +597,10 @@ class Shell(cmd.Cmd):
 
         self.current_keyspace = keyspace
 
+        self.display_timestamp_format = display_timestamp_format
+        self.display_nanotime_format = display_nanotime_format
+        self.display_date_format = display_date_format
+
         self.max_trace_wait = max_trace_wait
         self.session.max_trace_wait = max_trace_wait
         if encoding is None:
@@ -634,8 +647,10 @@ class Shell(cmd.Cmd):
         if isinstance(val, DecodeError):
             self.decoding_errors.append(val)
         try:
+            dtformats = DateTimeFormat(timestamp_format=self.display_timestamp_format,
+                                       date_format=self.display_date_format, nanotime_format=self.display_nanotime_format)
             return format_value(val, self.output_codec.name,
-                                addcolor=self.color, time_format=self.display_time_format,
+                                addcolor=self.color, date_time_format=dtformats,
                                 float_precision=self.display_float_precision, **kwargs)
         except Exception, e:
             err = FormatError(val, e)
@@ -1458,7 +1473,7 @@ class Shell(cmd.Cmd):
             cqltype = table_meta.columns[name].typestring
 
             if value != nullval:
-                if cqltype in ('ascii', 'text', 'timestamp', 'inet'):
+                if cqltype in ('ascii', 'text', 'timestamp', 'date', 'time', 'inet'):
                     rowmap[name] = protect_value(value)
                 else:
                     rowmap[name] = value
@@ -1520,7 +1535,7 @@ class Shell(cmd.Cmd):
                 return 0
         wmeter = meter.Meter()
         try:
-
+            dtformats = DateTimeFormat(self.display_timestamp_format, self.display_date_format, self.display_nanotime_format)
             dump = self.prep_export_dump(ks, cf, columns)
             writer = csv.writer(csvdest, **dialect_options)
             if header:
@@ -1528,7 +1543,7 @@ class Shell(cmd.Cmd):
             for row in dump:
                 fmt = lambda v: \
                     format_value(v, output_encoding=encoding, nullval=nullval,
-                                 time_format=self.display_time_format,
+                                 date_time_format=dtformats,
                                  float_precision=self.display_float_precision).strval
                 writer.writerow(map(fmt, row.values()))
                 wmeter.mark_written()
@@ -1610,7 +1625,9 @@ class Shell(cmd.Cmd):
         subshell = Shell(self.hostname, self.port,
                          color=self.color, encoding=self.encoding, stdin=f,
                          tty=False, use_conn=self.conn, cqlver=self.cql_version,
-                         display_time_format=self.display_time_format,
+                         display_timestamp_format=self.display_timestamp_format,
+                         display_date_format=self.display_date_format,
+                         display_nanotime_format=self.display_nanotime_format,
                          display_float_precision=self.display_float_precision,
                          max_trace_wait=self.max_trace_wait)
         subshell.cmdloop()
@@ -1918,7 +1935,11 @@ def read_options(cmdlineargs, environment):
                                                 DEFAULT_COMPLETEKEY)
     optvalues.color = option_with_default(configs.getboolean, 'ui', 'color')
     optvalues.time_format = raw_option_with_default(configs, 'ui', 'time_format',
-                                                    DEFAULT_TIME_FORMAT)
+                                                    DEFAULT_TIMESTAMP_FORMAT)
+    optvalues.nanotime_format = raw_option_with_default(configs, 'ui', 'nanotime_format',
+                                                    DEFAULT_NANOTIME_FORMAT)
+    optvalues.date_format = raw_option_with_default(configs, 'ui', 'date_format',
+                                                    DEFAULT_DATE_FORMAT)
     optvalues.float_precision = option_with_default(configs.getint, 'ui', 'float_precision',
                                                     DEFAULT_FLOAT_PRECISION)
     optvalues.max_trace_wait = option_with_default(configs.getfloat, 'tracing', 'max_trace_wait',
@@ -2035,7 +2056,9 @@ def main(options, hostname, port):
                       completekey=options.completekey,
                       cqlver=options.cqlversion,
                       keyspace=options.keyspace,
-                      display_time_format=options.time_format,
+                      display_timestamp_format=options.time_format,
+                      display_nanotime_format=options.nanotime_format,
+                      display_date_format=options.date_format,
                       display_float_precision=options.float_precision,
                       max_trace_wait=options.max_trace_wait,
                       ssl=options.ssl,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/doc/cql3/CQL.textile
----------------------------------------------------------------------
diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile
index 0221bc1..0c09b32 100644
--- a/doc/cql3/CQL.textile
+++ b/doc/cql3/CQL.textile
@@ -944,12 +944,14 @@ bc(syntax)..
                 | blob
                 | boolean
                 | counter
+                | date
                 | decimal
                 | double
                 | float
                 | inet
                 | int
                 | text
+                | time
                 | timestamp
                 | timeuuid
                 | uuid
@@ -970,13 +972,15 @@ p. The following table gives additional informations on the native data types, a
 |@blob@     |   blobs              |Arbitrary bytes (no validation)|
 |@boolean@  |   booleans           |true or false|
 |@counter@  |   integers           |Counter column (64-bit signed value). See "Counters":#counters for details|
+|@date@     |   integers, strings  |A date (with no corresponding time value).  See "Working with dates":#usingdates below for more information.|
 |@decimal@  |   integers, floats   |Variable-precision decimal|
 |@double@   |   integers           |64-bit IEEE-754 floating point|
 |@float@    |   integers, floats   |32-bit IEEE-754 floating point|
 |@inet@     |   strings            |An IP address. It can be either 4 bytes long (IPv4) or 16 bytes long (IPv6). There is no @inet@ constant, IP address should be inputed as strings|
 |@int@      |   integers           |32-bit signed int|
 |@text@     |   strings            |UTF8 encoded string|
-|@timestamp@|   integers, strings  |A timestamp. Strings constant are allow to input timestamps as dates, see "Working with dates":#usingdates below for more information.|
+|@time@     |   integers, strings  |A time with nanosecond precision.  See "Working with time":#usingtime below for more information.|
+|@timestamp@|   integers, strings  |A timestamp. Strings constant are allow to input timestamps as dates, see "Working with timestamps":#usingtimestamps below for more information.|
 |@timeuuid@ |   uuids              |Type 1 UUID. This is generally used as a "conflict-free" timestamp. Also see the "functions on Timeuuid":#timeuuidFun|
 |@uuid@     |   uuids              |Type 1 or type 4 UUID|
 |@varchar@  |   strings            |UTF8 encoded string|
@@ -984,7 +988,7 @@ p. The following table gives additional informations on the native data types, a
 
 For more information on how to use the collection types, see the "Working with collections":#collections section below.
 
-h3(#usingdates). Working with dates
+h3(#usingtimestamps). Working with timestamps
 
 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.
 
@@ -1018,6 +1022,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(#usingdates). Working with dates
+
+Values of the @date@ type are encoded as 32-bit unsigned integers representing a number of days with "the epoch" at the center of the range (2^31). Epoch is January 1st, 1970
+
+A date can be input in CQL as an unsigned integer as defined above.
+
+They can also be input as string literals in the following format:
+
+* @2014-01-01@
+
+
+h3(#usingtime). Working with time
+
+Values of the @time@ type are encoded as 64-bit signed integers representing the number of nanoseconds since midnight.
+
+A time can be input in CQL as simple long integers, giving the number of nanoseconds since midnight.
+
+They can also be input as string literals in any of the following formats:
+
+* @08:12:54@
+* @08:12:54.123@
+* @08:12:54.123456@
+* @08:12:54.123456789@
+
+
 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).
@@ -1182,7 +1211,7 @@ will never return any result by design, since the value returned by @now()@ is g
 
 h4. @minTimeuuid@ and @maxTimeuuid@
 
-The @minTimeuuid@ (resp. @maxTimeuuid@) function takes a @timestamp@ value @t@ (which can be "either a timestamp or a date string":#usingdates) and return a _fake_ @timeuuid@ corresponding to the _smallest_ (resp. _biggest_) possible @timeuuid@ having for timestamp @t@. So for instance:
+The @minTimeuuid@ (resp. @maxTimeuuid@) function takes a @timestamp@ value @t@ (which can be "either a timestamp or a date string":#usingtimestamps) and return a _fake_ @timeuuid@ corresponding to the _smallest_ (resp. _biggest_) possible @timeuuid@ having for timestamp @t@. 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')
@@ -1290,6 +1319,8 @@ CQL distinguishes between _reserved_ and _non-reserved_ keywords. Reserved keywo
 | @WITH@         | yes |
 | @WRITETIME@    | no  |
 | @DISTINCT@     | no  |
+| @DATE@         | no  |
+| @TIME@         | no  |
 
 h2(#appendixB). Appendix B: CQL Reserved Types
 
@@ -1338,7 +1369,7 @@ h3. 3.1.4
 
 h3. 3.1.3
 
-* Millisecond precision formats have been added to the timestamp parser (see "working with dates":#usingdates).
+* Millisecond precision formats have been added to the timestamp parser (see "working with dates":#usingtimestamps).
 
 h3. 3.1.2
 
@@ -1378,7 +1409,7 @@ h3. 3.0.2
 
 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.
+* "Date strings":#usingtimestamps (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.
 * "Float constants"#constants now support the exponent notation. In other words, @4.2E10@ is now a valid floating point value.
 
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/lib/joda-time-2.4.jar
----------------------------------------------------------------------
diff --git a/lib/joda-time-2.4.jar b/lib/joda-time-2.4.jar
new file mode 100644
index 0000000..ace67d7
Binary files /dev/null and b/lib/joda-time-2.4.jar differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/lib/licenses/joda-time-2.4.txt
----------------------------------------------------------------------
diff --git a/lib/licenses/joda-time-2.4.txt b/lib/licenses/joda-time-2.4.txt
new file mode 100644
index 0000000..261eeb9
--- /dev/null
+++ b/lib/licenses/joda-time-2.4.txt
@@ -0,0 +1,201 @@
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index 88f042e..0da6a7c 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -19,8 +19,8 @@ from cassandra.metadata import maybe_escape_name
 from cassandra.metadata import escape_name
 
 
-simple_cql_types = set(('ascii', 'bigint', 'blob', 'boolean', 'counter', 'decimal', 'double', 'float', 'inet', 'int',
-                        'text', 'timestamp', 'timeuuid', 'uuid', 'varchar', 'varint'))
+simple_cql_types = set(('ascii', 'bigint', 'blob', 'boolean', 'counter', 'date', 'decimal', 'double', 'float', 'inet', 'int',
+                        'text', 'time', 'timestamp', 'timeuuid', 'uuid', 'varchar', 'varint'))
 simple_cql_types.difference_update(('set', 'map', 'list'))
 
 from . import helptopics
@@ -42,7 +42,7 @@ class Cql3ParsingRuleSet(CqlParsingRuleSet):
         'limit', 'using', 'use', 'count', 'set',
         'begin', 'apply', 'batch', 'truncate', 'delete', 'in', 'create',
         'keyspace', 'schema', 'columnfamily', 'table', 'index', 'on', 'drop',
-        'primary', 'into', 'values', 'timestamp', 'ttl', 'alter', 'add', 'type',
+        'primary', 'into', 'values', 'date', 'time', 'timestamp', 'ttl', 'alter', 'add', 'type',
         'compact', 'storage', 'order', 'by', 'asc', 'desc', 'clustering',
         'token', 'writetime', 'map', 'list', 'to', 'custom', 'if', 'not'
     ))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/pylib/cqlshlib/displaying.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/displaying.py b/pylib/cqlshlib/displaying.py
index 13e3cf4..0cac309 100644
--- a/pylib/cqlshlib/displaying.py
+++ b/pylib/cqlshlib/displaying.py
@@ -95,6 +95,8 @@ DEFAULT_VALUE_COLORS = dict(
     error=RED,
     blob=DARK_MAGENTA,
     timestamp=GREEN,
+    date=GREEN,
+    time=GREEN,
     int=GREEN,
     float=GREEN,
     decimal=GREEN,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/pylib/cqlshlib/formatting.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/formatting.py b/pylib/cqlshlib/formatting.py
index f03540d..ac12fe6 100644
--- a/pylib/cqlshlib/formatting.py
+++ b/pylib/cqlshlib/formatting.py
@@ -14,13 +14,15 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-import re
-import time
 import calendar
 import math
+import re
+import time
+import sys
 from collections import defaultdict
 from . import wcwidth
 from .displaying import colorme, FormattedValue, DEFAULT_VALUE_COLORS
+from datetime import datetime, timedelta
 from cassandra.cqltypes import EMPTY
 
 unicode_controlchars_re = re.compile(r'[\x00-\x31\x7f-\xa0]')
@@ -45,13 +47,12 @@ def _make_turn_bits_red_f(color1, color2):
     return _turn_bits_red
 
 default_null_placeholder = 'null'
-default_time_format = ''
 default_float_precision = 3
 default_colormap = DEFAULT_VALUE_COLORS
 empty_colormap = defaultdict(lambda: '')
 
 def format_by_type(cqltype, val, encoding, colormap=None, addcolor=False,
-                   nullval=None, time_format=None, float_precision=None):
+                   nullval=None, date_time_format=None, float_precision=None):
     if nullval is None:
         nullval = default_null_placeholder
     if val is None:
@@ -60,12 +61,12 @@ def format_by_type(cqltype, val, encoding, colormap=None, addcolor=False,
         colormap = empty_colormap
     elif colormap is None:
         colormap = default_colormap
-    if time_format is None:
-        time_format = default_time_format
+    if date_time_format is None:
+        date_time_format = DateTimeFormat()
     if float_precision is None:
         float_precision = default_float_precision
     return format_value(cqltype, val, encoding=encoding, colormap=colormap,
-                        time_format=time_format, float_precision=float_precision,
+                        date_time_format=date_time_format, float_precision=float_precision,
                         nullval=nullval)
 
 def color_text(bval, colormap, displaywidth=None):
@@ -86,6 +87,16 @@ def color_text(bval, colormap, displaywidth=None):
         displaywidth -= bval.count(r'\\')
     return FormattedValue(bval, coloredval, displaywidth)
 
+DEFAULT_NANOTIME_FORMAT = '%H:%M:%S.%N'
+DEFAULT_DATE_FORMAT = '%Y-%m-%d'
+DEFAULT_TIMESTAMP_FORMAT = '%Y-%m-%d %H:%M:%S%z'
+
+class DateTimeFormat():
+    def __init__(self, timestamp_format=DEFAULT_TIMESTAMP_FORMAT, date_format=DEFAULT_DATE_FORMAT, nanotime_format=DEFAULT_NANOTIME_FORMAT):
+        self.timestamp_format=timestamp_format
+        self.date_format=date_format
+        self.nanotime_format=nanotime_format
+
 def format_value_default(val, colormap, **_):
     val = str(val)
     escapedval = val.replace('\\', '\\\\')
@@ -157,15 +168,13 @@ def format_integer_type(val, colormap, **_):
 formatter_for('long')(format_integer_type)
 formatter_for('int')(format_integer_type)
 
-@formatter_for('date')
-def format_value_timestamp(val, colormap, time_format, quote=False, **_):
-    bval = strftime(time_format, calendar.timegm(val.utctimetuple()))
+@formatter_for('datetime')
+def format_value_timestamp(val, colormap, date_time_format, quote=False, **_):
+    bval = strftime(date_time_format.timestamp_format, calendar.timegm(val.utctimetuple()))
     if quote:
         bval = "'%s'" % bval
     return colorme(bval, colormap, 'timestamp')
 
-formatter_for('datetime')(format_value_timestamp)
-
 def strftime(time_format, seconds):
     local = time.localtime(seconds)
     formatted = time.strftime(time_format, local)
@@ -183,6 +192,14 @@ def strftime(time_format, seconds):
     hours, minutes = divmod(abs(offset) / 60, 60)
     return formatted[:-5] + sign + '{0:0=2}{1:0=2}'.format(hours, minutes)
 
+@formatter_for('date')
+def format_value_uuid(val, colormap, **_):
+    return format_python_formatted_type(val, colormap, 'date')
+
+@formatter_for('Time')
+def format_value_time(val, colormap, **_):
+    return format_python_formatted_type(val, colormap, 'time')
+
 @formatter_for('str')
 def format_value_text(val, encoding, colormap, quote=False, **_):
     escapedval = val.replace(u'\\', u'\\\\')
@@ -199,9 +216,9 @@ def format_value_text(val, encoding, colormap, quote=False, **_):
 formatter_for('unicode')(format_value_text)
 
 def format_simple_collection(val, lbracket, rbracket, encoding,
-                             colormap, time_format, float_precision, nullval):
+                             colormap, date_time_format, float_precision, nullval):
     subs = [format_value(type(sval), sval, encoding=encoding, colormap=colormap,
-                         time_format=time_format, float_precision=float_precision,
+                         date_time_format=date_time_format, float_precision=float_precision,
                          nullval=nullval, quote=True)
             for sval in val]
     bval = lbracket + ', '.join(sval.strval for sval in subs) + rbracket
@@ -212,28 +229,28 @@ def format_simple_collection(val, lbracket, rbracket, encoding,
     return FormattedValue(bval, coloredval, displaywidth)
 
 @formatter_for('list')
-def format_value_list(val, encoding, colormap, time_format, float_precision, nullval, **_):
+def format_value_list(val, encoding, colormap, date_time_format, float_precision, nullval, **_):
     return format_simple_collection(val, '[', ']', encoding, colormap,
-                                    time_format, float_precision, nullval)
+                                    date_time_format, float_precision, nullval)
 
 @formatter_for('tuple')
-def format_value_tuple(val, encoding, colormap, time_format, float_precision, nullval, **_):
+def format_value_tuple(val, encoding, colormap, date_time_format, float_precision, nullval, **_):
     return format_simple_collection(val, '(', ')', encoding, colormap,
-                                    time_format, float_precision, nullval)
+                                    date_time_format, float_precision, nullval)
 
 @formatter_for('set')
-def format_value_set(val, encoding, colormap, time_format, float_precision, nullval, **_):
+def format_value_set(val, encoding, colormap, date_time_format, float_precision, nullval, **_):
     return format_simple_collection(sorted(val), '{', '}', encoding, colormap,
-                                    time_format, float_precision, nullval)
+                                    date_time_format, float_precision, nullval)
 formatter_for('frozenset')(format_value_set)
 formatter_for('sortedset')(format_value_set)
 
 
 @formatter_for('dict')
-def format_value_map(val, encoding, colormap, time_format, float_precision, nullval, **_):
+def format_value_map(val, encoding, colormap, date_time_format, float_precision, nullval, **_):
     def subformat(v):
         return format_value(type(v), v, encoding=encoding, colormap=colormap,
-                            time_format=time_format, float_precision=float_precision,
+                            date_time_format=date_time_format, float_precision=float_precision,
                             nullval=nullval, quote=True)
 
     subs = [(subformat(k), subformat(v)) for (k, v) in sorted(val.items())]
@@ -249,12 +266,12 @@ formatter_for('OrderedDict')(format_value_map)
 formatter_for('OrderedMap')(format_value_map)
 
 
-def format_value_utype(val, encoding, colormap, time_format, float_precision, nullval, **_):
+def format_value_utype(val, encoding, colormap, date_time_format, float_precision, nullval, **_):
     def format_field_value(v):
         if v is None:
             return colorme(nullval, colormap, 'error')
         return format_value(type(v), v, encoding=encoding, colormap=colormap,
-                            time_format=time_format, float_precision=float_precision,
+                            date_time_format=date_time_format, float_precision=float_precision,
                             nullval=nullval, quote=True)
 
     def format_field_name(name):

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/pylib/cqlshlib/helptopics.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/helptopics.py b/pylib/cqlshlib/helptopics.py
index bb5b382..65918ba 100644
--- a/pylib/cqlshlib/helptopics.py
+++ b/pylib/cqlshlib/helptopics.py
@@ -33,6 +33,8 @@ class CQLHelpTopics(object):
         one of the following topics:
 
           HELP TIMESTAMP_INPUT
+          HELP DATE_INPUT
+          HELP TIME_INPUT
           HELP BLOB_INPUT
           HELP UUID_INPUT
           HELP BOOLEAN_INPUT
@@ -69,6 +71,27 @@ class CQLHelpTopics(object):
         server node will be used.
         """
 
+    def help_date_input(self):
+        print """
+        Date input
+
+        CQL supports the following format for date specification:
+
+          yyyy-mm-dd
+        """
+
+    def help_time_input(self):
+        print """
+        Time input
+
+        CQL supports the following format for time specification:
+
+          HH:MM:SS
+          HH:MM:SS.mmm
+          HH:MM:SS.mmmuuu
+          HH:MM:SS.mmmuuunnn
+        """
+
     def help_blob_input(self):
         print """
         Blob input

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/pylib/cqlshlib/test/test_cqlsh_output.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/test/test_cqlsh_output.py b/pylib/cqlshlib/test/test_cqlsh_output.py
index 60ef3a4..7a7ed89 100644
--- a/pylib/cqlshlib/test/test_cqlsh_output.py
+++ b/pylib/cqlshlib/test/test_cqlsh_output.py
@@ -113,10 +113,11 @@ class TestCqlshOutput(BaseTestCase):
             self.assertTrue(6 <= len(output) <= 8,
                             msg='output: %r' % '\n'.join(output))
             self.assertEqual(output[0], '')
-            self.assertNicelyFormattedTableHeader(output[1])
-            self.assertNicelyFormattedTableRule(output[2])
-            self.assertNicelyFormattedTableData(output[3])
-            self.assertEqual(output[4].strip(), '')
+            self.assertEqual(output[1], 'Warning!  Non-ISO 8601 dates detected (< 1-1-1 or > 9999-12-31).  Returning raw days since epoch.');
+            self.assertNicelyFormattedTableHeader(output[2])
+            self.assertNicelyFormattedTableRule(output[3])
+            self.assertNicelyFormattedTableData(output[4])
+            self.assertEqual(output[5].strip(), '')
 
     def test_color_output(self):
         for termname in ('xterm', 'unknown-garbage'):
@@ -274,9 +275,9 @@ class TestCqlshOutput(BaseTestCase):
         # same query should show up as empty in cql 3
         self.assertQueriesGiveColoredOutput((
             (q, """
-             num | asciicol | bigintcol | blobcol | booleancol | decimalcol | doublecol | floatcol | intcol | textcol | timestampcol | uuidcol | varcharcol | varintcol
-             RRR   MMMMMMMM   MMMMMMMMM   MMMMMMM   MMMMMMMMMM   MMMMMMMMMM   MMMMMMMMM   MMMMMMMM   MMMMMM   MMMMMMM   MMMMMMMMMMMM   MMMMMMM   MMMMMMMMMM   MMMMMMMMM
-            -----+----------+-----------+---------+------------+------------+-----------+----------+--------+---------+--------------+---------+------------+-----------
+             num | asciicol | bigintcol | blobcol | booleancol | datecol | decimalcol | doublecol | floatcol | intcol | textcol | timecol | timestampcol | uuidcol | varcharcol | varintcol
+             RRR   MMMMMMMM   MMMMMMMMM   MMMMMMM   MMMMMMMMMM   MMMMMMM   MMMMMMMMMM   MMMMMMMMM   MMMMMMMM   MMMMMM   MMMMMMM   MMMMMMM   MMMMMMMMMMMM   MMMMMMM   MMMMMMMMMM   MMMMMMMMM
+            -----+----------+-----------+---------+------------+---------+------------+-----------+----------+--------+---------+---------+--------------+---------+------------+-----------
 
 
             (0 rows)
@@ -597,11 +598,13 @@ class TestCqlshOutput(BaseTestCase):
                 bigintcol bigint,
                 blobcol blob,
                 booleancol boolean,
+                datecol date,
                 decimalcol decimal,
                 doublecol double,
                 floatcol float,
                 intcol int,
                 textcol text,
+                timecol time,
                 timestampcol timestamp,
                 uuidcol uuid,
                 varcharcol text,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/pylib/cqlshlib/test/test_keyspace_init.cql
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/test/test_keyspace_init.cql b/pylib/cqlshlib/test/test_keyspace_init.cql
index cd5ac75..1d1e5aa 100644
--- a/pylib/cqlshlib/test/test_keyspace_init.cql
+++ b/pylib/cqlshlib/test/test_keyspace_init.cql
@@ -5,10 +5,12 @@ CREATE TABLE has_all_types (
     bigintcol bigint,
     blobcol blob,
     booleancol boolean,
+    datecol date,
     decimalcol decimal,
     doublecol double,
     floatcol float,
     textcol text,
+    timecol time,
     timestampcol timestamp,
     uuidcol uuid,
     varcharcol varchar,
@@ -16,40 +18,41 @@ CREATE TABLE has_all_types (
 ) WITH compression = {'sstable_compression':'LZ4Compressor'};
 
 INSERT INTO has_all_types (num, intcol, asciicol, bigintcol, blobcol, booleancol,
-                           decimalcol, doublecol, floatcol, textcol,
-                           timestampcol, uuidcol, varcharcol, varintcol)
+                           datecol, decimalcol, doublecol, floatcol, textcol,
+                           timecol, timestampcol, uuidcol, varcharcol, varintcol)
 VALUES (0, -12, 'abcdefg', 1234567890123456789, 0x000102030405fffefd, true,
-        19952.11882, 1.0, -2.1, 'Voilá!', '2012-05-14 12:53:20+0000',
-        bd1924e1-6af8-44ae-b5e1-f24131dbd460, '"', 10000000000000000000000000);
+        '2001-07-13', 19952.11882, 1.0, -2.1, 'Voilá!',
+        '23:12:15.889445123', '2012-05-14 12:53:20+0000', bd1924e1-6af8-44ae-b5e1-f24131dbd460, '"', 10000000000000000000000000);
 
 INSERT INTO has_all_types (num, intcol, asciicol, bigintcol, blobcol, booleancol,
-                           decimalcol, doublecol, floatcol, textcol,
-                           timestampcol, uuidcol, varcharcol, varintcol)
+                           datecol, decimalcol, doublecol, floatcol, textcol,
+                           timecol, timestampcol, uuidcol, varcharcol, varintcol)
 VALUES (1, 2147483647, '__!''$#@!~"', 9223372036854775807, 0xffffffffffffffffff, true,
-        0.00000000000001, 9999999.999, 99999.99, '∭Ƕ⑮ฑ➳❏''', '1900-01-01+0000',
-        ffffffff-ffff-ffff-ffff-ffffffffffff, 'newline->
+        '2012-11-30', 0.00000000000001, 9999999.999, 99999.99, '∭Ƕ⑮ฑ➳❏''',
+	'00:01:59.998994', '1900-01-01+0000', ffffffff-ffff-ffff-ffff-ffffffffffff, 'newline->
 <-', 9);
 
 INSERT INTO has_all_types (num, intcol, asciicol, bigintcol, blobcol, booleancol,
-                           decimalcol, doublecol, floatcol, textcol,
-                           timestampcol, uuidcol, varcharcol, varintcol)
+                           datecol, decimalcol, doublecol, floatcol, textcol,
+                           timecol, timestampcol, uuidcol, varcharcol, varintcol)
 VALUES (2, 0, '', 0, 0x, false,
-        0.0, 0.0, 0.0, '', 0,
-        00000000-0000-0000-0000-000000000000, '', 0);
+        0, 0.0, 0.0, 0.0, '',
+        '0:0:0.1', 0, 00000000-0000-0000-0000-000000000000, '', 0);
 
 INSERT INTO has_all_types (num, intcol, asciicol, bigintcol, blobcol, booleancol,
-                           decimalcol, doublecol, floatcol, textcol,
-                           timestampcol, uuidcol, varcharcol, varintcol)
+                           datecol, decimalcol, doublecol, floatcol, textcol,
+                           timecol, timestampcol, uuidcol, varcharcol, varintcol)
 VALUES (3, -2147483648, '''''''', -9223372036854775808, 0x80, false,
-        10.0000000000000, -1004.10, 100000000.9, '龍馭鬱', '2038-01-19T03:14-1200',
-        ffffffff-ffff-1fff-8fff-ffffffffffff, '''', -10000000000000000000000000);
+        '9999-12-31', 10.0000000000000, -1004.10, 100000000.9, '龍馭鬱',
+	'23:59:59.999999999', '2038-01-19T03:14-1200', ffffffff-ffff-1fff-8fff-ffffffffffff,
+	'''', -10000000000000000000000000);
 
 INSERT INTO has_all_types (num, intcol, asciicol, bigintcol, blobcol, booleancol,
                            decimalcol, doublecol, floatcol, textcol,
                            timestampcol, uuidcol, varcharcol, varintcol)
-VALUES (4, blobAsInt(0x), '', blobAsBigint(0x), 0x, blobAsBoolean(0x), blobAsDecimal(0x),
-        blobAsDouble(0x), blobAsFloat(0x), '', blobAsTimestamp(0x), blobAsUuid(0x), '',
-        blobAsVarint(0x));
+VALUES (4, blobAsInt(0x), '', blobAsBigint(0x), 0x, blobAsBoolean(0x),
+	blobAsDecimal(0x), blobAsDouble(0x), blobAsFloat(0x), '',
+	blobAsTimestamp(0x), blobAsUuid(0x), '', blobAsVarint(0x));
 
 
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/src/java/org/apache/cassandra/cql3/CQL3Type.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/CQL3Type.java b/src/java/org/apache/cassandra/cql3/CQL3Type.java
index b656de8..5fc518d 100644
--- a/src/java/org/apache/cassandra/cql3/CQL3Type.java
+++ b/src/java/org/apache/cassandra/cql3/CQL3Type.java
@@ -38,22 +38,24 @@ public interface CQL3Type
 
     public enum Native implements CQL3Type
     {
-        ASCII    (AsciiType.instance),
-        BIGINT   (LongType.instance),
-        BLOB     (BytesType.instance),
-        BOOLEAN  (BooleanType.instance),
-        COUNTER  (CounterColumnType.instance),
-        DECIMAL  (DecimalType.instance),
-        DOUBLE   (DoubleType.instance),
-        FLOAT    (FloatType.instance),
-        INET     (InetAddressType.instance),
-        INT      (Int32Type.instance),
-        TEXT     (UTF8Type.instance),
-        TIMESTAMP(TimestampType.instance),
-        UUID     (UUIDType.instance),
-        VARCHAR  (UTF8Type.instance),
-        VARINT   (IntegerType.instance),
-        TIMEUUID (TimeUUIDType.instance);
+        ASCII       (AsciiType.instance),
+        BIGINT      (LongType.instance),
+        BLOB        (BytesType.instance),
+        BOOLEAN     (BooleanType.instance),
+        COUNTER     (CounterColumnType.instance),
+        DECIMAL     (DecimalType.instance),
+        DOUBLE      (DoubleType.instance),
+        FLOAT       (FloatType.instance),
+        INET        (InetAddressType.instance),
+        INT         (Int32Type.instance),
+        TEXT        (UTF8Type.instance),
+        TIMESTAMP   (TimestampType.instance),
+        UUID        (UUIDType.instance),
+        VARCHAR     (UTF8Type.instance),
+        VARINT      (IntegerType.instance),
+        TIMEUUID    (TimeUUIDType.instance),
+        DATE        (SimpleDateType.instance),
+        TIME        (TimeType.instance);
 
         private final AbstractType<?> type;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/src/java/org/apache/cassandra/cql3/Constants.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Constants.java b/src/java/org/apache/cassandra/cql3/Constants.java
index 01fbdf0..2d5fdaa 100644
--- a/src/java/org/apache/cassandra/cql3/Constants.java
+++ b/src/java/org/apache/cassandra/cql3/Constants.java
@@ -44,7 +44,7 @@ public abstract class Constants
 
     public enum Type
     {
-        STRING, INTEGER, UUID, FLOAT, BOOLEAN, HEX;
+        STRING, INTEGER, UUID, FLOAT, DATE, TIME, BOOLEAN, HEX;
     }
 
     public static final Term.Raw NULL_LITERAL = new Term.Raw()
@@ -179,6 +179,8 @@ public abstract class Constants
                         case TEXT:
                         case INET:
                         case VARCHAR:
+                        case DATE:
+                        case TIME:
                         case TIMESTAMP:
                             return true;
                     }
@@ -188,6 +190,7 @@ public abstract class Constants
                     {
                         case BIGINT:
                         case COUNTER:
+                        case DATE:
                         case DECIMAL:
                         case DOUBLE:
                         case FLOAT:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/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 6a21a8b..a1545d3 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -1161,6 +1161,8 @@ native_type returns [CQL3Type t]
     | K_VARCHAR   { $t = CQL3Type.Native.VARCHAR; }
     | K_VARINT    { $t = CQL3Type.Native.VARINT; }
     | K_TIMEUUID  { $t = CQL3Type.Native.TIMEUUID; }
+    | K_DATE      { $t = CQL3Type.Native.DATE; }
+    | K_TIME      { $t = CQL3Type.Native.TIME; }
     ;
 
 collection_type returns [CQL3Type.Raw pt]
@@ -1326,6 +1328,8 @@ K_VARINT:      V A R I N T;
 K_TIMEUUID:    T I M E U U I D;
 K_TOKEN:       T O K E N;
 K_WRITETIME:   W R I T E T I M E;
+K_DATE:        D A T E;
+K_TIME:        T I M E;
 
 K_NULL:        N U L L;
 K_NOT:         N O T;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/src/java/org/apache/cassandra/db/marshal/IntegerType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/IntegerType.java b/src/java/org/apache/cassandra/db/marshal/IntegerType.java
index ec1c7ad..6da8d39 100644
--- a/src/java/org/apache/cassandra/db/marshal/IntegerType.java
+++ b/src/java/org/apache/cassandra/db/marshal/IntegerType.java
@@ -60,6 +60,11 @@ public final class IntegerType extends AbstractType<BigInteger>
 
     public int compare(ByteBuffer lhs, ByteBuffer rhs)
     {
+        return IntegerType.compareIntegers(lhs, rhs);
+    }
+
+    public static int compareIntegers(ByteBuffer lhs, ByteBuffer rhs)
+    {
         int lhsLen = lhs.remaining();
         int rhsLen = rhs.remaining();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/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
new file mode 100644
index 0000000..1dbcf03
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/marshal/SimpleDateType.java
@@ -0,0 +1,72 @@
+/*
+ * 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.db.marshal;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.serializers.SimpleDateSerializer;
+import org.apache.cassandra.serializers.TypeSerializer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class SimpleDateType extends AbstractType<Integer>
+{
+    public static final SimpleDateType instance = new SimpleDateType();
+
+    SimpleDateType() {} // singleton
+
+    public int compare(ByteBuffer o1, ByteBuffer o2)
+    {
+        // We add Integer.MIN_VALUE to overflow to allow unsigned comparison
+        return ByteBufferUtil.compareUnsigned(o1, o2);
+    }
+
+    public boolean isByteOrderComparable()
+    {
+        return true;
+    }
+
+    public ByteBuffer fromString(String source) throws MarshalException
+    {
+        return ByteBufferUtil.bytes(SimpleDateSerializer.dateStringToDays(source));
+    }
+
+    @Override
+    public boolean isCompatibleWith(AbstractType<?> previous)
+    {
+        return super.isCompatibleWith(previous);
+    }
+
+    @Override
+    public boolean isValueCompatibleWithInternal(AbstractType<?> otherType)
+    {
+        return this == otherType || otherType == IntegerType.instance;
+    }
+
+    @Override
+    public CQL3Type asCQL3Type()
+    {
+        return CQL3Type.Native.DATE;
+    }
+
+    public TypeSerializer<Integer> getSerializer()
+    {
+        return SimpleDateSerializer.instance;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/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
new file mode 100644
index 0000000..b9a0076
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/marshal/TimeType.java
@@ -0,0 +1,72 @@
+/*
+ * 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.db.marshal;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.serializers.TimeSerializer;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.serializers.TypeSerializer;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+/**
+ * Nanosecond resolution time values
+ */
+public class TimeType extends AbstractType<Long>
+{
+    public static final TimeType instance = new TimeType();
+    private TimeType() {} // singleton
+
+    public int compare(ByteBuffer o1, ByteBuffer o2)
+    {
+        return ByteBufferUtil.compareUnsigned(o1, o2);
+    }
+
+    public ByteBuffer fromString(String source) throws MarshalException
+    {
+        return decompose(TimeSerializer.timeStringToLong(source));
+    }
+
+    public boolean isByteOrderComparable()
+    {
+        return true;
+    }
+
+    @Override
+    public boolean isCompatibleWith(AbstractType<?> previous)
+    {
+        return super.isCompatibleWith(previous);
+    }
+
+    @Override
+    public boolean isValueCompatibleWithInternal(AbstractType<?> otherType)
+    {
+        return this == otherType || otherType == LongType.instance;
+    }
+
+    public CQL3Type asCQL3Type()
+    {
+        return CQL3Type.Native.TIME;
+    }
+
+    public TypeSerializer<Long> getSerializer()
+    {
+        return TimeSerializer.instance;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/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
new file mode 100644
index 0000000..221842b
--- /dev/null
+++ b/src/java/org/apache/cassandra/serializers/SimpleDateSerializer.java
@@ -0,0 +1,117 @@
+/*
+ * 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.serializers;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+import org.joda.time.DateTime;
+import org.joda.time.DateTimeZone;
+import org.joda.time.LocalDate;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+// For byte-order comparability, we shift by Integer.MIN_VALUE and treat the data as an unsigned integer ranging from
+// min date to max date w/epoch sitting in the center @ 2^31
+public class SimpleDateSerializer implements TypeSerializer<Integer>
+{
+    private static final DateTimeFormatter formatter = DateTimeFormat.forPattern("yyyy-MM-dd").withZone(DateTimeZone.UTC);
+    private static final long minSupportedDateMillis = TimeUnit.DAYS.toMillis(Integer.MIN_VALUE);
+    private static final long maxSupportedDateMillis = TimeUnit.DAYS.toMillis(Integer.MAX_VALUE);
+    private static final long maxSupportedDays = (long)Math.pow(2,32) - 1;
+    private static final long byteOrderShift = (long)Math.pow(2,31) * 2;
+
+    private static final Pattern rawPattern = Pattern.compile("^-?\\d+$");
+    public static final SimpleDateSerializer instance = new SimpleDateSerializer();
+
+    public Integer deserialize(ByteBuffer bytes)
+    {
+        return bytes.remaining() == 0 ? null : ByteBufferUtil.toInt(bytes);
+    }
+
+    public ByteBuffer serialize(Integer value)
+    {
+        return value == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : ByteBufferUtil.bytes(value);
+    }
+
+    public static Integer dateStringToDays(String source) throws MarshalException
+    {
+        // Raw day value in unsigned int form, epoch @ 2^31
+        if (rawPattern.matcher(source).matches())
+        {
+            try
+            {
+                Long result = Long.parseLong(source);
+
+                if (result < 0 || result > maxSupportedDays)
+                    throw new NumberFormatException("Input out of bounds: " + source);
+
+                // Shift > epoch days into negative portion of Integer result for byte order comparability
+                if (result >= Integer.MAX_VALUE)
+                    result -= byteOrderShift;
+
+                return result.intValue();
+            }
+            catch (NumberFormatException e)
+            {
+                throw new MarshalException(String.format("Unable to make unsigned int (for date) from: '%s'", source), e);
+            }
+        }
+
+        // Attempt to parse as date string
+        try
+        {
+            DateTime parsed = formatter.parseDateTime(source);
+            long millis = parsed.getMillis();
+            if (millis < minSupportedDateMillis)
+                throw new MarshalException(String.format("Input date %s is less than min supported date %s", source, new LocalDate(minSupportedDateMillis).toString()));
+            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;
+        }
+        catch (IllegalArgumentException e1)
+        {
+            throw new MarshalException(String.format("Unable to coerce '%s' to a formatted date (long)", source), e1);
+        }
+    }
+
+    public void validate(ByteBuffer bytes) throws MarshalException
+    {
+        if (bytes.remaining() != 4)
+            throw new MarshalException(String.format("Expected 4 byte long for date (%d)", bytes.remaining()));
+    }
+
+    public String toString(Integer value)
+    {
+        if (value == null)
+            return "";
+
+        return formatter.print(new LocalDate(TimeUnit.DAYS.toMillis(value - Integer.MIN_VALUE), DateTimeZone.UTC));
+    }
+
+    public Class<Integer> getType()
+    {
+        return Integer.class;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/src/java/org/apache/cassandra/serializers/TimeSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/serializers/TimeSerializer.java b/src/java/org/apache/cassandra/serializers/TimeSerializer.java
new file mode 100644
index 0000000..b02d425
--- /dev/null
+++ b/src/java/org/apache/cassandra/serializers/TimeSerializer.java
@@ -0,0 +1,198 @@
+/*
+ * 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.serializers;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class TimeSerializer implements TypeSerializer<Long>
+{
+    public static final Pattern timePattern = Pattern.compile("^-?\\d+$");
+    public static final TimeSerializer instance = new TimeSerializer();
+
+    public Long deserialize(ByteBuffer bytes)
+    {
+        return bytes.remaining() == 0 ? null : ByteBufferUtil.toLong(bytes);
+    }
+
+    public ByteBuffer serialize(Long value)
+    {
+        return value == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : ByteBufferUtil.bytes(value);
+    }
+
+    public static Long timeStringToLong(String source) throws MarshalException
+    {
+        // nano since start of day, raw
+        if (timePattern.matcher(source).matches())
+        {
+            try
+            {
+                Long result = Long.parseLong(source);
+                if (result < 0 || result > TimeUnit.DAYS.toNanos(1))
+                    throw new NumberFormatException("Input long out of bounds: " + source);
+                return result;
+            }
+            catch (NumberFormatException e)
+            {
+                throw new MarshalException(String.format("Unable to make long (for time) from: '%s'", source), e);
+            }
+        }
+
+        // Last chance, attempt to parse as time string
+        try
+        {
+            return parseTimeStrictly(source);
+        }
+        catch (IllegalArgumentException e1)
+        {
+            throw new MarshalException(String.format("(TimeType) Unable to coerce '%s' to a formatted time (long)", source), e1);
+        }
+    }
+
+    public void validate(ByteBuffer bytes) throws MarshalException
+    {
+        if (bytes.remaining() != 8)
+            throw new MarshalException(String.format("Expected 8 byte long for time (%d)", bytes.remaining()));
+    }
+
+    public String toString(Long value)
+    {
+        if (value == null)
+            return "null";
+
+        int nano = (int)(value % 1000);
+        value -= nano;
+        value /= 1000;
+        int micro = (int)(value % 1000);
+        value -= micro;
+        value /= 1000;
+        int milli = (int)(value % 1000);
+        value -= milli;
+        value /= 1000;
+        int seconds = (int)(value % 60);
+        value -= seconds;
+        value /= 60;
+        int minutes = (int)(value % 60);
+        value -= minutes;
+        value /= 60;
+        int hours = (int)(value % 24);
+        value -= hours;
+        value /= 24;
+        assert(value == 0);
+
+        StringBuilder sb = new StringBuilder();
+        leftPadZeros(hours, 2, sb);
+        sb.append(":");
+        leftPadZeros(minutes, 2, sb);
+        sb.append(":");
+        leftPadZeros(seconds, 2, sb);
+        sb.append(".");
+        leftPadZeros(milli, 3, sb);
+        leftPadZeros(micro, 3, sb);
+        leftPadZeros(nano, 3, sb);
+        return sb.toString();
+    }
+
+    private void leftPadZeros(int value, int digits, StringBuilder sb)
+    {
+        for (int i = 1; i < digits; ++i)
+        {
+            if (value < Math.pow(10, i))
+                sb.append("0");
+        }
+        sb.append(value);
+    }
+
+    public Class<Long> getType()
+    {
+        return Long.class;
+    }
+
+    // Time specific parsing loosely based on java.sql.Timestamp
+    private static Long parseTimeStrictly(String s) throws IllegalArgumentException
+    {
+        String nanos_s;
+
+        long hour;
+        long minute;
+        long second;
+        long a_nanos = 0;
+
+        int firstColon = 0;
+        int secondColon = 0;
+        int period = 0;
+        String formatError = "Timestamp format must be hh:mm:ss[.fffffffff]";
+        String zeros = "000000000";
+
+        if (s == null)
+            throw new java.lang.IllegalArgumentException(formatError);
+        s = s.trim();
+
+        // Parse the time
+        firstColon = s.indexOf(':');
+        secondColon = s.indexOf(':', firstColon+1);
+        period = s.indexOf('.', secondColon+1);
+
+        // Convert the time; default missing nanos
+        if (firstColon > 0 && secondColon > 0 && secondColon < s.length() - 1)
+        {
+            hour = Integer.parseInt(s.substring(0, firstColon));
+            if (hour < 0 || hour >= 24)
+                throw new IllegalArgumentException("Hour out of bounds.");
+
+            minute = Integer.parseInt(s.substring(firstColon + 1, secondColon));
+            if (minute < 0 || minute >= 60)
+                throw new IllegalArgumentException("Minute out of bounds.");
+
+            if (period > 0 && period < s.length() - 1)
+            {
+                second = Integer.parseInt(s.substring(secondColon + 1, period));
+                if (second < 0 || second >= 60)
+                    throw new IllegalArgumentException("Second out of bounds.");
+
+                nanos_s = s.substring(period + 1);
+                if (nanos_s.length() > 9)
+                    throw new IllegalArgumentException(formatError);
+                if (!Character.isDigit(nanos_s.charAt(0)))
+                    throw new IllegalArgumentException(formatError);
+                nanos_s = nanos_s + zeros.substring(0, 9 - nanos_s.length());
+                a_nanos = Integer.parseInt(nanos_s);
+            }
+            else if (period > 0)
+                throw new IllegalArgumentException(formatError);
+            else
+            {
+                second = Integer.parseInt(s.substring(secondColon + 1));
+                if (second < 0 || second >= 60)
+                    throw new IllegalArgumentException("Second out of bounds.");
+            }
+        }
+        else
+            throw new IllegalArgumentException(formatError);
+
+        long rawTime = 0;
+        rawTime += TimeUnit.HOURS.toNanos(hour);
+        rawTime += TimeUnit.MINUTES.toNanos(minute);
+        rawTime += TimeUnit.SECONDS.toNanos(second);
+        rawTime += a_nanos;
+        return rawTime;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/src/java/org/apache/cassandra/serializers/TypeSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/serializers/TypeSerializer.java b/src/java/org/apache/cassandra/serializers/TypeSerializer.java
index 7b037c0..cddef08 100644
--- a/src/java/org/apache/cassandra/serializers/TypeSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/TypeSerializer.java
@@ -35,3 +35,4 @@ public interface TypeSerializer<T>
 
     public Class<T> getType();
 }
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/src/java/org/apache/cassandra/transport/DataType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/DataType.java b/src/java/org/apache/cassandra/transport/DataType.java
index 0ea353a..e13194d 100644
--- a/src/java/org/apache/cassandra/transport/DataType.java
+++ b/src/java/org/apache/cassandra/transport/DataType.java
@@ -49,6 +49,8 @@ public enum DataType implements OptionCodec.Codecable<DataType>
     VARINT   (14, IntegerType.instance),
     TIMEUUID (15, TimeUUIDType.instance),
     INET     (16, InetAddressType.instance),
+    DATE     (17, DateType.instance),
+    TIME     (18, TimeType.instance),
     LIST     (32, null),
     MAP      (33, null),
     SET      (34, null),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/test/unit/org/apache/cassandra/db/marshal/SimpleDateTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/marshal/SimpleDateTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/SimpleDateTypeTest.java
new file mode 100644
index 0000000..5c9ed4e
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/marshal/SimpleDateTypeTest.java
@@ -0,0 +1,153 @@
+/**
+ * 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.db.marshal;
+
+import java.nio.ByteBuffer;
+
+import org.junit.Test;
+import org.apache.cassandra.serializers.SimpleDateSerializer;
+
+public class SimpleDateTypeTest
+{
+    @Test public void TestComparison()
+    {
+        ByteBuffer d1 = SimpleDateType.instance.fromString("1970-01-05");
+        ByteBuffer d2 = SimpleDateSerializer.instance.serialize(makeUnsigned(4));
+        assert SimpleDateType.instance.compare(d1, d2) == 0 : "Failed == comparison";
+            String.format("Failed == comparison with %s and %s",
+                SimpleDateSerializer.instance.deserialize(d1),
+                SimpleDateSerializer.instance.deserialize(d2));
+
+        d1 = SimpleDateType.instance.fromString("1970-01-05");
+        d2 = SimpleDateSerializer.instance.serialize(makeUnsigned(10));
+        assert SimpleDateType.instance.compare(d1, d2) < 0 :
+            String.format("Failed comparison of %s and %s, expected <",
+                SimpleDateSerializer.instance.deserialize(d1),
+                SimpleDateSerializer.instance.deserialize(d2));
+
+        d1 = SimpleDateType.instance.fromString("1970-01-05"); // 4
+        d2 = SimpleDateSerializer.instance.serialize(makeUnsigned(-10));
+        assert SimpleDateType.instance.compare(d1, d2) > 0 :
+            String.format("Failed comparison of %s and %s, expected > 0",
+                SimpleDateSerializer.instance.deserialize(d1),
+                SimpleDateSerializer.instance.deserialize(d2));
+
+        d1 = SimpleDateType.instance.fromString("1");
+        d2 = SimpleDateType.instance.fromString("1000");
+        assert SimpleDateType.instance.compare(d1, d2) < 0 :
+                String.format("Failed < comparison with string inputs %s and %s",
+                        SimpleDateSerializer.instance.deserialize(d1),
+                        SimpleDateSerializer.instance.deserialize(d2));
+
+        Integer intLimit = Integer.MAX_VALUE;
+        d1 = SimpleDateType.instance.fromString("0");
+        d2 = SimpleDateType.instance.fromString(intLimit.toString());
+        assert SimpleDateType.instance.compare(d1, d2) < 0 :
+                String.format("Failed < comparison with string inputs at integer bounds %s and %s",
+                        SimpleDateSerializer.instance.deserialize(d1),
+                        SimpleDateSerializer.instance.deserialize(d2));
+
+        Long overLimit = (long)(Integer.MAX_VALUE);
+        d1 = SimpleDateType.instance.fromString("0");
+        d2 = SimpleDateType.instance.fromString(overLimit.toString());
+        assert SimpleDateType.instance.compare(d1, d2) < 0 :
+                String.format("Failed < comparison with string inputs at integer bounds %s and %s",
+                        SimpleDateSerializer.instance.deserialize(d1),
+                        SimpleDateSerializer.instance.deserialize(d2));
+
+        Long i1 = 0L;
+        Long i2 = (long)Math.pow(2,32) - 1;
+        d1 = SimpleDateType.instance.fromString(i1.toString());
+        d2 = SimpleDateType.instance.fromString(i2.toString());
+        assert SimpleDateType.instance.compare(d1, d2) < 0 :
+            String.format("Failed limits comparison with %s and %s",
+                SimpleDateSerializer.instance.deserialize(d1),
+                SimpleDateSerializer.instance.deserialize(d2));
+
+        d1 = SimpleDateType.instance.fromString("256");
+        d2 = SimpleDateType.instance.fromString("512");
+        assert SimpleDateType.instance.compare(d1, d2) < 0 :
+            String.format("Failed comparison with %s and %s",
+                SimpleDateSerializer.instance.deserialize(d1),
+                SimpleDateSerializer.instance.deserialize(d2));
+
+        d1 = SimpleDateSerializer.instance.serialize(makeUnsigned(0));
+        d2 = SimpleDateSerializer.instance.serialize(makeUnsigned(Integer.MAX_VALUE));
+        assert SimpleDateType.instance.compare(d1, d2) < 0 :
+            String.format("Failed neg/pos comparison with %s and %s",
+                SimpleDateSerializer.instance.deserialize(d1),
+                SimpleDateSerializer.instance.deserialize(d2));
+
+        d1 = SimpleDateType.instance.fromString("-10000-10-10");
+        d2 = SimpleDateType.instance.fromString("10000-10-10");
+        assert SimpleDateType.instance.compare(d1, d2) < 0 :
+            String.format("Failed neg/pos string comparison with %s and %s",
+                SimpleDateSerializer.instance.deserialize(d1),
+                SimpleDateSerializer.instance.deserialize(d2));
+
+        d1 = SimpleDateType.instance.fromString("1969-12-31");
+        d2 = SimpleDateType.instance.fromString("1970-1-1");
+        assert SimpleDateType.instance.compare(d1, d2) < 0 :
+            String.format("Failed pre/post epoch comparison with %s and %s",
+                SimpleDateSerializer.instance.deserialize(d1),
+                SimpleDateSerializer.instance.deserialize(d2));
+
+        d1 = SimpleDateType.instance.fromString("1970-1-1");
+        d2 = SimpleDateType.instance.fromString("1970-1-1");
+        assert SimpleDateType.instance.compare(d1, d2) == 0 :
+            String.format("Failed == date from string comparison with %s and %s",
+                SimpleDateSerializer.instance.deserialize(d1),
+                SimpleDateSerializer.instance.deserialize(d2));
+
+        d1 = SimpleDateType.instance.fromString("1970-1-1");
+        d2 = SimpleDateType.instance.fromString("1970-1-2");
+        assert SimpleDateType.instance.compare(d1, d2) < 0 :
+            String.format("Failed post epoch string comparison with %s and %s",
+                SimpleDateSerializer.instance.deserialize(d1),
+                SimpleDateSerializer.instance.deserialize(d2));
+
+        for (int i = 0; i < 32; ++i)
+        {
+            int offset = (int)Math.pow(2,i);
+            d1 = SimpleDateSerializer.instance.serialize(makeUnsigned(0 - offset));
+            d2 = SimpleDateSerializer.instance.serialize(makeUnsigned(offset));
+            assert SimpleDateType.instance.compare(d1, d2) < 0 :
+                String.format("Failed < comparison of %s and %s",
+                    SimpleDateSerializer.instance.deserialize(d1),
+                    SimpleDateSerializer.instance.deserialize(d2));
+        }
+
+        for (int i = 0; i < 32; ++i)
+        {
+            int offset = (int)Math.pow(2,i);
+            d1 = SimpleDateSerializer.instance.serialize(makeUnsigned(offset));
+            d2 = SimpleDateSerializer.instance.serialize(makeUnsigned(0 - offset));
+            assert SimpleDateType.instance.compare(d1, d2) > 0 :
+                String.format("Failed > comparison of %s and %s",
+                    SimpleDateSerializer.instance.deserialize(d1),
+                    SimpleDateSerializer.instance.deserialize(d2));
+        }
+    }
+
+    private Integer makeUnsigned(int input)
+    {
+        return input - Integer.MIN_VALUE;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/107545b3/test/unit/org/apache/cassandra/db/marshal/TimeTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/marshal/TimeTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/TimeTypeTest.java
new file mode 100644
index 0000000..3057b9c
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/marshal/TimeTypeTest.java
@@ -0,0 +1,61 @@
+/**
+ * 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.db.marshal;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.Test;
+import org.apache.cassandra.serializers.TimeSerializer;
+
+public class TimeTypeTest
+{
+    @Test public void TestComparison()
+    {
+        Long t1 = TimeSerializer.timeStringToLong("01:00:00.123456789");
+        Long t2 = new Long((1L * 60L * 60L * 1000L * 1000L * 1000L) + 123456789);
+        ByteBuffer b1 = TimeSerializer.instance.serialize(t1);
+        ByteBuffer b2 = TimeSerializer.instance.serialize(t2);
+        assert TimeType.instance.compare(b1, b2) == 0 : "Failed == comparison";
+
+        b2 = TimeSerializer.instance.serialize(123456789L);
+        assert TimeType.instance.compare(b1, b2) > 0 : "Failed > comparison";
+
+        t2 = new Long(2L * 60L * 60L * 1000L * 1000L * 1000L + 123456789);
+        b2 = TimeSerializer.instance.serialize(t2);
+        assert TimeType.instance.compare(b1, b2) < 0 : "Failed < comparison";
+
+        b1 = TimeSerializer.instance.serialize(0L);
+        b2 = TimeSerializer.instance.serialize(0L);
+        assert TimeType.instance.compare(b1, b2) == 0 : "Failed == comparison on 0";
+
+        b1 = TimeSerializer.instance.serialize(0L);
+        b2 = TimeSerializer.instance.serialize(10000000L);
+        assert TimeType.instance.compare(b1, b2) == -1 : "Failed < comparison on 0";
+
+        b1 = TimeSerializer.instance.serialize(0L);
+        b2 = TimeSerializer.instance.serialize(TimeUnit.DAYS.toNanos(1));
+        assert TimeType.instance.compare(b1, b2) == -1 : "Failed < comparison against max range.";
+
+        b1 = TimeSerializer.instance.serialize(TimeUnit.DAYS.toNanos(1));
+        b2 = TimeSerializer.instance.serialize(0L);
+        assert TimeType.instance.compare(b1, b2) == 1 : "Failed > comparison against max range.";
+    }
+}