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

cassandra git commit: Fix equality comparisons of columns using the duration type

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.11 0fe76da17 -> 6487876dd


Fix equality comparisons of columns using the duration type

patch by Benjamin Lerer; reviewed by Tyler Hobbs for CASSANDRA-13174


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

Branch: refs/heads/cassandra-3.11
Commit: 6487876dde14c46d5753f972909e5acec854cb53
Parents: 0fe76da
Author: Benjamin Lerer <b....@gmail.com>
Authored: Thu Feb 23 14:05:30 2017 +0100
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Thu Feb 23 14:05:30 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/cql3/ColumnCondition.java  |  19 ++
 .../org/apache/cassandra/cql3/Operator.java     |   9 +
 .../cassandra/cql3/SingleColumnRelation.java    |  10 +-
 .../cql3/statements/CreateIndexStatement.java   |  12 +
 .../cassandra/db/marshal/AbstractType.java      |   5 +
 .../cassandra/db/marshal/DurationType.java      |   3 +-
 .../apache/cassandra/db/marshal/TupleType.java  |   6 +
 .../apache/cassandra/db/marshal/UserType.java   |   6 +
 .../validation/entities/SecondaryIndexTest.java |  27 ++
 .../cql3/validation/operations/CreateTest.java  |   4 +-
 .../operations/InsertUpdateIfConditionTest.java | 250 +++++++++++++++++++
 .../cql3/validation/operations/SelectTest.java  | 165 ++++++++++++
 13 files changed, 512 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6487876d/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 3e38844..233898f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.11.0
+ * Fix equality comparisons of columns using the duration type (CASSANDRA-13174)
  * Obfuscate password in stress-graphs (CASSANDRA-12233)
  * Move to FastThreadLocalThread and FastThreadLocal (CASSANDRA-13034)
  * nodetool stopdaemon errors out (CASSANDRA-13030)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6487876d/src/java/org/apache/cassandra/cql3/ColumnCondition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ColumnCondition.java b/src/java/org/apache/cassandra/cql3/ColumnCondition.java
index 5395a9b..acb95a4 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnCondition.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnCondition.java
@@ -26,12 +26,16 @@ import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.Term.Terminal;
 import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.statements.RequestValidations;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
 /**
  * A CQL3 condition on the value of a column or collection element.  For example, "UPDATE .. IF a = 0".
  */
@@ -1037,6 +1041,7 @@ public class ColumnCondition
                     default:
                         throw new AssertionError();
                 }
+
                 if (operator == Operator.IN)
                 {
                     if (inValues == null)
@@ -1048,6 +1053,7 @@ public class ColumnCondition
                 }
                 else
                 {
+                    validateOperationOnDurations(valueSpec.type);
                     return ColumnCondition.condition(receiver, collectionElement.prepare(keyspace, elementSpec), value.prepare(keyspace, valueSpec), operator);
                 }
             }
@@ -1071,6 +1077,7 @@ public class ColumnCondition
                 }
                 else
                 {
+                    validateOperationOnDurations(fieldReceiver.type);
                     return ColumnCondition.condition(receiver, udtField, value.prepare(keyspace, fieldReceiver), operator);
                 }
             }
@@ -1087,9 +1094,21 @@ public class ColumnCondition
                 }
                 else
                 {
+                    validateOperationOnDurations(receiver.type);
                     return ColumnCondition.condition(receiver, value.prepare(keyspace, receiver), operator);
                 }
             }
         }
+
+        private void validateOperationOnDurations(AbstractType<?> type)
+        {
+            if (type.referencesDuration() && operator.isSlice())
+            {
+                checkFalse(type.isCollection(), "Slice conditions are not supported on collections containing durations");
+                checkFalse(type.isTuple(), "Slice conditions are not supported on tuples containing durations");
+                checkFalse(type.isUDT(), "Slice conditions are not supported on UDTs containing durations");
+                throw invalidRequest("Slice conditions are not supported on durations", operator);
+            }
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6487876d/src/java/org/apache/cassandra/cql3/Operator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Operator.java b/src/java/org/apache/cassandra/cql3/Operator.java
index 07c92f0..8c04bef 100644
--- a/src/java/org/apache/cassandra/cql3/Operator.java
+++ b/src/java/org/apache/cassandra/cql3/Operator.java
@@ -249,6 +249,15 @@ public enum Operator
         return 4;
     }
 
+    /**
+     * Checks if this operator is a slice operator.
+     * @return {@code true} if this operator is a slice operator, {@code false} otherwise.
+     */
+    public boolean isSlice()
+    {
+        return this == LT || this == LTE || this == GT || this == GTE;
+    }
+
     @Override
     public String toString()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6487876d/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
