This is an automated email from the ASF dual-hosted git repository.

marcuse pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 5e003af516 Fix token restrictions with MIN_TOKEN
5e003af516 is described below

commit 5e003af5167661f3426de5701fd3607b28b7eee5
Author: Marcus Eriksson <marc...@apache.org>
AuthorDate: Tue Apr 15 14:02:44 2025 +0200

    Fix token restrictions with MIN_TOKEN
    
    Patch by marcuse and David Capwell; reviewed by David Capwell for 
CASSANDRA-20557
---
 CHANGES.txt                                        |  1 +
 src/java/org/apache/cassandra/cql3/Operator.java   | 41 ++++++++-------
 .../restrictions/ClusteringColumnRestrictions.java | 13 +++--
 .../cql3/restrictions/ClusteringElements.java      | 59 +++++++++++++---------
 .../cql3/restrictions/MergedRestriction.java       |  5 +-
 .../restrictions/PartitionKeyRestrictions.java     |  2 +-
 .../cql3/restrictions/SimpleRestriction.java       |  7 +--
 .../cql3/restrictions/SingleRestriction.java       |  4 +-
 .../test/cql3/SingleNodeTableWalkTest.java         | 18 +++----
 .../test/cql3/SingleNodeTokenConflictTest.java     |  1 +
 .../distributed/test/cql3/StatefulASTBase.java     | 42 +++++++++++++++
 .../cassandra/harry/model/ASTSingleTableModel.java | 45 ++++++++++++-----
 .../org/apache/cassandra/cql3/SimpleQueryTest.java | 14 ++++-
 .../cql3/restrictions/ClusteringElementsTest.java  | 27 +++++-----
 14 files changed, 189 insertions(+), 90 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 2572c1f313..135f59e5e3 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 5.1
+ * Fix token restrictions with MIN_TOKEN (CASSANDRO-20557)
  * Upgrade logback version to 1.5.18 and slf4j dependencies to 2.0.17 
(CASSANDRA-20429)
  * Switch memtable-related off-heap objects to Native Endian and Memory to 
Little Endian (CASSANDRA-20190)
  * Change SSTableSimpleScanner to use SSTableReader#openDataReaderForScan 
(CASSANDRA-20538)
diff --git a/src/java/org/apache/cassandra/cql3/Operator.java 
b/src/java/org/apache/cassandra/cql3/Operator.java
index 201a046b2f..93a81fa2f4 100644
--- a/src/java/org/apache/cassandra/cql3/Operator.java
+++ b/src/java/org/apache/cassandra/cql3/Operator.java
@@ -40,6 +40,7 @@ import org.apache.cassandra.db.marshal.MultiElementType;
 import org.apache.cassandra.db.marshal.SetType;
 import org.apache.cassandra.db.rows.CellPath;
 import org.apache.cassandra.db.rows.ComplexColumnData;
+import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.schema.ColumnMetadata;
@@ -95,12 +96,12 @@ public enum Operator
         }
 
         @Override
-        public void restrict(RangeSet<ClusteringElements> rangeSet, 
List<ClusteringElements> args)
+        public void restrict(RangeSet<ClusteringElements> rangeSet, 
List<ClusteringElements> args, IPartitioner partitioner)
         {
             assert args.size() == 1 : this + " accept only one single value";
             ClusteringElements arg = args.get(0);
-            rangeSet.removeAll(ClusteringElements.lessThan(arg));
-            rangeSet.removeAll(ClusteringElements.greaterThan(arg));
+            rangeSet.removeAll(ClusteringElements.lessThan(arg, partitioner));
+            rangeSet.removeAll(ClusteringElements.greaterThan(arg, 
partitioner));
         }
 
         @Override
@@ -144,10 +145,10 @@ public enum Operator
         }
 
         @Override
-        public void restrict(RangeSet<ClusteringElements> rangeSet, 
List<ClusteringElements> args)
+        public void restrict(RangeSet<ClusteringElements> rangeSet, 
List<ClusteringElements> args, IPartitioner partitioner)
         {
             assert args.size() == 1 : this + " accept only one single value";
-            rangeSet.removeAll(ClusteringElements.atLeast(args.get(0)));
+            rangeSet.removeAll(ClusteringElements.atLeast(args.get(0), 
partitioner));
         }
 
         @Override
@@ -198,10 +199,10 @@ public enum Operator
         }
 
         @Override
-        public void restrict(RangeSet<ClusteringElements> rangeSet, 
List<ClusteringElements> args)
+        public void restrict(RangeSet<ClusteringElements> rangeSet, 
List<ClusteringElements> args, IPartitioner partitioner)
         {
             assert args.size() == 1 : this + " accept only one single value";
-            rangeSet.removeAll(ClusteringElements.greaterThan(args.get(0)));
+            rangeSet.removeAll(ClusteringElements.greaterThan(args.get(0), 
partitioner));
         }
 
         @Override
@@ -252,10 +253,10 @@ public enum Operator
         }
 
         @Override
-        public void restrict(RangeSet<ClusteringElements> rangeSet, 
List<ClusteringElements> args)
+        public void restrict(RangeSet<ClusteringElements> rangeSet, 
List<ClusteringElements> args, IPartitioner partitioner)
         {
             assert args.size() == 1 : this + " accept only one single value";
-            rangeSet.removeAll(ClusteringElements.lessThan(args.get(0)));
+            rangeSet.removeAll(ClusteringElements.lessThan(args.get(0), 
partitioner));
         }
 
         @Override
@@ -305,10 +306,10 @@ public enum Operator
         }
 
         @Override
-        public void restrict(RangeSet<ClusteringElements> rangeSet, 
List<ClusteringElements> args)
+        public void restrict(RangeSet<ClusteringElements> rangeSet, 
List<ClusteringElements> args, IPartitioner partitioner)
         {
             assert args.size() == 1 : this + " accept only one single value";
-            rangeSet.removeAll(ClusteringElements.atMost(args.get(0)));
+            rangeSet.removeAll(ClusteringElements.atMost(args.get(0), 
partitioner));
         }
 
         @Override
@@ -499,7 +500,7 @@ public enum Operator
         }
 
         @Override
-        public void restrict(RangeSet<ClusteringElements> rangeSet, 
List<ClusteringElements> args)
+        public void restrict(RangeSet<ClusteringElements> rangeSet, 
List<ClusteringElements> args, IPartitioner partitioner)
         {
             assert args.size() == 1;
             rangeSet.remove(ClusteringElements.notEqualTo(args.get(0)));
@@ -676,7 +677,7 @@ public enum Operator
         }
 
         @Override
-        public void restrict(RangeSet<ClusteringElements> rangeSet, 
List<ClusteringElements> args)
+        public void restrict(RangeSet<ClusteringElements> rangeSet, 
List<ClusteringElements> args, IPartitioner partitioner)
         {
             for (ClusteringElements clustering : args)
                 rangeSet.remove(ClusteringElements.notEqualTo(clustering));
@@ -805,12 +806,16 @@ public enum Operator
         }
 
         @Override
-        public void restrict(RangeSet<ClusteringElements> rangeSet, 
List<ClusteringElements> args)
+        public void restrict(RangeSet<ClusteringElements> rangeSet, 
List<ClusteringElements> args, IPartitioner partitioner)
         {
             assert args.size() == 2 : this + " accepts exactly two values";
-            args.sort(ClusteringElements.CQL_COMPARATOR);
-            rangeSet.removeAll(ClusteringElements.lessThan(args.get(0)));
-            rangeSet.removeAll(ClusteringElements.greaterThan(args.get(1)));
+            // avoid sorting when working with token restrictions, otherwise 
we can't know the difference between these queries:
+            // select * from x.y where token(id) between 0 and MIN_TOKEN
+            // select * from x.y where token(id) between MIN_TOKEN and 0
+            if (!args.get(0).token)
+                args.sort(ClusteringElements.CQL_COMPARATOR);
+            rangeSet.removeAll(ClusteringElements.lessThan(args.get(0), 
partitioner));
+            rangeSet.removeAll(ClusteringElements.greaterThan(args.get(1), 
partitioner));
         }
 
         @Override
@@ -1074,7 +1079,7 @@ public enum Operator
      * @param rangeSet the range set to restrict
      * @param args the operator arguments
      */
-    public void restrict(RangeSet<ClusteringElements> rangeSet, 
List<ClusteringElements> args)
+    public void restrict(RangeSet<ClusteringElements> rangeSet, 
List<ClusteringElements> args, IPartitioner partitioner)
     {
         throw new UnsupportedOperationException(this + " is not a range 
operator");
     }
diff --git 
a/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java
 
b/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java
index 10b3864aec..dd43bb0023 100644
--- 
a/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java
+++ 
b/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java
@@ -24,6 +24,7 @@ import javax.annotation.Nullable;
 import com.google.common.collect.RangeSet;
 
 import org.apache.cassandra.db.guardrails.Guardrails;
+import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.QueryOptions;
@@ -51,18 +52,22 @@ final class ClusteringColumnRestrictions extends 
RestrictionSetWrapper
      */
     private final boolean allowFiltering;
 
+    private final IPartitioner partitioner;
+
     public ClusteringColumnRestrictions(TableMetadata table, boolean 
allowFiltering)
     {
-        this(table.comparator, RestrictionSet.empty(), allowFiltering);
+        this(table.comparator, RestrictionSet.empty(), allowFiltering, 
table.partitioner);
     }
 
     private ClusteringColumnRestrictions(ClusteringComparator comparator,
                                          RestrictionSet restrictionSet,
-                                         boolean allowFiltering)
+                                         boolean allowFiltering,
+                                         IPartitioner partitioner)
     {
         super(restrictionSet);
         this.comparator = comparator;
         this.allowFiltering = allowFiltering;
+        this.partitioner = partitioner;
     }
 
     public ClusteringColumnRestrictions mergeWith(Restriction restriction, 
@Nullable IndexRegistry indexRegistry) throws InvalidRequestException
@@ -89,7 +94,7 @@ final class ClusteringColumnRestrictions extends 
RestrictionSetWrapper
                                      newRestrictionStart.name);
         }
 
-        return new ClusteringColumnRestrictions(this.comparator, 
newRestrictionSet, allowFiltering);
+        return new ClusteringColumnRestrictions(this.comparator, 
newRestrictionSet, allowFiltering, partitioner);
     }
 
     public NavigableSet<Clustering<?>> valuesAsClustering(QueryOptions 
options, ClientState state) throws InvalidRequestException
@@ -123,7 +128,7 @@ final class ClusteringColumnRestrictions extends 
RestrictionSetWrapper
             if (r.isSlice())
             {
                 RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.all();
-                r.restrict(rangeSet, options);
+                r.restrict(rangeSet, options, partitioner);
                 return builder.extend(rangeSet).buildSlices();
             }
 
diff --git 
a/src/java/org/apache/cassandra/cql3/restrictions/ClusteringElements.java 
b/src/java/org/apache/cassandra/cql3/restrictions/ClusteringElements.java
index f8f04ebb57..104c73d3ef 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/ClusteringElements.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/ClusteringElements.java
@@ -34,6 +34,7 @@ import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.db.BufferClusteringBound;
 import org.apache.cassandra.db.ClusteringBound;
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.schema.ColumnMetadata;
 
 /**
@@ -94,7 +95,7 @@ public class ClusteringElements extends 
ForwardingList<ByteBuffer> implements Co
     /**
      * The empty {@code ClusteringElements} instance used to avoid creating 
unecessary empty instances.
      */