index 719ef68..ae07f56 100644
--- a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
+++ b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
@@ -28,7 +28,6 @@ import org.apache.cassandra.cql3.restrictions.Restriction;
 import org.apache.cassandra.cql3.restrictions.SingleColumnRestriction;
 import org.apache.cassandra.cql3.statements.Bound;
 import org.apache.cassandra.db.marshal.CollectionType;
-import org.apache.cassandra.db.marshal.DurationType;
 import org.apache.cassandra.db.marshal.ListType;
 import org.apache.cassandra.db.marshal.MapType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -198,7 +197,14 @@ public final class SingleColumnRelation extends Relation
                                               boolean inclusive) throws InvalidRequestException
     {
         ColumnDefinition columnDef = entity.prepare(cfm);
-        checkFalse(columnDef.type instanceof DurationType, "Slice restriction are not supported on duration columns");
+
+        if (columnDef.type.referencesDuration())
+        {
+            checkFalse(columnDef.type.isCollection(), "Slice restrictions are not supported on collections containing durations");
+            checkFalse(columnDef.type.isTuple(), "Slice restrictions are not supported on tuples containing durations");
+            checkFalse(columnDef.type.isUDT(), "Slice restrictions are not supported on UDTs containing durations");
+            throw invalidRequest("Slice restrictions are not supported on duration columns");
+        }
 
         Term term = toTerm(toReceivers(columnDef), value, cfm.ksName, boundNames);
         return new SingleColumnRestriction.SliceRestriction(columnDef, bound, inclusive, term);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6487876d/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
index 7ee7d30..ed4658f 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
@@ -33,6 +33,7 @@ import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.CFName;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.IndexName;
+import org.apache.cassandra.db.marshal.DurationType;
 import org.apache.cassandra.db.marshal.MapType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.RequestValidationException;
@@ -45,6 +46,9 @@ import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.thrift.ThriftValidation;
 import org.apache.cassandra.transport.Event;
 
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
 /** A <code>CREATE INDEX</code> statement parsed from a CQL query. */
 public class CreateIndexStatement extends SchemaAlteringStatement
 {
@@ -103,6 +107,14 @@ public class CreateIndexStatement extends SchemaAlteringStatement
             if (cd == null)
                 throw new InvalidRequestException("No column definition found for column " + target.column);
 
+            if (cd.type.referencesDuration())
+            {
+                checkFalse(cd.type.isCollection(), "Secondary indexes are not supported on collections containing durations");
+                checkFalse(cd.type.isTuple(), "Secondary indexes are not supported on tuples containing durations");
+                checkFalse(cd.type.isUDT(), "Secondary indexes are not supported on UDTs containing durations");
+                throw invalidRequest("Secondary indexes are not supported on duration columns");
+            }
+
             // TODO: we could lift that limitation
             if (cfm.isCompactTable() && cd.isPrimaryKeyColumn())
                 throw new InvalidRequestException("Secondary indexes are not supported on PRIMARY KEY columns in COMPACT STORAGE tables");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6487876d/src/java/org/apache/cassandra/db/marshal/AbstractType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractType.java b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
index 37a1959..6176494 100644
--- a/src/java/org/apache/cassandra/db/marshal/AbstractType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
@@ -319,6 +319,11 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>, Assignm
         return false;
     }
 
+    public boolean isTuple()
+    {
+        return false;
+    }
+
     public boolean isMultiCell()
     {
         return false;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6487876d/src/java/org/apache/cassandra/db/marshal/DurationType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/DurationType.java b/src/java/org/apache/cassandra/db/marshal/DurationType.java
index e29265a..63e634c 100644
--- a/src/java/org/apache/cassandra/db/marshal/DurationType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DurationType.java
@@ -28,6 +28,7 @@ import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FastByteOperations;
 
 /**
  * Represents a duration. The duration is stored as  months, days, and nanoseconds. This is done
@@ -39,7 +40,7 @@ public class DurationType extends AbstractType<Duration>
 
     DurationType()
     {
-        super(ComparisonType.NOT_COMPARABLE);
+        super(ComparisonType.BYTE_ORDER);
     } // singleton
 
     public ByteBuffer fromString(String source) throws MarshalException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6487876d/src/java/org/apache/cassandra/db/marshal/TupleType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/TupleType.java b/src/java/org/apache/cassandra/db/marshal/TupleType.java
index 60a63aa..5a90ae9 100644
--- a/src/java/org/apache/cassandra/db/marshal/TupleType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TupleType.java
@@ -412,6 +412,12 @@ public class TupleType extends AbstractType<ByteBuffer>
     }
 
     @Override
+    public boolean isTuple()
+    {
+        return true;
+    }
+
+    @Override
     public CQL3Type asCQL3Type()
     {
         return CQL3Type.Tuple.create(this);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6487876d/src/java/org/apache/cassandra/db/marshal/UserType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/UserType.java b/src/java/org/apache/cassandra/db/marshal/UserType.java
index 475c01d..a59000a 100644
--- a/src/java/org/apache/cassandra/db/marshal/UserType.java
+++ b/src/java/org/apache/cassandra/db/marshal/UserType.java
@@ -407,6 +407,12 @@ public class UserType extends TupleType
     }
 
     @Override
+    public boolean isTuple()
+    {
+        return false;
+    }
+
+    @Override
     public String toString(boolean ignoreFreezing)
     {
         boolean includeFrozenType = !ignoreFreezing && !isMultiCell();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6487876d/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
index 8a8bdcc..c03b0cc 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
@@ -30,6 +30,7 @@ import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.Duration;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
 import org.apache.cassandra.cql3.statements.IndexTarget;
@@ -1343,6 +1344,32 @@ public class SecondaryIndexTest extends CQLTester
                    row(1, 1, 9, 1));
     }
 
+    @Test
+    public void testIndexOnDurationColumn() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, d duration)");
+        assertInvalidMessage("Secondary indexes are not supported on duration columns",
+                             "CREATE INDEX ON %s (d)");
+
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, l list<duration>)");
+        assertInvalidMessage("Secondary indexes are not supported on collections containing durations",
+                             "CREATE INDEX ON %s (l)");
+
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, m map<int, duration>)");
+        assertInvalidMessage("Secondary indexes are not supported on collections containing durations",
+                             "CREATE INDEX ON %s (m)");
+
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, t tuple<int, duration>)");
+        assertInvalidMessage("Secondary indexes are not supported on tuples containing durations",
+                             "CREATE INDEX ON %s (t)");
+
+        String udt = createType("CREATE TYPE %s (i int, d duration)");
+
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, t " + udt + ")");
+        assertInvalidMessage("Secondary indexes are not supported on UDTs containing durations",
+                             "CREATE INDEX ON %s (t)");
+    }
+
     private ResultMessage.Prepared prepareStatement(String cql, boolean forThrift)
     {
         return QueryProcessor.prepare(String.format(cql, KEYSPACE, currentTable()),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6487876d/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
index e60bf36..bb6ead9 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
@@ -151,10 +151,10 @@ public class CreateTest extends CQLTester
                    row(1, 21, Duration.newInstance(12, 10, 0)),
                    row(1, 22, Duration.newInstance(-12, -10, 0)));
 
-        assertInvalidMessage("Slice restriction are not supported on duration columns",
+        assertInvalidMessage("Slice restrictions are not supported on duration columns",
                              "SELECT * FROM %s WHERE c > 1y ALLOW FILTERING");
 
-        assertInvalidMessage("Slice restriction are not supported on duration columns",
+        assertInvalidMessage("Slice restrictions are not supported on duration columns",
                              "SELECT * FROM %s WHERE c <= 1y ALLOW FILTERING");
 
         assertInvalidMessage("Expected at least 3 bytes for a duration (1)",

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6487876d/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
index edbc818..596ef62 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
@@ -25,6 +25,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.config.SchemaConstants;
 import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.Duration;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.schema.SchemaKeyspace;
@@ -2060,6 +2061,99 @@ public class InsertUpdateIfConditionTest extends CQLTester
     }
 
     @Test
+    public void testConditionalOnDurationColumns() throws Throwable
+    {
+        createTable(" CREATE TABLE %s (k int PRIMARY KEY, v int, d duration)");
+
+        assertInvalidMessage("Slice conditions are not supported on durations",
+                             "UPDATE %s SET v = 3 WHERE k = 0 IF d > 1s");
+        assertInvalidMessage("Slice conditions are not supported on durations",
+                             "UPDATE %s SET v = 3 WHERE k = 0 IF d >= 1s");
+        assertInvalidMessage("Slice conditions are not supported on durations",
+                             "UPDATE %s SET v = 3 WHERE k = 0 IF d <= 1s");
+        assertInvalidMessage("Slice conditions are not supported on durations",
+                             "UPDATE %s SET v = 3 WHERE k = 0 IF d < 1s");
+
+        execute("INSERT INTO %s (k, v, d) VALUES (1, 1, 2s)");
+
+        assertRows(execute("UPDATE %s SET v = 4 WHERE k = 1 IF d = 1s"), row(false, Duration.from("2s")));
+        assertRows(execute("UPDATE %s SET v = 3 WHERE k = 1 IF d = 2s"), row(true));
+
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, Duration.from("2s"), 3));
+
+        assertRows(execute("UPDATE %s SET d = 10s WHERE k = 1 IF d != 2s"), row(false, Duration.from("2s")));
+        assertRows(execute("UPDATE %s SET v = 6 WHERE k = 1 IF d != 1s"), row(true));
+
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, Duration.from("2s"), 6));
+
+        assertRows(execute("UPDATE %s SET v = 5 WHERE k = 1 IF d IN (1s, 5s)"), row(false, Duration.from("2s")));
+        assertRows(execute("UPDATE %s SET d = 10s WHERE k = 1 IF d IN (1s, 2s)"), row(true));
+
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, Duration.from("10s"), 6));
+    }
+
+    @Test
+    public void testConditionalOnDurationWithinLists() throws Throwable
+    {
+        for (Boolean frozen : new Boolean[]{Boolean.FALSE, Boolean.TRUE})
+        {
+            String listType = String.format(frozen ? "frozen<%s>" : "%s", "list<duration>");
+
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, v int, l " + listType + " )");
+
+            assertInvalidMessage("Slice conditions are not supported on collections containing durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF l > [1s, 2s]");
+            assertInvalidMessage("Slice conditions are not supported on collections containing durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF l >= [1s, 2s]");
+            assertInvalidMessage("Slice conditions are not supported on collections containing durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF l <= [1s, 2s]");
+            assertInvalidMessage("Slice conditions are not supported on collections containing durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF l < [1s, 2s]");
+
+            execute("INSERT INTO %s (k, v, l) VALUES (1, 1, [1s, 2s])");
+
+            assertRows(execute("UPDATE %s SET v = 4 WHERE k = 1 IF l = [2s]"), row(false, list(Duration.from("1000ms"), Duration.from("2s"))));
+            assertRows(execute("UPDATE %s SET v = 3 WHERE k = 1 IF l = [1s, 2s]"), row(true));
+
+            assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, list(Duration.from("1000ms"), Duration.from("2s")), 3));
+
+            assertRows(execute("UPDATE %s SET l = [10s] WHERE k = 1 IF l != [1s, 2s]"), row(false, list(Duration.from("1000ms"), Duration.from("2s"))));
+            assertRows(execute("UPDATE %s SET v = 6 WHERE k = 1 IF l != [1s]"), row(true));
+
+            assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, list(Duration.from("1000ms"), Duration.from("2s")), 6));
+
+            assertRows(execute("UPDATE %s SET v = 5 WHERE k = 1 IF l IN ([1s], [1s, 5s])"), row(false, list(Duration.from("1000ms"), Duration.from("2s"))));
+            assertRows(execute("UPDATE %s SET l = [5s, 10s] WHERE k = 1 IF l IN ([1s], [1s, 2s])"), row(true));
+
+            assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, list(Duration.from("5s"), Duration.from("10s")), 6));
+
+            assertInvalidMessage("Slice conditions are not supported on durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF l[0] > 1s");
+            assertInvalidMessage("Slice conditions are not supported on durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF l[0] >= 1s");
+            assertInvalidMessage("Slice conditions are not supported on durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF l[0] <= 1s");
+            assertInvalidMessage("Slice conditions are not supported on durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF l[0] < 1s");
+
+            assertRows(execute("UPDATE %s SET v = 4 WHERE k = 1 IF l[0] = 2s"), row(false, list(Duration.from("5s"), Duration.from("10s"))));
+            assertRows(execute("UPDATE %s SET v = 3 WHERE k = 1 IF l[0] = 5s"), row(true));
+
+            assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, list(Duration.from("5s"), Duration.from("10s")), 3));
+
+            assertRows(execute("UPDATE %s SET l = [10s] WHERE k = 1 IF l[1] != 10s"), row(false, list(Duration.from("5s"), Duration.from("10s"))));
+            assertRows(execute("UPDATE %s SET v = 6 WHERE k = 1 IF l[1] != 1s"), row(true));
+
+            assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, list(Duration.from("5s"), Duration.from("10s")), 6));
+
+            assertRows(execute("UPDATE %s SET v = 5 WHERE k = 1 IF l[0] IN (2s, 10s)"), row(false, list(Duration.from("5s"), Duration.from("10s"))));
+            assertRows(execute("UPDATE %s SET l = [6s, 12s] WHERE k = 1 IF l[0] IN (5s, 10s)"), row(true));
+
+            assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, list(Duration.from("6s"), Duration.from("12s")), 6));
+        }
+    }
+
+    @Test
     public void testInMarkerWithMaps() throws Throwable
     {
         for (boolean frozen : new boolean[] {false, true})
@@ -2115,4 +2209,160 @@ public class InsertUpdateIfConditionTest extends CQLTester
                                  "UPDATE %s SET  m = {'foo' : 'foobar'} WHERE k = 0 IF m[?] IN ?", "foo", unset());
         }
     }
+
+    @Test
+    public void testConditionalOnDurationWithinMaps() throws Throwable
+    {
+        for (Boolean frozen : new Boolean[]{Boolean.FALSE, Boolean.TRUE})
+        {
+            String mapType = String.format(frozen ? "frozen<%s>" : "%s", "map<int, duration>");
+
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, v int, m " + mapType + " )");
+
+            assertInvalidMessage("Slice conditions are not supported on collections containing durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF m > {1: 1s, 2: 2s}");
+            assertInvalidMessage("Slice conditions are not supported on collections containing durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF m >= {1: 1s, 2: 2s}");
+            assertInvalidMessage("Slice conditions are not supported on collections containing durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF m <= {1: 1s, 2: 2s}");
+            assertInvalidMessage("Slice conditions are not supported on collections containing durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF m < {1: 1s, 2: 2s}");
+
+            execute("INSERT INTO %s (k, v, m) VALUES (1, 1, {1: 1s, 2: 2s})");
+
+            assertRows(execute("UPDATE %s SET v = 4 WHERE k = 1 IF m = {2: 2s}"), row(false, map(1, Duration.from("1000ms"), 2, Duration.from("2s"))));
+            assertRows(execute("UPDATE %s SET v = 3 WHERE k = 1 IF m = {1: 1s, 2: 2s}"), row(true));
+
+            assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, map(1, Duration.from("1000ms"), 2, Duration.from("2s")), 3));
+
+            assertRows(execute("UPDATE %s SET m = {1 :10s} WHERE k = 1 IF m != {1: 1s, 2: 2s}"), row(false, map(1, Duration.from("1000ms"), 2, Duration.from("2s"))));
+            assertRows(execute("UPDATE %s SET v = 6 WHERE k = 1 IF m != {1: 1s}"), row(true));
+
+            assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, map(1, Duration.from("1000ms"), 2, Duration.from("2s")), 6));
+
+            assertRows(execute("UPDATE %s SET v = 5 WHERE k = 1 IF m IN ({1: 1s}, {1: 5s})"), row(false, map(1, Duration.from("1000ms"), 2, Duration.from("2s"))));
+            assertRows(execute("UPDATE %s SET m = {1: 5s, 2: 10s} WHERE k = 1 IF m IN ({1: 1s}, {1: 1s, 2: 2s})"), row(true));
+
+            assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, map(1, Duration.from("5s"), 2, Duration.from("10s")), 6));
+
+            assertInvalidMessage("Slice conditions are not supported on durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF m[1] > 1s");
+            assertInvalidMessage("Slice conditions are not supported on durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF m[1] >= 1s");
+            assertInvalidMessage("Slice conditions are not supported on durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF m[1] <= 1s");
+            assertInvalidMessage("Slice conditions are not supported on durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF m[1] < 1s");
+
+            assertRows(execute("UPDATE %s SET v = 4 WHERE k = 1 IF m[1] = 2s"), row(false, map(1, Duration.from("5s"), 2, Duration.from("10s"))));
+            assertRows(execute("UPDATE %s SET v = 3 WHERE k = 1 IF m[1] = 5s"), row(true));
+
+            assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, map(1, Duration.from("5s"), 2, Duration.from("10s")), 3));
+
+            assertRows(execute("UPDATE %s SET m = {1: 10s} WHERE k = 1 IF m[2] != 10s"), row(false, map(1, Duration.from("5s"), 2, Duration.from("10s"))));
+            assertRows(execute("UPDATE %s SET v = 6 WHERE k = 1 IF m[2] != 1s"), row(true));
+
+            assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, map(1, Duration.from("5s"), 2, Duration.from("10s")), 6));
+
+            assertRows(execute("UPDATE %s SET v = 5 WHERE k = 1 IF m[1] IN (2s, 10s)"), row(false, map(1, Duration.from("5s"), 2, Duration.from("10s"))));
+            assertRows(execute("UPDATE %s SET m = {1: 6s, 2: 12s} WHERE k = 1 IF m[1] IN (5s, 10s)"), row(true));
+
+            assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, map(1, Duration.from("6s"), 2, Duration.from("12s")), 6));
+        }
+    }
+
+    @Test
+    public void testConditionalOnDurationWithinUdts() throws Throwable
+    {
+        String udt = createType("CREATE TYPE %s (i int, d duration)");
+
+        for (Boolean frozen : new Boolean[]{Boolean.FALSE, Boolean.TRUE})
+        {
+            udt = String.format(frozen ? "frozen<%s>" : "%s", udt);
+
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, v int, u " + udt + " )");
+
+            assertInvalidMessage("Slice conditions are not supported on UDTs containing durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF u > {i: 1, d: 2s}");
+            assertInvalidMessage("Slice conditions are not supported on UDTs containing durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF u >= {i: 1, d: 2s}");
+            assertInvalidMessage("Slice conditions are not supported on UDTs containing durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF u <= {i: 1, d: 2s}");
+            assertInvalidMessage("Slice conditions are not supported on UDTs containing durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF u < {i: 1, d: 2s}");
+
+            execute("INSERT INTO %s (k, v, u) VALUES (1, 1, {i:1, d:2s})");
+
+            assertRows(execute("UPDATE %s SET v = 4 WHERE k = 1 IF u = {i: 2, d: 2s}"), row(false, userType("i", 1, "d", Duration.from("2s"))));
+            assertRows(execute("UPDATE %s SET v = 3 WHERE k = 1 IF u = {i: 1, d: 2s}"), row(true));
+
+            assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, userType("i", 1, "d", Duration.from("2s")), 3));
+
+            assertRows(execute("UPDATE %s SET u = {i: 1, d: 10s} WHERE k = 1 IF u != {i: 1, d: 2s}"), row(false, userType("i", 1, "d", Duration.from("2s"))));
+            assertRows(execute("UPDATE %s SET v = 6 WHERE k = 1 IF u != {i: 1, d: 1s}"), row(true));
+
+            assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, userType("i", 1, "d", Duration.from("2s")), 6));
+
+            assertRows(execute("UPDATE %s SET v = 5 WHERE k = 1 IF u IN ({i: 1, d: 1s}, {i: 1, d: 5s})"), row(false, userType("i", 1, "d", Duration.from("2s"))));
+            assertRows(execute("UPDATE %s SET u = {i: 1, d: 10s} WHERE k = 1 IF u IN ({i: 1, d: 1s}, {i: 1, d: 2s})"), row(true));
+
+            assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, userType("i", 1, "d", Duration.from("10s")), 6));
+
+            assertInvalidMessage("Slice conditions are not supported on durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF u.d > 1s");
+            assertInvalidMessage("Slice conditions are not supported on durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF u.d >= 1s");
+            assertInvalidMessage("Slice conditions are not supported on durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF u.d <= 1s");
+            assertInvalidMessage("Slice conditions are not supported on durations",
+                                 "UPDATE %s SET v = 3 WHERE k = 0 IF u.d < 1s");
+
+            assertRows(execute("UPDATE %s SET v = 4 WHERE k = 1 IF u.d = 2s"), row(false, userType("i", 1, "d", Duration.from("10s"))));
+            assertRows(execute("UPDATE %s SET v = 3 WHERE k = 1 IF u.d = 10s"), row(true));
+
+            assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, userType("i", 1, "d", Duration.from("10s")), 3));
+
+            assertRows(execute("UPDATE %s SET u = {i: 1, d: 10s} WHERE k = 1 IF u.d != 10s"), row(false, userType("i", 1, "d", Duration.from("10s"))));
+            assertRows(execute("UPDATE %s SET v = 6 WHERE k = 1 IF u.d != 1s"), row(true));
+
+            assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, userType("i", 1, "d", Duration.from("10s")), 6));
+
+            assertRows(execute("UPDATE %s SET v = 5 WHERE k = 1 IF u.d IN (2s, 5s)"), row(false, userType("i", 1, "d", Duration.from("10s"))));
+            assertRows(execute("UPDATE %s SET u = {i: 6, d: 12s} WHERE k = 1 IF u.d IN (5s, 10s)"), row(true));
+
+            assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, userType("i", 6, "d", Duration.from("12s")), 6));
+        }
+    }
+
+    @Test
+    public void testConditionalOnDurationWithinTuples() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v int, u tuple<int, duration> )");
+
+        assertInvalidMessage("Slice conditions are not supported on tuples containing durations",
+                             "UPDATE %s SET v = 3 WHERE k = 0 IF u > (1, 2s)");
+        assertInvalidMessage("Slice conditions are not supported on tuples containing durations",
+                             "UPDATE %s SET v = 3 WHERE k = 0 IF u >= (1, 2s)");
+        assertInvalidMessage("Slice conditions are not supported on tuples containing durations",
+                             "UPDATE %s SET v = 3 WHERE k = 0 IF u <= (1, 2s)");
+        assertInvalidMessage("Slice conditions are not supported on tuples containing durations",
+                             "UPDATE %s SET v = 3 WHERE k = 0 IF u < (1, 2s)");
+
+        execute("INSERT INTO %s (k, v, u) VALUES (1, 1, (1, 2s))");
+
+        assertRows(execute("UPDATE %s SET v = 4 WHERE k = 1 IF u = (2, 2s)"), row(false, tuple(1, Duration.from("2s"))));
+        assertRows(execute("UPDATE %s SET v = 3 WHERE k = 1 IF u = (1, 2s)"), row(true));
+
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, tuple(1, Duration.from("2s")), 3));
+
+        assertRows(execute("UPDATE %s SET u = (1, 10s) WHERE k = 1 IF u != (1, 2s)"), row(false, tuple(1, Duration.from("2s"))));
+        assertRows(execute("UPDATE %s SET v = 6 WHERE k = 1 IF u != (1, 1s)"), row(true));
+
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, tuple(1, Duration.from("2s")), 6));
+
+        assertRows(execute("UPDATE %s SET v = 5 WHERE k = 1 IF u IN ((1, 1s), (1, 5s))"), row(false, tuple(1, Duration.from("2s"))));
+        assertRows(execute("UPDATE %s SET u = (1, 10s) WHERE k = 1 IF u IN ((1, 1s), (1, 2s))"), row(true));
+
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, tuple(1, Duration.from("10s")), 6));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6487876d/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
index f167955..f1fcfc3 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
@@ -24,6 +24,7 @@ import org.junit.Test;
 import org.junit.Assert;
 
 import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.Duration;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
 import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -4478,4 +4479,168 @@ public class SelectTest extends CQLTester
             });
         }
     }
+
+    @Test
+    public void testFilteringOnDurationColumn() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, d duration)");
+        execute("INSERT INTO %s (k, d) VALUES (0, 1s)");
+        execute("INSERT INTO %s (k, d) VALUES (1, 2s)");
+        execute("INSERT INTO %s (k, d) VALUES (2, 1s)");
+
+        assertRows(execute("SELECT * FROM %s WHERE d=1s ALLOW FILTERING"),
+                   row(0, Duration.from("1s")),
+                   row(2, Duration.from("1s")));
+
+        assertInvalidMessage("IN predicates on non-primary-key columns (d) is not yet supported",
+                             "SELECT * FROM %s WHERE d IN (1s, 2s) ALLOW FILTERING");
+
+        assertInvalidMessage("Slice restrictions are not supported on duration columns",
+                             "SELECT * FROM %s WHERE d > 1s ALLOW FILTERING");
+
+        assertInvalidMessage("Slice restrictions are not supported on duration columns",
+                             "SELECT * FROM %s WHERE d >= 1s ALLOW FILTERING");
+
+        assertInvalidMessage("Slice restrictions are not supported on duration columns",
+                             "SELECT * FROM %s WHERE d <= 1s ALLOW FILTERING");
+
+        assertInvalidMessage("Slice restrictions are not supported on duration columns",
+                             "SELECT * FROM %s WHERE d < 1s ALLOW FILTERING");
+    }
+
+    @Test
+    public void testFilteringOnListContainingDurations() throws Throwable
+    {
+        for (Boolean frozen : new Boolean[]{Boolean.FALSE, Boolean.TRUE})
+        {
+            String listType = String.format(frozen ? "frozen<%s>" : "%s", "list<duration>");
+
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, l " + listType + ")");
+            execute("INSERT INTO %s (k, l) VALUES (0, [1s, 2s])");
+            execute("INSERT INTO %s (k, l) VALUES (1, [2s, 3s])");
+            execute("INSERT INTO %s (k, l) VALUES (2, [1s, 3s])");
+
+            if (frozen)
+                assertRows(execute("SELECT * FROM %s WHERE l = [1s, 2s] ALLOW FILTERING"),
+                           row(0, list(Duration.from("1s"), Duration.from("2s"))));
+
+            assertInvalidMessage("IN predicates on non-primary-key columns (l) is not yet supported",
+                                 "SELECT * FROM %s WHERE l IN ([1s, 2s], [2s, 3s]) ALLOW FILTERING");
+
+            assertInvalidMessage("Slice restrictions are not supported on collections containing durations",
+                                 "SELECT * FROM %s WHERE l > [2s, 3s] ALLOW FILTERING");
+
+            assertInvalidMessage("Slice restrictions are not supported on collections containing durations",
+                                 "SELECT * FROM %s WHERE l >= [2s, 3s] ALLOW FILTERING");
+
+            assertInvalidMessage("Slice restrictions are not supported on collections containing durations",
+                                 "SELECT * FROM %s WHERE l <= [2s, 3s] ALLOW FILTERING");
+
+            assertInvalidMessage("Slice restrictions are not supported on collections containing durations",
+                                 "SELECT * FROM %s WHERE l < [2s, 3s] ALLOW FILTERING");
+
+            assertRows(execute("SELECT * FROM %s WHERE l CONTAINS 1s ALLOW FILTERING"),
+                       row(0, list(Duration.from("1s"), Duration.from("2s"))),
+                       row(2, list(Duration.from("1s"), Duration.from("3s"))));
+        }
+    }
+
+    @Test
+    public void testFilteringOnMapContainingDurations() throws Throwable
+    {
+        for (Boolean frozen : new Boolean[]{Boolean.FALSE, Boolean.TRUE})
+        {
+            String mapType = String.format(frozen ? "frozen<%s>" : "%s", "map<int, duration>");
+
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, m " + mapType + ")");
+            execute("INSERT INTO %s (k, m) VALUES (0, {1:1s, 2:2s})");
+            execute("INSERT INTO %s (k, m) VALUES (1, {2:2s, 3:3s})");
+            execute("INSERT INTO %s (k, m) VALUES (2, {1:1s, 3:3s})");
+
+            if (frozen)
+                assertRows(execute("SELECT * FROM %s WHERE m = {1:1s, 2:2s} ALLOW FILTERING"),
+                           row(0, map(1, Duration.from("1s"), 2, Duration.from("2s"))));
+
+            assertInvalidMessage("IN predicates on non-primary-key columns (m) is not yet supported",
+                    "SELECT * FROM %s WHERE m IN ({1:1s, 2:2s}, {1:1s, 3:3s}) ALLOW FILTERING");
+
+            assertInvalidMessage("Slice restrictions are not supported on collections containing durations",
+                    "SELECT * FROM %s WHERE m > {1:1s, 3:3s} ALLOW FILTERING");
+
+            assertInvalidMessage("Slice restrictions are not supported on collections containing durations",
+                    "SELECT * FROM %s WHERE m >= {1:1s, 3:3s} ALLOW FILTERING");
+
+            assertInvalidMessage("Slice restrictions are not supported on collections containing durations",
+                    "SELECT * FROM %s WHERE m <= {1:1s, 3:3s} ALLOW FILTERING");
+
+            assertInvalidMessage("Slice restrictions are not supported on collections containing durations",
+                    "SELECT * FROM %s WHERE m < {1:1s, 3:3s} ALLOW FILTERING");
+
+            assertRows(execute("SELECT * FROM %s WHERE m CONTAINS 1s ALLOW FILTERING"),
+                       row(0, map(1, Duration.from("1s"), 2, Duration.from("2s"))),
+                       row(2, map(1, Duration.from("1s"), 3, Duration.from("3s"))));
+        }
+    }
+
+    @Test
+    public void testFilteringOnTupleContainingDurations() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, t tuple<int, duration>)");
+        execute("INSERT INTO %s (k, t) VALUES (0, (1, 2s))");
+        execute("INSERT INTO %s (k, t) VALUES (1, (2, 3s))");
+        execute("INSERT INTO %s (k, t) VALUES (2, (1, 3s))");
+
+        assertRows(execute("SELECT * FROM %s WHERE t = (1, 2s) ALLOW FILTERING"),
+                   row(0, tuple(1, Duration.from("2s"))));
+
+        assertInvalidMessage("IN predicates on non-primary-key columns (t) is not yet supported",
+                "SELECT * FROM %s WHERE t IN ((1, 2s), (1, 3s)) ALLOW FILTERING");
+
+        assertInvalidMessage("Slice restrictions are not supported on tuples containing durations",
+                "SELECT * FROM %s WHERE t > (1, 2s) ALLOW FILTERING");
+
+        assertInvalidMessage("Slice restrictions are not supported on tuples containing durations",
+                "SELECT * FROM %s WHERE t >= (1, 2s) ALLOW FILTERING");
+
+        assertInvalidMessage("Slice restrictions are not supported on tuples containing durations",
+                "SELECT * FROM %s WHERE t <= (1, 2s) ALLOW FILTERING");
+
+        assertInvalidMessage("Slice restrictions are not supported on tuples containing durations",
+                "SELECT * FROM %s WHERE t < (1, 2s) ALLOW FILTERING");
+    }
+
+    @Test
+    public void testFilteringOnUdtContainingDurations() throws Throwable
+    {
+        String udt = createType("CREATE TYPE %s (i int, d duration)");
+
+        for (Boolean frozen : new Boolean[]{Boolean.FALSE, Boolean.TRUE})
+        {
+            udt = String.format(frozen ? "frozen<%s>" : "%s", udt);
+
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, u " + udt + ")");
+            execute("INSERT INTO %s (k, u) VALUES (0, {i: 1, d:2s})");
+            execute("INSERT INTO %s (k, u) VALUES (1, {i: 2, d:3s})");
+            execute("INSERT INTO %s (k, u) VALUES (2, {i: 1, d:3s})");
+
+            if (frozen)
+                assertRows(execute("SELECT * FROM %s WHERE u = {i: 1, d:2s} ALLOW FILTERING"),
+                           row(0, userType("i", 1, "d", Duration.from("2s"))));
+
+            assertInvalidMessage("IN predicates on non-primary-key columns (u) is not yet supported",
+                    "SELECT * FROM %s WHERE u IN ({i: 2, d:3s}, {i: 1, d:3s}) ALLOW FILTERING");
+
+            assertInvalidMessage("Slice restrictions are not supported on UDTs containing durations",
+                    "SELECT * FROM %s WHERE u > {i: 1, d:3s} ALLOW FILTERING");
+
+            assertInvalidMessage("Slice restrictions are not supported on UDTs containing durations",
+                    "SELECT * FROM %s WHERE u >= {i: 1, d:3s} ALLOW FILTERING");
+
+            assertInvalidMessage("Slice restrictions are not supported on UDTs containing durations",
+                    "SELECT * FROM %s WHERE u <= {i: 1, d:3s} ALLOW FILTERING");
+
+            assertInvalidMessage("Slice restrictions are not supported on UDTs containing durations",
+                    "SELECT * FROM %s WHERE u < {i: 1, d:3s} ALLOW FILTERING");
+        }
+    }
 }