-    private static final ClusteringElements EMPTY = new 
ClusteringElements(ImmutableList.of(), ImmutableList.of());
+    private static final ClusteringElements EMPTY = new 
ClusteringElements(ImmutableList.of(), ImmutableList.of(), false);
 
     /**
      * A range representing all {@code ClusteringElements}.
@@ -112,7 +113,12 @@ public class ClusteringElements extends 
ForwardingList<ByteBuffer> implements Co
      */
     private final ImmutableList<ByteBuffer> values;
 
-    private ClusteringElements(ImmutableList<? extends ColumnSpecification> 
columns, ImmutableList<ByteBuffer> values)
+    /**
+     * We need to special case token restrictions to properly handle MIN_TOKEN
+     */
+    public final boolean token;
+
+    private ClusteringElements(ImmutableList<? extends ColumnSpecification> 
columns, ImmutableList<ByteBuffer> values, boolean token)
     {
         if (columns.size() != values.size())
             throw new IllegalArgumentException("columns and values should have 
the same size");
@@ -121,6 +127,7 @@ public class ClusteringElements extends 
ForwardingList<ByteBuffer> implements Co
 
         this.columns = columns;
         this.values = values;
+        this.token = token;
     }
 
     private static void checkColumnsOrder(ImmutableList<? extends 
ColumnSpecification> columns)
@@ -163,9 +170,9 @@ public class ClusteringElements extends 
ForwardingList<ByteBuffer> implements Co
      * @param value the element value
      * @return a {@code ClusteringElements} with a single element.
      */
-    public static ClusteringElements of(ColumnSpecification column, ByteBuffer 
value)
+    public static ClusteringElements of(ColumnSpecification column, ByteBuffer 
value, boolean onToken)
     {
-        return new ClusteringElements(ImmutableList.of(column), 
ImmutableList.of(value));
+        return new ClusteringElements(ImmutableList.of(column), 
ImmutableList.of(value), onToken);
     }
 
     /**
@@ -176,7 +183,7 @@ public class ClusteringElements extends 
ForwardingList<ByteBuffer> implements Co
      */
     public static ClusteringElements of(List<? extends ColumnSpecification> 
columns, List<ByteBuffer> values)
     {
-        return new ClusteringElements(ImmutableList.copyOf(columns), 
ImmutableList.copyOf(values));
+        return new ClusteringElements(ImmutableList.copyOf(columns), 
ImmutableList.copyOf(values), false);
     }
 
     /**
@@ -200,9 +207,9 @@ public class ClusteringElements extends 
ForwardingList<ByteBuffer> implements Co
         ImmutableList<? extends ColumnSpecification> newColumns = 
concat(columns, suffix.columns);
         ImmutableList<ByteBuffer> newValues = concat(values, suffix.values);
 
-        return suffix instanceof Top ? new Top(newColumns, newValues)
-                                     : suffix instanceof Bottom ? new 
Bottom(newColumns, newValues)
-                                                                : new 
ClusteringElements(newColumns, newValues);
+        return suffix instanceof Top ? new Top(newColumns, newValues, token)
+                                     : suffix instanceof Bottom ? new 
Bottom(newColumns, newValues, token)
+                                                                : new 
ClusteringElements(newColumns, newValues, token);
     }
 
     private void checkSuffix(ClusteringElements suffix)
@@ -245,36 +252,36 @@ public class ClusteringElements extends 
ForwardingList<ByteBuffer> implements Co
      * Returns a {@code RangeSet} that contains all values less than or equal 
to endpoint.
      * @return a {@code RangeSet} that contains all values less than or equal 
to endpoint.
      */
-    public static RangeSet<ClusteringElements> atMost(ClusteringElements 
endpoint)
+    public static RangeSet<ClusteringElements> atMost(ClusteringElements 
endpoint, IPartitioner partitioner)
     {
-        return buildRangeSet(endpoint, true, BoundType.CLOSED);
+        return buildRangeSet(endpoint, true, BoundType.CLOSED, partitioner);
     }
 
     /**
      * Returns a {@code RangeSet} that contains all values less than endpoint.
      * @return a {@code RangeSet} that contains all values less than endpoint.
      */
-    public static RangeSet<ClusteringElements> lessThan(ClusteringElements 
endpoint)
+    public static RangeSet<ClusteringElements> lessThan(ClusteringElements 
endpoint, IPartitioner partitioner)
     {
-        return buildRangeSet(endpoint, true, BoundType.OPEN);
+        return buildRangeSet(endpoint, true, BoundType.OPEN, partitioner);
     }
 
     /**
      * Returns a {@code RangeSet} that contains all values greater or equal to 
endpoint.
      * @return a {@code RangeSet} that contains all values greater or equal to 
endpoint.
      */
-    public static RangeSet<ClusteringElements> atLeast(ClusteringElements 
endpoint)
+    public static RangeSet<ClusteringElements> atLeast(ClusteringElements 
endpoint, IPartitioner partitioner)
     {
-        return buildRangeSet(endpoint, false, BoundType.CLOSED);
+        return buildRangeSet(endpoint, false, BoundType.CLOSED, partitioner);
     }
 
     /**
      * Returns a {@code RangeSet} that contains all values greater than 
endpoint.
      * @return a {@code RangeSet} that contains all values greater than 
endpoint.
      */
-    public static RangeSet<ClusteringElements> greaterThan(ClusteringElements 
endpoint)
+    public static RangeSet<ClusteringElements> greaterThan(ClusteringElements 
endpoint, IPartitioner partitioner)
     {
-        return buildRangeSet(endpoint, false, BoundType.OPEN);
+        return buildRangeSet(endpoint, false, BoundType.OPEN, partitioner);
     }
 
     public static Range<ClusteringElements> notEqualTo(ClusteringElements 
endpoint)
@@ -282,7 +289,7 @@ public class ClusteringElements extends 
ForwardingList<ByteBuffer> implements Co
         return Range.closed(endpoint.bottom(), endpoint.top());
     }
 
-    private static RangeSet<ClusteringElements> 
buildRangeSet(ClusteringElements endpoint, boolean upperBound, BoundType 
boundType)
+    private static RangeSet<ClusteringElements> 
buildRangeSet(ClusteringElements endpoint, boolean upperBound, BoundType 
boundType, IPartitioner partitioner)
     {
         TreeRangeSet<ClusteringElements> rangeSet = TreeRangeSet.create();
         boolean reversed = endpoint.columnType(0).isReversed();
@@ -312,12 +319,16 @@ public class ClusteringElements extends 
ForwardingList<ByteBuffer> implements Co
                 oppositeEndpoint = upperBound ? e.bottom() : e.top();
             }
         }
+        boolean minToken = false;
+        if (endpoint.token && !upperBound)
+            minToken = 
partitioner.getTokenFactory().fromByteArray(endpoint.get(0)).isMinimum();
         // We need to add the last range or the only one if there was no 
change of direction.
         Range<ClusteringElements> range = upperBound ? 
Range.range(oppositeEndpoint,
                                                                    
BoundType.CLOSED,
                                                                    boundType 
== BoundType.OPEN ? endpoint.bottom() : endpoint.top(),
                                                                    boundType)
-                                                     : Range.range(boundType 
== BoundType.OPEN ? endpoint.top() : endpoint.bottom(),
+                                                     : Range.range(minToken ? 
oppositeEndpoint
+                                                                            : 
boundType == BoundType.OPEN ? endpoint.top() : endpoint.bottom(),
                                                                    boundType,
                                                                    
oppositeEndpoint,
                                                                    
BoundType.CLOSED);
@@ -331,7 +342,7 @@ public class ClusteringElements extends 
ForwardingList<ByteBuffer> implements Co
      */
     public ClusteringElements top()
     {
-        return new Top(columns, values);
+        return new Top(columns, values, token);
     }
 
     /**
@@ -340,7 +351,7 @@ public class ClusteringElements extends 
ForwardingList<ByteBuffer> implements Co
      */
     public ClusteringElements bottom()
     {
-        return new Bottom(columns, values);
+        return new Bottom(columns, values, token);
     }
 
     @Override
@@ -472,9 +483,9 @@ public class ClusteringElements extends 
ForwardingList<ByteBuffer> implements Co
      */
     private static class Bottom extends ClusteringElements
     {
-        private Bottom(ImmutableList<? extends ColumnSpecification> columns, 
ImmutableList<ByteBuffer> values)
+        private Bottom(ImmutableList<? extends ColumnSpecification> columns, 
ImmutableList<ByteBuffer> values, boolean token)
         {
-            super(columns, values);
+            super(columns, values, token);
         }
 
         @Override
@@ -491,9 +502,9 @@ public class ClusteringElements extends 
ForwardingList<ByteBuffer> implements Co
      */
     private static class Top extends ClusteringElements
     {
-        private Top(ImmutableList<? extends ColumnSpecification> columns, 
ImmutableList<ByteBuffer> values)
+        private Top(ImmutableList<? extends ColumnSpecification> columns, 
ImmutableList<ByteBuffer> values, boolean token)
         {
-            super(columns, values);
+            super(columns, values, token);
         }
 
         @Override
diff --git 
a/src/java/org/apache/cassandra/cql3/restrictions/MergedRestriction.java 
b/src/java/org/apache/cassandra/cql3/restrictions/MergedRestriction.java
index 7976b78b7c..9296e00aed 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/MergedRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/MergedRestriction.java
@@ -29,6 +29,7 @@ import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.filter.RowFilter;
+import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.index.Index;
 import org.apache.cassandra.index.IndexRegistry;
 import org.apache.cassandra.schema.ColumnMetadata;
@@ -325,11 +326,11 @@ public final class MergedRestriction implements 
SingleRestriction
     }
 
     @Override
-    public void restrict(RangeSet<ClusteringElements> rangeSet, QueryOptions 
options)
+    public void restrict(RangeSet<ClusteringElements> rangeSet, QueryOptions 
options, IPartitioner partitioner)
     {
         for (int i = 0, m = restrictions.size(); i < m; i++)
         {
-            restrictions.get(i).restrict(rangeSet, options);
+            restrictions.get(i).restrict(rangeSet, options, partitioner);
         }
     }
 
diff --git 
a/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeyRestrictions.java 
b/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeyRestrictions.java
index e4df4c7a69..a33c32ae9a 100644
--- 
a/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeyRestrictions.java
+++ 
b/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeyRestrictions.java
@@ -296,7 +296,7 @@ final class PartitionKeyRestrictions extends 
RestrictionSetWrapper
     private RangeSet<Token> toRangeSet(IPartitioner partitioner, 
SingleRestriction slice, QueryOptions options)
     {
         RangeSet<ClusteringElements>  rangeSet = ClusteringElements.all();
-        slice.restrict(rangeSet, options);
+        slice.restrict(rangeSet, options, partitioner);
 
         ImmutableRangeSet.Builder<Token> builder = ImmutableRangeSet.builder();
 
diff --git 
a/src/java/org/apache/cassandra/cql3/restrictions/SimpleRestriction.java 
b/src/java/org/apache/cassandra/cql3/restrictions/SimpleRestriction.java
index 8592fbbb7b..b5bb2f43fa 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/SimpleRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/SimpleRestriction.java
@@ -34,6 +34,7 @@ import org.apache.cassandra.cql3.terms.Term;
 import org.apache.cassandra.cql3.terms.Terms;
 import org.apache.cassandra.db.filter.RowFilter;
 import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.index.Index;
 import org.apache.cassandra.index.IndexRegistry;
 import org.apache.cassandra.schema.ColumnMetadata;
@@ -226,10 +227,10 @@ public final class SimpleRestriction implements 
SingleRestriction
     }
 
     @Override
-    public void restrict(RangeSet<ClusteringElements> rangeSet, QueryOptions 
options)
+    public void restrict(RangeSet<ClusteringElements> rangeSet, QueryOptions 
options, IPartitioner partitioner)
     {
         assert operator.isSlice() || operator == Operator.EQ;
-        operator.restrict(rangeSet, bindAndGetClusteringElements(options));
+        operator.restrict(rangeSet, bindAndGetClusteringElements(options), 
partitioner);
     }
 
     private List<ClusteringElements> bindAndGetClusteringElements(QueryOptions 
options)
@@ -254,7 +255,7 @@ public final class SimpleRestriction implements 
SingleRestriction
 
         List<ClusteringElements> elements = new ArrayList<>(values.size());
         for (int i = 0; i < values.size(); i++)
-            
elements.add(ClusteringElements.of(columnsExpression.columnSpecification(), 
values.get(i)));
+            
elements.add(ClusteringElements.of(columnsExpression.columnSpecification(), 
values.get(i), isOnToken()));
         return elements;
     }
 
diff --git 
a/src/java/org/apache/cassandra/cql3/restrictions/SingleRestriction.java 
b/src/java/org/apache/cassandra/cql3/restrictions/SingleRestriction.java
index e317e8742d..7720fb1bb6 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/SingleRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/SingleRestriction.java
@@ -22,6 +22,7 @@ import java.util.List;
 import com.google.common.collect.RangeSet;
 
 import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.index.Index;
 
 /**
@@ -103,7 +104,8 @@ public interface SingleRestriction extends Restriction
      *
      * @param rangeSet the range set to add to
      * @param options the query options
+     * @param partitioner the partitioner, used to identify MIN_TOKEN when 
using token restrictions
      * @throws UnsupportedOperationException if the operator is not an 
operator selecting ranges of data.
      */
-    void restrict(RangeSet<ClusteringElements> rangeSet, QueryOptions options);
+    void restrict(RangeSet<ClusteringElements> rangeSet, QueryOptions options, 
IPartitioner partitioner);
 }
diff --git 
a/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTableWalkTest.java
 
b/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTableWalkTest.java
index 581b664db5..762a2b83bc 100644
--- 
a/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTableWalkTest.java
+++ 
b/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTableWalkTest.java
@@ -44,7 +44,7 @@ import accord.utils.Property;
 import accord.utils.RandomSource;
 import org.apache.cassandra.cql3.KnownIssue;
 import org.apache.cassandra.cql3.ast.Bind;
-import org.apache.cassandra.cql3.ast.Conditional;
+import org.apache.cassandra.cql3.ast.Conditional.Where.Inequality;
 import org.apache.cassandra.cql3.ast.CreateIndexDDL;
 import org.apache.cassandra.cql3.ast.FunctionCall;
 import org.apache.cassandra.cql3.ast.Mutation;
@@ -170,7 +170,7 @@ public class SingleNodeTableWalkTest extends StatefulASTBase
 
         Select.Builder builder = Select.builder().table(state.metadata);
         
builder.where(FunctionCall.tokenByColumns(state.model.factory.partitionColumns),
-                      Conditional.Where.Inequality.EQUAL,
+                      Inequality.EQUAL,
                       token(state, ref));
 
         Select select = builder.build();
@@ -213,10 +213,10 @@ public class SingleNodeTableWalkTest extends 
StatefulASTBase
         else
         {
             builder.where(pkToken,
-                          startInclusive ? 
Conditional.Where.Inequality.GREATER_THAN_EQ : 
Conditional.Where.Inequality.GREATER_THAN,
+                          startInclusive ? Inequality.GREATER_THAN_EQ : 
Inequality.GREATER_THAN,
                           token(state, start));
             builder.where(pkToken,
-                          endInclusive ? 
Conditional.Where.Inequality.LESS_THAN_EQ : 
Conditional.Where.Inequality.LESS_THAN,
+                          endInclusive ? Inequality.LESS_THAN_EQ : 
Inequality.LESS_THAN,
                           token(state, end));
         }
         Select select = builder.build();
@@ -330,7 +330,7 @@ public class SingleNodeTableWalkTest extends StatefulASTBase
     private Property.Command<State, Void, ?> simpleRangeSearch(RandomSource 
rs, State state, Symbol symbol, ByteBuffer value, Select.Builder builder)
     {
         // do a simple search, like > or <
-        Conditional.Where.Inequality kind = state.rangeInequalityGen.next(rs);
+        Inequality kind = state.rangeInequalityGen.next(rs);
         builder.where(symbol, kind, value);
         Select select = builder.build();
         var indexed = state.indexes.get(symbol);
@@ -368,7 +368,8 @@ public class SingleNodeTableWalkTest extends StatefulASTBase
                                   .add(StatefulASTBase::fullTableScan)
                                   .addIf(State::hasPartitions, 
this::selectExisting)
                                   .addAllIf(State::supportTokens, b -> 
b.add(this::selectToken)
-                                                                        
.add(this::selectTokenRange))
+                                                                        
.add(this::selectTokenRange)
+                                                                        
.add(StatefulASTBase::selectMinTokenRange))
                                   .addIf(State::hasEnoughMemtable, 
StatefulASTBase::flushTable)
                                   .addIf(State::hasEnoughSSTables, 
StatefulASTBase::compactTable)
                                   .addIf(State::allowNonPartitionQuery, 
this::nonPartitionQuery)
@@ -557,11 +558,6 @@ public class SingleNodeTableWalkTest extends 
StatefulASTBase
             return indexed;
         }
 
-        public boolean hasPartitions()
-        {
-            return !model.isEmpty();
-        }
-
         public boolean supportTokens()
         {
             return hasPartitions();
diff --git 
a/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTokenConflictTest.java
 
b/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTokenConflictTest.java
index 1f754f3360..090843c4b3 100644
--- 
a/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTokenConflictTest.java
+++ 
b/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTokenConflictTest.java
@@ -280,6 +280,7 @@ public class SingleNodeTokenConflictTest extends 
StatefulASTBase
                                   
.add(SingleNodeTokenConflictTest::tokenBetween)
                                   .add(SingleNodeTokenConflictTest::tokenRange)
                                   
.add(SingleNodeTokenConflictTest::tokenBoundRange)
+                                  .addIf(State::hasPartitions, 
StatefulASTBase::selectMinTokenRange)
                                   .addIf(State::hasEnoughMemtable, 
StatefulASTBase::flushTable)
                                   .addIf(State::hasEnoughSSTables, 
StatefulASTBase::compactTable)
                                   .destroyState(State::close)
diff --git 
a/test/distributed/org/apache/cassandra/distributed/test/cql3/StatefulASTBase.java
 
b/test/distributed/org/apache/cassandra/distributed/test/cql3/StatefulASTBase.java
index 3a23e1bfcc..a16b47c1d7 100644
--- 
a/test/distributed/org/apache/cassandra/distributed/test/cql3/StatefulASTBase.java
+++ 
b/test/distributed/org/apache/cassandra/distributed/test/cql3/StatefulASTBase.java
@@ -55,6 +55,7 @@ import org.apache.cassandra.cql3.KnownIssue;
 import org.apache.cassandra.cql3.ast.Bind;
 import org.apache.cassandra.cql3.ast.CQLFormatter;
 import org.apache.cassandra.cql3.ast.Conditional;
+import org.apache.cassandra.cql3.ast.FunctionCall;
 import org.apache.cassandra.cql3.ast.Literal;
 import org.apache.cassandra.cql3.ast.Mutation;
 import org.apache.cassandra.cql3.ast.Select;
@@ -238,6 +239,42 @@ public class StatefulASTBase extends TestBaseImpl
         return state.command(rs, select, "full table scan");
     }
 
+    protected static <S extends BaseState> Property.Command<S, Void, ?> 
selectMinTokenRange(RandomSource rs, S state)
+    {
+        var key = rs.pickOrderedSet(state.model.partitionKeys());
+        FunctionCall tokenCall = 
FunctionCall.tokenByColumns(state.model.factory.partitionColumns);
+        Literal min = Literal.of(key.token.getLongValue());
+        Literal max = Literal.of(Long.MIN_VALUE);
+        if (rs.nextBoolean())
+        {
+            Literal tmp = min;
+            min = max;
+            max = tmp;
+        }
+        Select select;
+        if (rs.nextBoolean())
+        {
+            select = Select.builder(state.metadata)
+                           .where(tokenCall, state.greaterThanGen.next(rs), 
min)
+                           .where(tokenCall, state.lessThanGen.next(rs), max)
+                           .build();
+        }
+        else
+        {
+            // it's possible that the range was flipped, which is known bug 
with BETWEEN, so
+            // make sure the range is not flipped until that bug is fixed
+            if 
(IGNORED_ISSUES.contains(KnownIssue.BETWEEN_START_LARGER_THAN_END))
+            {
+                min = Literal.of(key.token.getLongValue());
+                max = Literal.of(Long.MIN_VALUE);
+            }
+            select = Select.builder(state.metadata)
+                           .between(tokenCall, min, max)
+                           .build();
+        }
+        return state.command(rs, select, "min token range");
+    }
+
     protected static abstract class BaseState implements AutoCloseable
     {
         protected final RandomSource rs;
@@ -294,6 +331,11 @@ public class StatefulASTBase extends TestBaseImpl
             createTable(metadata);
         }
 
+        public boolean hasPartitions()
+        {
+            return !model.isEmpty();
+        }
+
         protected boolean readAfterWrite()
         {
             return false;
diff --git 
a/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModel.java 
b/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModel.java
index e1df90e0c2..a781e10e21 100644
--- a/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModel.java
+++ b/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModel.java
@@ -71,6 +71,8 @@ import org.apache.cassandra.db.Clustering;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.BooleanType;
 import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.harry.model.BytesPartitionState.PrimaryKey;
 import org.apache.cassandra.harry.util.StringUtils;
@@ -1436,7 +1438,7 @@ public class ASTSingleTableModel
         NavigableSet<BytesPartitionState.Ref> keys = 
partitions.navigableKeySet();
         // To support the case where 2+ keys share the same token, need to 
create a token ref before and after the token, to make sure
         // the head/tail sets find the matches correctly
-        if (tokenLowerBound != null)
+        if (tokenLowerBound != null && !tokenLowerBound.token.isMinimum())
         {
             boolean inclusive;
             switch (tokenLowerBound.inequality)
@@ -1454,7 +1456,7 @@ public class ASTSingleTableModel
             // when inclusive=false the ref should be after the token, that 
way they are excluded
             keys = keys.tailSet(factory.createRef(tokenLowerBound.token, 
!inclusive), inclusive);
         }
-        if (tokenUpperBound != null)
+        if (tokenUpperBound != null && !tokenUpperBound.token.isMinimum())
         {
             boolean inclusive;
             switch (tokenUpperBound.inequality)
@@ -1600,6 +1602,30 @@ public class ASTSingleTableModel
         return ExpressionEvaluator.evalEncoded(e);
     }
 
+    private BytesPartitionState.Ref processToken(Expression e)
+    {
+        BytesPartitionState.Ref ref;
+        if (e instanceof FunctionCall)
+        {
+            FunctionCall rhs = (FunctionCall) e;
+            List<ByteBuffer> pkValues = 
rhs.arguments.stream().map(ASTSingleTableModel::eval).collect(Collectors.toList());
+            ref = factory.createRef(new 
BufferClustering(pkValues.toArray(ByteBuffer[]::new)));
+        }
+        else if (e instanceof Value)
+        {
+            var value = (Value) e;
+            if (value.type() != LongType.instance)
+                throw new AssertionError("Token values only expected to be 
bigint but given " + value.type().asCQL3Type());
+            var token = new 
Murmur3Partitioner.LongToken(LongType.instance.compose(value.valueEncoded()));
+            ref = factory.createRef(token, true); // should this be false?
+        }
+        else
+        {
+            throw new UnsupportedOperationException(e.getClass().toString());
+        }
+        return ref;
+    }
+
     private static class Row
     {
         private static final Row EMPTY = new Row(ImmutableUniqueList.empty(), 
ByteBufferUtil.EMPTY_ARRAY);
@@ -1727,7 +1753,7 @@ public class ASTSingleTableModel
             if (tokenLowerBound != null && tokenUpperBound != null)
             {
                 int rc = 
tokenLowerBound.token.compareTo(tokenUpperBound.token);
-                if (rc > 0)
+                if (rc > 0 && !tokenUpperBound.token.isMinimum())
                 {
                     // where token > 10 and < 0.... nothing matches that!
                     unmatchable = true;
@@ -1783,9 +1809,7 @@ public class ASTSingleTableModel
                     switch (fn.name())
                     {
                         case "token":
-                            FunctionCall rhs = (FunctionCall) w.rhs;
-                            List<ByteBuffer> pkValues = 
rhs.arguments.stream().map(ASTSingleTableModel::eval).collect(Collectors.toList());
-                            BytesPartitionState.Ref ref = 
factory.createRef(new BufferClustering(pkValues.toArray(ByteBuffer[]::new)));
+                            BytesPartitionState.Ref ref = processToken(w.rhs);
                             switch (w.kind)
                             {
                                 case EQUAL:
@@ -1881,17 +1905,14 @@ public class ASTSingleTableModel
                     {
                         case "token":
                             // if the ref is a token, the only valid start/end 
are also token
-                            List<ByteBuffer> start = ((FunctionCall) 
between.start).arguments.stream().map(ASTSingleTableModel::eval).collect(Collectors.toList());
-                            Token startToken = factory.createRef(new 
BufferClustering(start.toArray(ByteBuffer[]::new))).token;
-
-                            List<ByteBuffer> end = ((FunctionCall) 
between.end).arguments.stream().map(ASTSingleTableModel::eval).collect(Collectors.toList());
-                            Token endToken = factory.createRef(new 
BufferClustering(end.toArray(ByteBuffer[]::new))).token;
+                            Token startToken = 
processToken(between.start).token;
+                            Token endToken = processToken(between.end).token;
 
                             if (startToken.equals(endToken))
                             {
                                 token = startToken;
                             }
-                            else if (startToken.compareTo(endToken) > 0)
+                            else if (startToken.compareTo(endToken) > 0 && 
!endToken.isMinimum())
                             {
                                 // start is larger than end... no matches
                                 unmatchable = true;
diff --git a/test/unit/org/apache/cassandra/cql3/SimpleQueryTest.java 
b/test/unit/org/apache/cassandra/cql3/SimpleQueryTest.java
index 0c89f9b599..fe160f0af6 100644
--- a/test/unit/org/apache/cassandra/cql3/SimpleQueryTest.java
+++ b/test/unit/org/apache/cassandra/cql3/SimpleQueryTest.java
@@ -559,5 +559,17 @@ public class SimpleQueryTest extends CQLTester
         execute("DELETE FROM %s USING TIMESTAMP 6 WHERE k1 = 1");
 
         assertRows(execute("SELECT * FROM %s WHERE k1=1"), row(1, 1, 2));
-    } 
+    }
+
+    @Test
+    public void testTokenRestriction()
+    {
+        createTable("CREATE TABLE %s (id int primary key)");
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s (id) values (?)", i);
+
+        assertRows(execute("SELECT * FROM %s where token(id) > 0 AND token(id) 
< " + Long.MIN_VALUE),  row(7), row(6), row(9), row(3));
+        assertRows(execute("SELECT * FROM %s where token(id) > 0 AND token(id) 
<= " + Long.MIN_VALUE), row(7), row(6), row(9), row(3));
+        assertRows(execute("SELECT * FROM %s where token(id) BETWEEN 0 AND " + 
Long.MIN_VALUE),        row(7), row(6), row(9), row(3));
+    }
 }
diff --git 
a/test/unit/org/apache/cassandra/cql3/restrictions/ClusteringElementsTest.java 
b/test/unit/org/apache/cassandra/cql3/restrictions/ClusteringElementsTest.java
index 60e2508be2..8ad94d4638 100644
--- 
a/test/unit/org/apache/cassandra/cql3/restrictions/ClusteringElementsTest.java
+++ 
b/test/unit/org/apache/cassandra/cql3/restrictions/ClusteringElementsTest.java
@@ -31,6 +31,7 @@ import org.junit.Test;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.db.marshal.ReversedType;
+import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.harry.util.ByteUtils;
 import org.apache.cassandra.schema.ColumnMetadata;
 
@@ -187,7 +188,7 @@ public class ClusteringElementsTest
             ClusteringElements four = elements(type, 4);
             ClusteringElements six = elements(type, 6);
 
-            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.atMost(four);
+            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.atMost(four, Murmur3Partitioner.instance);
             assertTrue(rangeSet.contains(one));
             assertTrue(rangeSet.contains(four));
             assertFalse(rangeSet.contains(six));
@@ -208,7 +209,7 @@ public class ClusteringElementsTest
             ClusteringElements oneFive = elements(columns, 1, 5);
             ClusteringElements twoFive = elements(columns, 2, 5);
 
-            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.atMost(oneThree);
+            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.atMost(oneThree, Murmur3Partitioner.instance);
 
             assertTrue(rangeSet.contains(zeroZero));
             assertTrue(rangeSet.contains(oneZero));
@@ -258,7 +259,7 @@ public class ClusteringElementsTest
             ClusteringElements oneFiveOne = elements(columns, 1, 5, 1);
             ClusteringElements twoFiveFive = elements(columns, 2, 5, 5);
 
-            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.atMost(oneThreeOne);
+            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.atMost(oneThreeOne, Murmur3Partitioner.instance);
 
             assertTrue(rangeSet.contains(zeroZeroZero));
             assertTrue(rangeSet.contains(oneZeroOne));
@@ -279,7 +280,7 @@ public class ClusteringElementsTest
             ClusteringElements four = elements(column, 4);
             ClusteringElements six = elements(column, 6);
 
-            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.lessThan(four);
+            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.lessThan(four, Murmur3Partitioner.instance);
             assertTrue(rangeSet.contains(one));
             assertFalse(rangeSet.contains(four));
             assertFalse(rangeSet.contains(six));
@@ -300,7 +301,7 @@ public class ClusteringElementsTest
             ClusteringElements oneFive = elements(columns, 1, 5);
             ClusteringElements twoFive = elements(columns, 2, 5);
 
-            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.lessThan(oneThree);
+            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.lessThan(oneThree, Murmur3Partitioner.instance);
 
             assertTrue(rangeSet.contains(zeroZero));
             assertTrue(rangeSet.contains(oneZero));
@@ -351,7 +352,7 @@ public class ClusteringElementsTest
             ClusteringElements oneFiveOne = elements(columns, 1, 5, 1);
             ClusteringElements twoFiveFive = elements(columns, 2, 5, 5);
 
-            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.lessThan(oneThreeOne);
+            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.lessThan(oneThreeOne, Murmur3Partitioner.instance);
 
             assertTrue(rangeSet.contains(zeroZeroZero));
             assertTrue(rangeSet.contains(oneZeroOne));
@@ -372,7 +373,7 @@ public class ClusteringElementsTest
             ClusteringElements four = elements(column, 4);
             ClusteringElements six = elements(column, 6);
 
-            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.atLeast(four);
+            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.atLeast(four, Murmur3Partitioner.instance);
             assertFalse(rangeSet.contains(one));
             assertTrue(rangeSet.contains(four));
             assertTrue(rangeSet.contains(six));
@@ -393,7 +394,7 @@ public class ClusteringElementsTest
             ClusteringElements oneFive = elements(columns, 1, 5);
             ClusteringElements twoFive = elements(columns, 2, 5);
 
-            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.atLeast(oneThree);
+            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.atLeast(oneThree, Murmur3Partitioner.instance);
 
             assertFalse(rangeSet.contains(zeroZero));
             assertFalse(rangeSet.contains(oneZero));
@@ -444,7 +445,7 @@ public class ClusteringElementsTest
             ClusteringElements oneFiveOne = elements(columns, 1, 5, 1);
             ClusteringElements twoFiveFive = elements(columns, 2, 5, 5);
 
-            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.atLeast(oneThreeOne);
+            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.atLeast(oneThreeOne, Murmur3Partitioner.instance);
 
             assertFalse(rangeSet.contains(zeroZeroZero));
             assertFalse(rangeSet.contains(oneZeroOne));
@@ -465,7 +466,7 @@ public class ClusteringElementsTest
             ClusteringElements four = elements(column, 4);
             ClusteringElements six = elements(column, 6);
 
-            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.greaterThan(four);
+            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.greaterThan(four, Murmur3Partitioner.instance);
             assertFalse(rangeSet.contains(one));
             assertFalse(rangeSet.contains(four));
             assertTrue(rangeSet.contains(six));
@@ -486,7 +487,7 @@ public class ClusteringElementsTest
             ClusteringElements oneFive = elements(columns, 1, 5);
             ClusteringElements twoFive = elements(columns, 2, 5);
 
-            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.greaterThan(oneThree);
+            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.greaterThan(oneThree, Murmur3Partitioner.instance);
 
             assertFalse(rangeSet.contains(zeroZero));
             assertFalse(rangeSet.contains(oneZero));
@@ -537,7 +538,7 @@ public class ClusteringElementsTest
             ClusteringElements oneFiveOne = elements(columns, 1, 5, 1);
             ClusteringElements twoFiveFive = elements(columns, 2, 5, 5);
 
-            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.greaterThan(oneThreeOne);
+            RangeSet<ClusteringElements> rangeSet = 
ClusteringElements.greaterThan(oneThreeOne, Murmur3Partitioner.instance);
 
             assertFalse(rangeSet.contains(zeroZeroZero));
             assertFalse(rangeSet.contains(oneZeroOne));
@@ -662,7 +663,7 @@ public class ClusteringElementsTest
 
     private static ClusteringElements elements(ColumnMetadata column, int 
value)
     {
-        return ClusteringElements.of(column, bytes(value));
+        return ClusteringElements.of(column, bytes(value), false);
     }
 
     private static ClusteringElements elements(List<ColumnMetadata> columns, 
int... values)


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to