You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2012/01/19 17:06:44 UTC

git commit: Allow rangeSlice queries to be start/end inclusive/exclusive

Updated Branches:
  refs/heads/trunk bce44ff32 -> 35f7bc597


Allow rangeSlice queries to be start/end inclusive/exclusive

patch by slebresne; reviewed by jbellis for CASSANDRA-3749


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

Branch: refs/heads/trunk
Commit: 35f7bc597a8d26ca88526cf0d5208c13bdda523d
Parents: bce44ff
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Jan 19 17:05:45 2012 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Jan 19 17:05:45 2012 +0100

----------------------------------------------------------------------
 CHANGES.txt                                        |    1 +
 .../org/apache/cassandra/db/ColumnFamilyStore.java |   30 ++--
 .../org/apache/cassandra/dht/AbstractBounds.java   |  105 ++----------
 src/java/org/apache/cassandra/dht/Bounds.java      |   15 ++-
 .../org/apache/cassandra/dht/ExcludingBounds.java  |   98 ++++++++++
 .../cassandra/dht/IncludingExcludingBounds.java    |   97 ++++++++++
 src/java/org/apache/cassandra/dht/Range.java       |   91 +++++++++-
 src/java/org/apache/cassandra/dht/Token.java       |   27 ++--
 .../apache/cassandra/io/sstable/SSTableReader.java |    4 +-
 .../org/apache/cassandra/service/StorageProxy.java |   20 +-
 .../apache/cassandra/db/ColumnFamilyStoreTest.java |   70 +++++++-
 .../apache/cassandra/dht/AbstractBoundsTest.java   |  137 ---------------
 test/unit/org/apache/cassandra/dht/RangeTest.java  |   77 ++++++++
 .../apache/cassandra/service/StorageProxyTest.java |   32 ++++-
 14 files changed, 522 insertions(+), 282 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/35f7bc59/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 68147d7..4a4bf35 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -45,6 +45,7 @@
  * Make stress.java insert operation to use microseconds (CASSANDRA-3725)
  * Allows (internally) doing a range query with a limit of columns instead of
    rows (CASSANDRA-3742)
+ * Allow rangeSlice queries to be start/end inclusive/exclusive (CASSANDRA-3749)
 
 
 1.0.7

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35f7bc59/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 805c55c..0238344 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -1218,9 +1218,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      */
     public AbstractScanIterator getSequentialIterator(ByteBuffer superColumn, final AbstractBounds<RowPosition> range, IFilter columnFilter)
     {
-        assert range instanceof Bounds
-               || !((Range)range).isWrapAround() || range.right.isMinimum()
-               : range;
+        assert !(range instanceof Range) || !((Range)range).isWrapAround() || range.right.isMinimum() : range;
 
         final RowPosition startWith = range.left;
         final RowPosition stopAt = range.right;
@@ -1236,8 +1234,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
             return new AbstractScanIterator()
             {
-                boolean first = true;
-
                 protected Row computeNext()
                 {
                     // pull a row out of the iterator
@@ -1250,20 +1246,18 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                     if (!stopAt.isMinimum() && stopAt.compareTo(key) < 0)
                         return endOfData();
 
-                    // skip first one
-                    if (range instanceof Bounds || !first || !key.equals(startWith))
-                    {
-                        if (logger.isDebugEnabled())
-                            logger.debug("scanned " + key);
-                        // TODO this is necessary because when we collate supercolumns together, we don't check
-                        // their subcolumns for relevance, so we need to do a second prune post facto here.
-                        return current.cf != null && current.cf.isSuper()
-                             ? new Row(current.key, removeDeleted(current.cf, gcBefore))
-                             : current;
-                    }
-                    first = false;
+                    // skipping outside of assigned range
+                    if (!range.contains(key))
+                        return computeNext();
+
+                    if (logger.isDebugEnabled())
+                        logger.debug("scanned " + key);
 
-                    return computeNext();
+                    // TODO this is necessary because when we collate supercolumns together, we don't check
+                    // their subcolumns for relevance, so we need to do a second prune post facto here.
+                    return current.cf != null && current.cf.isSuper()
+                        ? new Row(current.key, removeDeleted(current.cf, gcBefore))
+                        : current;
                 }
 
                 public void close() throws IOException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35f7bc59/src/java/org/apache/cassandra/dht/AbstractBounds.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/AbstractBounds.java b/src/java/org/apache/cassandra/dht/AbstractBounds.java
index da218ef..b85fc65 100644
--- a/src/java/org/apache/cassandra/dht/AbstractBounds.java
+++ b/src/java/org/apache/cassandra/dht/AbstractBounds.java
@@ -61,19 +61,20 @@ public abstract class AbstractBounds<T extends RingPosition> implements Serializ
     }
 
     /**
-     * Given token T and AbstractBounds ?L,R], returns Pair(?L,T], ]T,R])
-     * (where ? means that the same type of Bounds is returned -- Range or Bounds -- as the original.)
-     * The original AbstractBounds must contain the token T.
-     * If the split would cause one of the left or right side to be empty, it will be null in the result pair.
+     * Given token T and AbstractBounds ?L,R?, returns Pair(?L,T], (T,R?),
+     * where ? means that the same type of AbstractBounds is returned as the original.
+     *
+     * Put another way, returns a Pair of everything this AbstractBounds contains
+     * up to and including the split position, and everything it contains after
+     * (not including the split position).
+     *
+     * The original AbstractBounds must either contain the position T, or T
+     * should be equals to the left bound L.
+     *
+     * If the split would only yield the same AbstractBound, null is returned
+     * instead.
      */
-    public Pair<AbstractBounds<T>,AbstractBounds<T>> split(T pos)
-    {
-        assert left.equals(pos) || contains(pos);
-        AbstractBounds<T> lb = createFrom(pos);
-        // we contain this token, so only one of the left or right can be empty
-        AbstractBounds<T> rb = lb != null && pos.equals(right) ? null : new Range<T>(pos, right);
-        return new Pair<AbstractBounds<T>,AbstractBounds<T>>(lb, rb);
-    }
+    public abstract Pair<AbstractBounds<T>, AbstractBounds<T>> split(T position);
 
     @Override
     public int hashCode()
@@ -83,89 +84,9 @@ public abstract class AbstractBounds<T extends RingPosition> implements Serializ
 
     public abstract boolean contains(T start);
 
-    /** @return A clone of this AbstractBounds with a new right T, or null if an identical range would be created. */
-    public abstract AbstractBounds<T> createFrom(T right);
-
     public abstract List<? extends AbstractBounds<T>> unwrap();
 
     /**
-     * @return A copy of the given list of with all bounds unwrapped, sorted by bound.left and with overlapping bounds merged.
-     * This method does not allow allow to mix Range and Bound in the input list.
-     */
-    public static <T extends RingPosition> List<AbstractBounds<T>> normalize(Collection<? extends AbstractBounds<T>> bounds)
-    {
-        // unwrap all
-        List<AbstractBounds<T>> output = new ArrayList<AbstractBounds<T>>();
-        for (AbstractBounds<T> bound : bounds)
-            output.addAll(bound.unwrap());
-
-        // sort by left
-        Collections.sort(output, new Comparator<AbstractBounds<T>>()
-        {
-            public int compare(AbstractBounds<T> b1, AbstractBounds<T> b2)
-            {
-                return b1.left.compareTo(b2.left);
-            }
-        });
-
-        // deoverlap
-        return deoverlap(output);
-    }
-
-    /**
-     * Given a list of unwrapped bounds sorted by left token, return a list a equivalent
-     * list of bounds but with no overlapping bounds.
-     */
-    private static <T extends RingPosition> List<AbstractBounds<T>> deoverlap(List<AbstractBounds<T>> bounds)
-    {
-        if (bounds.isEmpty())
-            return bounds;
-
-        List<AbstractBounds<T>> output = new ArrayList<AbstractBounds<T>>();
-
-        Iterator<AbstractBounds<T>> iter = bounds.iterator();
-        AbstractBounds<T> current = iter.next();
-        boolean isRange = current instanceof Range;
-
-        T min = (T) current.partitioner.minValue(current.left.getClass());
-        while (iter.hasNext())
-        {
-            if (current.right.equals(min))
-            {
-                // If one of the bound is the full range, we return only that
-                if (current.left.equals(min))
-                    return Collections.<AbstractBounds<T>>singletonList(current);
-
-                output.add(current.createFrom(min));
-                return output;
-            }
-
-            AbstractBounds<T> next = iter.next();
-            assert isRange ? next instanceof Range : next instanceof Bounds;
-
-            // For Ranges, if next left is equal to current right, we do not intersect per se, but replacing (A, B] and (B, C] by (A, C] is
-            // legit, and since this actually avoid special casing and will result in more "optimal" ranges, we do this transformation
-            if (next.left.compareTo(current.right) <= 0)
-            {
-                // We do overlap
-                // (we've handler current.right.equals(min) already)
-                T newRight = next.right.equals(min) || current.right.compareTo(next.right) < 0 ? next.right : current.right;
-                current = current.createFrom(newRight);
-                if (current == null)
-                    // current is the full ring, can only happen for Range
-                    return Collections.<AbstractBounds<T>>singletonList(new Range<T>(min, min));
-            }
-            else
-            {
-                output.add(current);
-                current = next;
-            }
-        }
-        output.add(current);
-        return output;
-    }
-
-    /**
      * Transform this abstract bounds to equivalent covering bounds of row positions.
      * If this abstract bounds was already an abstractBounds of row positions, this is a noop.
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35f7bc59/src/java/org/apache/cassandra/dht/Bounds.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/Bounds.java b/src/java/org/apache/cassandra/dht/Bounds.java
index f2cbeff..b521178 100644
--- a/src/java/org/apache/cassandra/dht/Bounds.java
+++ b/src/java/org/apache/cassandra/dht/Bounds.java
@@ -26,7 +26,11 @@ import java.util.List;
 
 import org.apache.cassandra.db.RowPosition;
 import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.Pair;
 
+/**
+ * AbstractBounds containing both its endpoints: [left, right].  Used by "classic" by-key range scans.
+ */
 public class Bounds<T extends RingPosition> extends AbstractBounds<T>
 {
     public Bounds(T left, T right)
@@ -46,9 +50,16 @@ public class Bounds<T extends RingPosition> extends AbstractBounds<T>
         return Range.contains(left, right, position) || left.equals(position);
     }
 
-    public AbstractBounds<T> createFrom(T position)
+    public Pair<AbstractBounds<T>, AbstractBounds<T>> split(T position)
     {
-        return new Bounds<T>(left, position, partitioner);
+        assert contains(position);
+        // Check if the split would have no effect on the range
+        if (position.equals(right))
+            return null;
+
+        AbstractBounds<T> lb = new Bounds<T>(left, position, partitioner);
+        AbstractBounds<T> rb = new Range<T>(position, right, partitioner);
+        return new Pair<AbstractBounds<T>, AbstractBounds<T>>(lb, rb);
     }
 
     public List<? extends AbstractBounds<T>> unwrap()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35f7bc59/src/java/org/apache/cassandra/dht/ExcludingBounds.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/ExcludingBounds.java b/src/java/org/apache/cassandra/dht/ExcludingBounds.java
new file mode 100644
index 0000000..5b53dd0
--- /dev/null
+++ b/src/java/org/apache/cassandra/dht/ExcludingBounds.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.cassandra.dht;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.cassandra.db.RowPosition;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * AbstractBounds containing neither of its endpoints: (left, right).  Used by CQL key > X AND key < Y range scans.
+ */
+public class ExcludingBounds<T extends RingPosition> extends AbstractBounds<T>
+{
+    public ExcludingBounds(T left, T right)
+    {
+        this(left, right, StorageService.getPartitioner());
+    }
+
+    ExcludingBounds(T left, T right, IPartitioner partitioner)
+    {
+        super(left, right, partitioner);
+        // unlike a Range, an ExcludingBounds may not wrap, nor be empty
+        assert left.compareTo(right) < 0 || right.isMinimum(partitioner) : "(" + left + "," + right + ")";
+    }
+
+    public boolean contains(T position)
+    {
+        return Range.contains(left, right, position) && !right.equals(position);
+    }
+
+    public Pair<AbstractBounds<T>, AbstractBounds<T>> split(T position)
+    {
+        assert contains(position) || left.equals(position);
+        if (left.equals(position))
+            return null;
+        AbstractBounds<T> lb = new Range<T>(left, position, partitioner);
+        AbstractBounds<T> rb = new ExcludingBounds<T>(position, right, partitioner);
+        return new Pair<AbstractBounds<T>, AbstractBounds<T>>(lb, rb);
+    }
+
+    public List<? extends AbstractBounds<T>> unwrap()
+    {
+        // ExcludingBounds objects never wrap
+        return Collections.<AbstractBounds<T>>singletonList(this);
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (!(o instanceof ExcludingBounds))
+            return false;
+        ExcludingBounds<T> rhs = (ExcludingBounds<T>)o;
+        return left.equals(rhs.left) && right.equals(rhs.right);
+    }
+
+    @Override
+    public String toString()
+    {
+        return "(" + left + "," + right + ")";
+    }
+
+    /**
+     * Compute a bounds of keys corresponding to a given bounds of token.
+     */
+    private static ExcludingBounds<RowPosition> makeRowBounds(Token left, Token right, IPartitioner partitioner)
+    {
+        return new ExcludingBounds<RowPosition>(left.maxKeyBound(partitioner), right.minKeyBound(partitioner), partitioner);
+    }
+
+    public AbstractBounds<RowPosition> toRowBounds()
+    {
+        return (left instanceof Token) ? makeRowBounds((Token)left, (Token)right, partitioner) : (ExcludingBounds<RowPosition>)this;
+    }
+
+    public AbstractBounds<Token> toTokenBounds()
+    {
+        return (left instanceof RowPosition) ? new ExcludingBounds<Token>(((RowPosition)left).getToken(), ((RowPosition)right).getToken(), partitioner) : (ExcludingBounds<Token>)this;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35f7bc59/src/java/org/apache/cassandra/dht/IncludingExcludingBounds.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/IncludingExcludingBounds.java b/src/java/org/apache/cassandra/dht/IncludingExcludingBounds.java
new file mode 100644
index 0000000..61bc22f
--- /dev/null
+++ b/src/java/org/apache/cassandra/dht/IncludingExcludingBounds.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.cassandra.dht;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.cassandra.db.RowPosition;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * AbstractBounds containing only its left endpoint: [left, right).  Used by CQL key >= X AND key < Y range scans.
+ */
+public class IncludingExcludingBounds<T extends RingPosition> extends AbstractBounds<T>
+{
+    public IncludingExcludingBounds(T left, T right)
+    {
+        this(left, right, StorageService.getPartitioner());
+    }
+
+    IncludingExcludingBounds(T left, T right, IPartitioner partitioner)
+    {
+        super(left, right, partitioner);
+        // unlike a Range, an IncludingExcludingBounds may not wrap, nor have
+        // right == left unless the right is the min token
+        assert left.compareTo(right) < 0 || right.isMinimum(partitioner) : "[" + left + "," + right + ")";
+    }
+
+    public boolean contains(T position)
+    {
+        return (Range.contains(left, right, position) || left.equals(position)) && !right.equals(position);
+    }
+
+    public Pair<AbstractBounds<T>, AbstractBounds<T>> split(T position)
+    {
+        assert contains(position);
+        AbstractBounds<T> lb = new Bounds<T>(left, position, partitioner);
+        AbstractBounds<T> rb = new ExcludingBounds<T>(position, right, partitioner);
+        return new Pair<AbstractBounds<T>, AbstractBounds<T>>(lb, rb);
+    }
+
+    public List<? extends AbstractBounds<T>> unwrap()
+    {
+        // IncludingExcludingBounds objects never wrap
+        return Collections.<AbstractBounds<T>>singletonList(this);
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (!(o instanceof IncludingExcludingBounds))
+            return false;
+        IncludingExcludingBounds<T> rhs = (IncludingExcludingBounds<T>)o;
+        return left.equals(rhs.left) && right.equals(rhs.right);
+    }
+
+    @Override
+    public String toString()
+    {
+        return "(" + left + "," + right + ")";
+    }
+
+    /**
+     * Compute a bounds of keys corresponding to a given bounds of token.
+     */
+    private static IncludingExcludingBounds<RowPosition> makeRowBounds(Token left, Token right, IPartitioner partitioner)
+    {
+        return new IncludingExcludingBounds<RowPosition>(left.maxKeyBound(partitioner), right.minKeyBound(partitioner), partitioner);
+    }
+
+    public AbstractBounds<RowPosition> toRowBounds()
+    {
+        return (left instanceof Token) ? makeRowBounds((Token)left, (Token)right, partitioner) : (IncludingExcludingBounds<RowPosition>)this;
+    }
+
+    public AbstractBounds<Token> toTokenBounds()
+    {
+        return (left instanceof RowPosition) ? new IncludingExcludingBounds<Token>(((RowPosition)left).getToken(), ((RowPosition)right).getToken(), partitioner) : (IncludingExcludingBounds<Token>)this;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35f7bc59/src/java/org/apache/cassandra/dht/Range.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/Range.java b/src/java/org/apache/cassandra/dht/Range.java
index 3caa6d0..13335d1 100644
--- a/src/java/org/apache/cassandra/dht/Range.java
+++ b/src/java/org/apache/cassandra/dht/Range.java
@@ -28,11 +28,14 @@ import org.apache.commons.lang.ObjectUtils;
 import org.apache.cassandra.db.RowPosition;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
 
 /**
  * A representation of the range that a node is responsible for on the DHT ring.
  *
  * A Range is responsible for the tokens between (left, right].
+ *
+ * Used by the partitioner and by map/reduce by-token range scans.
  */
 public class Range<T extends RingPosition> extends AbstractBounds<T> implements Comparable<Range<T>>, Serializable
 {
@@ -197,19 +200,24 @@ public class Range<T extends RingPosition> extends AbstractBounds<T> implements
         return Collections.unmodifiableSet(intersection);
     }
 
-    public AbstractBounds<T> createFrom(T pos)
+    public Pair<AbstractBounds<T>, AbstractBounds<T>> split(T position)
     {
-        if (pos.equals(left))
+        assert contains(position) || left.equals(position);
+        // Check if the split would have no effect on the range
+        if (position.equals(left) || position.equals(right))
             return null;
-        return new Range<T>(left, pos, partitioner);
+
+        AbstractBounds<T> lb = new Range<T>(left, position, partitioner);
+        AbstractBounds<T> rb = new Range<T>(position, right, partitioner);
+        return new Pair<AbstractBounds<T>, AbstractBounds<T>>(lb, rb);
     }
 
-    public List<? extends AbstractBounds<T>> unwrap()
+    public List<Range<T>> unwrap()
     {
         T minValue = (T) partitioner.minValue(right.getClass());
         if (!isWrapAround() || right.equals(minValue))
             return Arrays.asList(this);
-        List<AbstractBounds<T>> unwrapped = new ArrayList<AbstractBounds<T>>(2);
+        List<Range<T>> unwrapped = new ArrayList<Range<T>>(2);
         unwrapped.add(new Range<T>(left, minValue, partitioner));
         unwrapped.add(new Range<T>(minValue, right, partitioner));
         return unwrapped;
@@ -340,6 +348,79 @@ public class Range<T extends RingPosition> extends AbstractBounds<T> implements
     }
 
     /**
+     * @return A copy of the given list of with all ranges unwrapped, sorted by left bound and with overlapping bounds merged.
+     */
+    public static <T extends RingPosition> List<Range<T>> normalize(Collection<Range<T>> ranges)
+    {
+        // unwrap all
+        List<Range<T>> output = new ArrayList<Range<T>>();
+        for (Range<T> range : ranges)
+            output.addAll(range.unwrap());
+
+        // sort by left
+        Collections.sort(output, new Comparator<Range<T>>()
+        {
+            public int compare(Range<T> b1, Range<T> b2)
+            {
+                return b1.left.compareTo(b2.left);
+            }
+        });
+
+        // deoverlap
+        return deoverlap(output);
+    }
+
+    /**
+     * Given a list of unwrapped ranges sorted by left position, return an
+     * equivalent list of ranges but with no overlapping ranges.
+     */
+    private static <T extends RingPosition> List<Range<T>> deoverlap(List<Range<T>> ranges)
+    {
+        if (ranges.isEmpty())
+            return ranges;
+
+        List<Range<T>> output = new ArrayList<Range<T>>();
+
+        Iterator<Range<T>> iter = ranges.iterator();
+        Range<T> current = iter.next();
+
+        T min = (T) current.partitioner.minValue(current.left.getClass());
+        while (iter.hasNext())
+        {
+            // If current goes to the end of the ring, we're done
+            if (current.right.equals(min))
+            {
+                // If one range is the full range, we return only that
+                if (current.left.equals(min))
+                    return Collections.<Range<T>>singletonList(current);
+
+                output.add(new Range<T>(current.left, min));
+                return output;
+            }
+
+            Range<T> next = iter.next();
+
+            // if next left is equal to current right, we do not intersect per se, but replacing (A, B] and (B, C] by (A, C] is
+            // legit, and since this avoid special casing and will result in more "optimal" ranges, we do the transformation
+            if (next.left.compareTo(current.right) <= 0)
+            {
+                // We do overlap
+                // (we've handled current.right.equals(min) already)
+                if (next.right.equals(min) || current.right.compareTo(next.right) < 0)
+                    current = new Range<T>(current.left, next.right);
+            }
+            else
+            {
+                output.add(current);
+                current = next;
+            }
+        }
+        output.add(current);
+        return output;
+    }
+
+
+    /**
      * Compute a range of keys corresponding to a given range of token.
      */
     public static Range<RowPosition> makeRowRange(Token left, Token right, IPartitioner partitioner)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35f7bc59/src/java/org/apache/cassandra/dht/Token.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/Token.java b/src/java/org/apache/cassandra/dht/Token.java
index dcbea16..e9969e1 100644
--- a/src/java/org/apache/cassandra/dht/Token.java
+++ b/src/java/org/apache/cassandra/dht/Token.java
@@ -42,9 +42,6 @@ public abstract class Token<T> implements RingPosition<Token<T>>, Serializable
 
     public final T token;
 
-    private final transient KeyBound minimumBound = new KeyBound(true);
-    private final transient KeyBound maximumBound = new KeyBound(false);
-
     protected Token(T token)
     {
         this.token = token;
@@ -143,7 +140,7 @@ public abstract class Token<T> implements RingPosition<Token<T>>, Serializable
      */
     public KeyBound minKeyBound(IPartitioner partitioner)
     {
-        return minimumBound;
+        return new KeyBound(this, true);
     }
 
     public KeyBound minKeyBound()
@@ -161,8 +158,8 @@ public abstract class Token<T> implements RingPosition<Token<T>>, Serializable
          * maxKeyBound for the minimun token.
          */
         if (isMinimum(partitioner))
-            return minimumBound;
-        return maximumBound;
+            return minKeyBound();
+        return new KeyBound(this, false);
     }
 
     public KeyBound maxKeyBound()
@@ -170,7 +167,7 @@ public abstract class Token<T> implements RingPosition<Token<T>>, Serializable
         return maxKeyBound(StorageService.getPartitioner());
     }
 
-    public <T extends RingPosition> T asSplitValue(Class<T> klass)
+    public <T extends RingPosition> T upperBound(Class<T> klass)
     {
         if (klass.equals(getClass()))
             return (T)this;
@@ -178,18 +175,20 @@ public abstract class Token<T> implements RingPosition<Token<T>>, Serializable
             return (T)maxKeyBound();
     }
 
-    public class KeyBound extends RowPosition
+    public static class KeyBound extends RowPosition
     {
+        private final Token token;
         public final boolean isMinimumBound;
 
-        private KeyBound(boolean isMinimumBound)
+        private KeyBound(Token t, boolean isMinimumBound)
         {
+            this.token = t;
             this.isMinimumBound = isMinimumBound;
         }
 
         public Token getToken()
         {
-            return Token.this;
+            return token;
         }
 
         public int compareTo(RowPosition pos)
@@ -201,8 +200,10 @@ public abstract class Token<T> implements RingPosition<Token<T>>, Serializable
             if (cmp != 0)
                 return cmp;
 
-            // We've already eliminated the == case
-            return isMinimumBound ? -1 : 1;
+            if (isMinimumBound)
+                return ((pos instanceof KeyBound) && ((KeyBound)pos).isMinimumBound) ? 0 : -1;
+            else
+                return ((pos instanceof KeyBound) && !((KeyBound)pos).isMinimumBound) ? 0 : 1;
         }
 
         public boolean isMinimum(IPartitioner partitioner)
@@ -224,7 +225,7 @@ public abstract class Token<T> implements RingPosition<Token<T>>, Serializable
                 return false;
 
             KeyBound other = (KeyBound)obj;
-            return getToken().equals(other.getToken());
+            return token.equals(other.token) && isMinimumBound == other.isMinimumBound;
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35f7bc59/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
index 6f9fcda..6eb0c4e 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
@@ -490,7 +490,7 @@ public class SSTableReader extends SSTable
         if (samples.isEmpty())
             return positions;
 
-        for (AbstractBounds<Token> range : AbstractBounds.<Token>normalize(ranges))
+        for (Range<Token> range : Range.<Token>normalize(ranges))
         {
             RowPosition leftPosition = range.left.maxKeyBound();
             RowPosition rightPosition = range.left.maxKeyBound();
@@ -587,7 +587,7 @@ public class SSTableReader extends SSTable
     {
         // use the index to determine a minimal section for each range
         List<Pair<Long,Long>> positions = new ArrayList<Pair<Long,Long>>();
-        for (AbstractBounds<Token> range : AbstractBounds.normalize(ranges))
+        for (Range<Token> range : Range.normalize(ranges))
         {
             AbstractBounds<RowPosition> keyRange = range.toRowBounds();
             long left = getPosition(keyRange.left, Operator.GT);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35f7bc59/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java
index 4137632..632f6fc 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -46,7 +46,6 @@ import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.Bounds;
 import org.apache.cassandra.dht.IPartitioner;
@@ -1025,28 +1024,29 @@ public class StorageProxy implements StorageProxyMBean
         AbstractBounds<T> remainder = queryRange;
         while (ringIter.hasNext())
         {
-            Token token = ringIter.next();
             /*
              * remainder can be a range/bounds of token _or_ keys and we want to split it with a token:
              *   - if remainder is tokens, then we'll just split using the provided token.
-             *   - if reaminer is keys, we want to split using token.upperBoundKey. For instance, if remainder
+             *   - if remainder is keys, we want to split using token.upperBoundKey. For instance, if remainder
              *     is [DK(10, 'foo'), DK(20, 'bar')], and we have 3 nodes with tokens 0, 15, 30. We want to
              *     split remainder to A=[DK(10, 'foo'), 15] and B=(15, DK(20, 'bar')]. But since we can't mix
              *     tokens and keys at the same time in a range, we uses 15.upperBoundKey() to have A include all
              *     keys having 15 as token and B include none of those (since that is what our node owns).
              * asSplitValue() abstracts that choice.
              */
-            T splitValue = (T)token.asSplitValue(queryRange.left.getClass());
-            if (remainder == null || !(remainder.left.equals(splitValue) || remainder.contains(splitValue)))
+            Token upperBoundToken = ringIter.next();
+            T upperBound = (T)upperBoundToken.upperBound(queryRange.left.getClass());
+            if (!remainder.left.equals(upperBound) && !remainder.contains(upperBound))
                 // no more splits
                 break;
-            Pair<AbstractBounds<T>,AbstractBounds<T>> splits = remainder.split(splitValue);
-            if (splits.left != null)
-                ranges.add(splits.left);
+            Pair<AbstractBounds<T>,AbstractBounds<T>> splits = remainder.split(upperBound);
+            if (splits == null)
+                continue;
+
+            ranges.add(splits.left);
             remainder = splits.right;
         }
-        if (remainder != null)
-            ranges.add(remainder);
+        ranges.add(remainder);
         if (logger.isDebugEnabled())
             logger.debug("restricted ranges for query {} are {}", queryRange, ranges);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35f7bc59/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
index d2b0a0a..9c871bf 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@ -36,8 +36,7 @@ import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.db.marshal.LexicalUUIDType;
-import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.*;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.SSTableReader;
@@ -53,6 +52,7 @@ import static junit.framework.Assert.assertEquals;
 import static junit.framework.Assert.assertTrue;
 import static org.apache.cassandra.Util.column;
 import static org.apache.cassandra.Util.getBytes;
+import static org.apache.cassandra.Util.rp;
 import static org.apache.cassandra.db.TableTest.assertColumns;
 import static org.junit.Assert.assertNull;
 
@@ -842,4 +842,70 @@ public class ColumnFamilyStoreTest extends CleanupHelper
         assertTotalColCount(cfs.getRangeSlice(null, Util.range("", ""), 6, QueryFilter.getFilter(sp, cfs.getComparator()), null, true), 8);
         assertTotalColCount(cfs.getRangeSlice(null, Util.range("", ""), 100, QueryFilter.getFilter(sp, cfs.getComparator()), null, true), 8);
     }
+
+    private static DecoratedKey idk(int i)
+    {
+        return Util.dk(String.valueOf(i));
+    }
+
+    @Test
+    public void testRangeSliceInclusionExclusion() throws Throwable
+    {
+        String tableName = "Keyspace1";
+        String cfName = "Standard1";
+        Table table = Table.open(tableName);
+        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
+        cfs.clearUnsafe();
+
+        Column[] cols = new Column[5];
+        for (int i = 0; i < 5; i++)
+            cols[i] = column("c" + i, "value", 1);
+
+        for (int i = 0; i <= 9; i++)
+        {
+            putColsStandard(cfs, idk(i), column("name", "value", 1));
+        }
+        cfs.forceBlockingFlush();
+
+        SlicePredicate sp = new SlicePredicate();
+        sp.setSlice_range(new SliceRange());
+        sp.getSlice_range().setCount(1);
+        sp.getSlice_range().setStart(ArrayUtils.EMPTY_BYTE_ARRAY);
+        sp.getSlice_range().setFinish(ArrayUtils.EMPTY_BYTE_ARRAY);
+        IFilter qf = QueryFilter.getFilter(sp, cfs.getComparator());
+
+        List<Row> rows;
+
+        // Start and end inclusive
+        rows = cfs.getRangeSlice(null, new Bounds<RowPosition>(rp("2"), rp("7")), 100, qf, null);
+        assert rows.size() == 6;
+        assert rows.get(0).key.equals(idk(2));
+        assert rows.get(rows.size() - 1).key.equals(idk(7));
+
+        // Start and end excluded
+        rows = cfs.getRangeSlice(null, new ExcludingBounds<RowPosition>(rp("2"), rp("7")), 100, qf, null);
+        assert rows.size() == 4;
+        assert rows.get(0).key.equals(idk(3));
+        assert rows.get(rows.size() - 1).key.equals(idk(6));
+
+        // Start excluded, end included
+        rows = cfs.getRangeSlice(null, new Range<RowPosition>(rp("2"), rp("7")), 100, qf, null);
+        assert rows.size() == 5;
+        assert rows.get(0).key.equals(idk(3));
+        assert rows.get(rows.size() - 1).key.equals(idk(7));
+
+        // Start included, end excluded
+        rows = cfs.getRangeSlice(null, new IncludingExcludingBounds<RowPosition>(rp("2"), rp("7")), 100, qf, null);
+        assert rows.size() == 5;
+        assert rows.get(0).key.equals(idk(2));
+        assert rows.get(rows.size() - 1).key.equals(idk(6));
+    }
+
+    private static String keys(List<Row> rows) throws Throwable
+    {
+        String k = "";
+        for (Row r : rows)
+            k += " " + ByteBufferUtil.string(r.key.key);
+        return k;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35f7bc59/test/unit/org/apache/cassandra/dht/AbstractBoundsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/dht/AbstractBoundsTest.java b/test/unit/org/apache/cassandra/dht/AbstractBoundsTest.java
deleted file mode 100644
index 5e007ac..0000000
--- a/test/unit/org/apache/cassandra/dht/AbstractBoundsTest.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.cassandra.dht;
-
-import java.util.List;
-import static java.util.Arrays.asList;
-
-import org.junit.Test;
-
-import org.apache.cassandra.db.RowPosition;
-import org.apache.cassandra.dht.RingPosition;
-import static org.apache.cassandra.Util.range;
-import static org.apache.cassandra.Util.bounds;
-
-public class AbstractBoundsTest
-{
-    private <T extends RingPosition> void assertNormalize(List<? extends AbstractBounds<T>> input, List<? extends AbstractBounds<T>> expected)
-    {
-        List<AbstractBounds<T>> result = AbstractBounds.normalize(input);
-        assert result.equals(expected) : "Expecting " + expected + " but got " + result;
-    }
-
-    @Test
-    public void testNormalizeNoop()
-    {
-        List<? extends AbstractBounds<RowPosition>> l;
-
-        l = asList(range("1", "3"), range("4", "5"));
-        assert AbstractBounds.normalize(l).equals(l);
-
-        l = asList(bounds("1", "3"), bounds("4", "5"));
-        assertNormalize(l, l);
-    }
-
-    @Test
-    public void testNormalizeSimpleOverlap()
-    {
-        List<? extends AbstractBounds<RowPosition>> input, expected;
-
-        input = asList(range("1", "4"), range("3", "5"));
-        expected = asList(range("1", "5"));
-        assertNormalize(input, expected);
-
-        input = asList(range("1", "4"), range("1", "4"));
-        expected = asList(range("1", "4"));
-        assertNormalize(input, expected);
-
-        input = asList(bounds("1", "4"), bounds("3", "5"));
-        expected = asList(bounds("1", "5"));
-        assertNormalize(input, expected);
-
-        input = asList(bounds("1", "4"), bounds("1", "4"));
-        expected = asList(bounds("1", "4"));
-        assertNormalize(input, expected);
-
-        input = asList(bounds("1", "1"), bounds("1", "1"));
-        expected = asList(bounds("1", "1"));
-        assertNormalize(input, expected);
-    }
-
-    @Test
-    public void testNormalizeSort()
-    {
-        List<? extends AbstractBounds<RowPosition>> input, expected;
-
-        input = asList(range("4", "5"), range("1", "3"));
-        expected = asList(range("1", "3"), range("4", "5"));
-        assertNormalize(input, expected);
-
-        input = asList(bounds("4", "5"), bounds("1", "3"));
-        expected = asList(bounds("1", "3"), bounds("4", "5"));
-        assertNormalize(input, expected);
-    }
-
-    @Test
-    public void testNormalizeUnwrap()
-    {
-        List<? extends AbstractBounds<RowPosition>> input, expected;
-
-        input = asList(range("9", "2"));
-        expected = asList(range("", "2"), range("9", ""));
-        assertNormalize(input, expected);
-
-        // Bounds cannot wrap
-    }
-
-    @Test
-    public void testNormalizeComplex()
-    {
-        List<? extends AbstractBounds<RowPosition>> input, expected;
-
-        input = asList(range("8", "2"), range("7", "9"), range("4", "5"));
-        expected = asList(range("", "2"), range("4", "5"), range("7", ""));
-        assertNormalize(input, expected);
-
-        input = asList(range("5", "9"), range("2", "5"));
-        expected = asList(range("2", "9"));
-        assertNormalize(input, expected);
-
-        input = asList(range ("", "1"), range("9", "2"), range("4", "5"), range("", ""));
-        expected = asList(range("", ""));
-        assertNormalize(input, expected);
-
-        input = asList(range ("", "1"), range("1", "4"), range("4", "5"), range("5", ""));
-        expected = asList(range("", ""));
-        assertNormalize(input, expected);
-
-        // bounds
-        input = asList(bounds("5", "9"), bounds("2", "5"));
-        expected = asList(bounds("2", "9"));
-        assertNormalize(input, expected);
-
-        input = asList(bounds ("", "1"), bounds("", "9"), bounds("4", "5"), bounds("", ""));
-        expected = asList(bounds("", ""));
-        assertNormalize(input, expected);
-
-        input = asList(bounds ("", "1"), bounds("1", "4"), bounds("4", "5"), bounds("5", ""));
-        expected = asList(bounds("", ""));
-        assertNormalize(input, expected);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35f7bc59/test/unit/org/apache/cassandra/dht/RangeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/dht/RangeTest.java b/test/unit/org/apache/cassandra/dht/RangeTest.java
index 7c27787..dc951f8 100644
--- a/test/unit/org/apache/cassandra/dht/RangeTest.java
+++ b/test/unit/org/apache/cassandra/dht/RangeTest.java
@@ -20,11 +20,17 @@ package org.apache.cassandra.dht;
 
 import java.nio.ByteBuffer;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
+import static java.util.Arrays.asList;
 
 import org.apache.commons.lang.StringUtils;
 import org.junit.Test;
 
+import org.apache.cassandra.db.RowPosition;
+import static org.apache.cassandra.Util.range;
+
+
 public class RangeTest
 {
     @Test
@@ -459,4 +465,75 @@ public class RangeTest
         String[][] expected6 = { { "40", "20" } };
         checkDifference(oldRange, newTokens6, expected6);
     }
+
+    private <T extends RingPosition> void assertNormalize(List<Range<T>> input, List<Range<T>> expected)
+    {
+        List<Range<T>> result = Range.normalize(input);
+        assert result.equals(expected) : "Expecting " + expected + " but got " + result;
+    }
+
+    @Test
+    public void testNormalizeNoop()
+    {
+        List<Range<RowPosition>> l;
+
+        l = asList(range("1", "3"), range("4", "5"));
+        assertNormalize(l, l);
+    }
+
+    @Test
+    public void testNormalizeSimpleOverlap()
+    {
+        List<Range<RowPosition>> input, expected;
+
+        input = asList(range("1", "4"), range("3", "5"));
+        expected = asList(range("1", "5"));
+        assertNormalize(input, expected);
+
+        input = asList(range("1", "4"), range("1", "4"));
+        expected = asList(range("1", "4"));
+        assertNormalize(input, expected);
+    }
+
+    @Test
+    public void testNormalizeSort()
+    {
+        List<Range<RowPosition>> input, expected;
+
+        input = asList(range("4", "5"), range("1", "3"));
+        expected = asList(range("1", "3"), range("4", "5"));
+        assertNormalize(input, expected);
+    }
+
+    @Test
+    public void testNormalizeUnwrap()
+    {
+        List<Range<RowPosition>> input, expected;
+
+        input = asList(range("9", "2"));
+        expected = asList(range("", "2"), range("9", ""));
+        assertNormalize(input, expected);
+    }
+
+    @Test
+    public void testNormalizeComplex()
+    {
+        List<Range<RowPosition>> input, expected;
+
+        input = asList(range("8", "2"), range("7", "9"), range("4", "5"));
+        expected = asList(range("", "2"), range("4", "5"), range("7", ""));
+        assertNormalize(input, expected);
+
+        input = asList(range("5", "9"), range("2", "5"));
+        expected = asList(range("2", "9"));
+        assertNormalize(input, expected);
+
+        input = asList(range ("", "1"), range("9", "2"), range("4", "5"), range("", ""));
+        expected = asList(range("", ""));
+        assertNormalize(input, expected);
+
+        input = asList(range ("", "1"), range("1", "4"), range("4", "5"), range("5", ""));
+        expected = asList(range("", ""));
+        assertNormalize(input, expected);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35f7bc59/test/unit/org/apache/cassandra/service/StorageProxyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/StorageProxyTest.java b/test/unit/org/apache/cassandra/service/StorageProxyTest.java
index 6b90d87..1c89cfb 100644
--- a/test/unit/org/apache/cassandra/service/StorageProxyTest.java
+++ b/test/unit/org/apache/cassandra/service/StorageProxyTest.java
@@ -35,6 +35,8 @@ import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.Bounds;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.dht.ExcludingBounds;
+import org.apache.cassandra.dht.IncludingExcludingBounds;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -52,6 +54,16 @@ public class StorageProxyTest extends CleanupHelper
         return new Bounds<RowPosition>(left, right);
     }
 
+    private static ExcludingBounds<RowPosition> exBounds(RowPosition left, RowPosition right)
+    {
+        return new ExcludingBounds<RowPosition>(left, right);
+    }
+
+    private static IncludingExcludingBounds<RowPosition> incExBounds(RowPosition left, RowPosition right)
+    {
+        return new IncludingExcludingBounds<RowPosition>(left, right);
+    }
+
     private static RowPosition startOf(String key)
     {
         return StorageService.getPartitioner().getToken(ByteBufferUtil.bytes(key)).minKeyBound();
@@ -125,18 +137,26 @@ public class StorageProxyTest extends CleanupHelper
         // no splits
         testGRRKeys(range(rp("2"), rp("5")), range(rp("2"), rp("5")));
         testGRRKeys(bounds(rp("2"), rp("5")), bounds(rp("2"), rp("5")));
+        testGRRKeys(exBounds(rp("2"), rp("5")), exBounds(rp("2"), rp("5")));
         // single split testGRRKeys(range("2", "7"), range(rp("2"), endOf("6")), range(endOf("6"), rp("7")));
         testGRRKeys(bounds(rp("2"), rp("7")), bounds(rp("2"), endOf("6")), range(endOf("6"), rp("7")));
+        testGRRKeys(exBounds(rp("2"), rp("7")), range(rp("2"), endOf("6")), exBounds(endOf("6"), rp("7")));
+        testGRRKeys(incExBounds(rp("2"), rp("7")), bounds(rp("2"), endOf("6")), exBounds(endOf("6"), rp("7")));
         // single split starting from min
         testGRRKeys(range(rp(""), rp("2")), range(rp(""), endOf("1")), range(endOf("1"), rp("2")));
         testGRRKeys(bounds(rp(""), rp("2")), bounds(rp(""), endOf("1")), range(endOf("1"), rp("2")));
+        testGRRKeys(exBounds(rp(""), rp("2")), range(rp(""), endOf("1")), exBounds(endOf("1"), rp("2")));
+        testGRRKeys(incExBounds(rp(""), rp("2")), bounds(rp(""), endOf("1")), exBounds(endOf("1"), rp("2")));
         // single split ending with max
         testGRRKeys(range(rp("5"), rp("")), range(rp("5"), endOf("6")), range(endOf("6"), rp("")));
         testGRRKeys(bounds(rp("5"), rp("")), bounds(rp("5"), endOf("6")), range(endOf("6"), rp("")));
+        testGRRKeys(exBounds(rp("5"), rp("")), range(rp("5"), endOf("6")), exBounds(endOf("6"), rp("")));
+        testGRRKeys(incExBounds(rp("5"), rp("")), bounds(rp("5"), endOf("6")), exBounds(endOf("6"), rp("")));
         // two splits
         testGRRKeys(range(rp("0"), rp("7")), range(rp("0"), endOf("1")), range(endOf("1"), endOf("6")), range(endOf("6"), rp("7")));
         testGRRKeys(bounds(rp("0"), rp("7")), bounds(rp("0"), endOf("1")), range(endOf("1"), endOf("6")), range(endOf("6"), rp("7")));
-
+        testGRRKeys(exBounds(rp("0"), rp("7")), range(rp("0"), endOf("1")), range(endOf("1"), endOf("6")), exBounds(endOf("6"), rp("7")));
+        testGRRKeys(incExBounds(rp("0"), rp("7")), bounds(rp("0"), endOf("1")), range(endOf("1"), endOf("6")), exBounds(endOf("6"), rp("7")));
     }
 
     @Test
@@ -158,12 +178,20 @@ public class StorageProxyTest extends CleanupHelper
         testGRRKeys(range(endOf("1"), endOf("5")), range(endOf("1"), endOf("5")));
         testGRRKeys(range(rp("1"), endOf("5")), range(rp("1"), endOf("1")), range(endOf("1"), endOf("5")));
         testGRRKeys(bounds(startOf("1"), endOf("5")), bounds(startOf("1"), endOf("1")), range(endOf("1"), endOf("5")));
+        testGRRKeys(exBounds(endOf("1"), rp("5")), exBounds(endOf("1"), rp("5")));
+        testGRRKeys(exBounds(rp("1"), rp("5")), range(rp("1"), endOf("1")), exBounds(endOf("1"), rp("5")));
+        testGRRKeys(exBounds(startOf("1"), endOf("5")), range(startOf("1"), endOf("1")), exBounds(endOf("1"), endOf("5")));
+        testGRRKeys(incExBounds(rp("1"), rp("5")), bounds(rp("1"), endOf("1")), exBounds(endOf("1"), rp("5")));
         // max
         testGRRKeys(range(endOf("2"), endOf("6")), range(endOf("2"), endOf("6")));
         testGRRKeys(bounds(startOf("2"), endOf("6")), bounds(startOf("2"), endOf("6")));
+        testGRRKeys(exBounds(rp("2"), rp("6")), exBounds(rp("2"), rp("6")));
+        testGRRKeys(incExBounds(rp("2"), rp("6")), incExBounds(rp("2"), rp("6")));
         // bothKeys
         testGRRKeys(range(rp("1"), rp("6")), range(rp("1"), endOf("1")), range(endOf("1"), rp("6")));
         testGRRKeys(bounds(rp("1"), rp("6")), bounds(rp("1"), endOf("1")), range(endOf("1"), rp("6")));
+        testGRRKeys(exBounds(rp("1"), rp("6")), range(rp("1"), endOf("1")), exBounds(endOf("1"), rp("6")));
+        testGRRKeys(incExBounds(rp("1"), rp("6")), bounds(rp("1"), endOf("1")), exBounds(endOf("1"), rp("6")));
     }
 
     @Test
@@ -212,5 +240,7 @@ public class StorageProxyTest extends CleanupHelper
         testGRRKeys(bounds(rp("0"), rp("0")), bounds(rp("0"), rp("0")));
         // completely empty bounds match everything
         testGRRKeys(bounds(rp(""), rp("")), bounds(rp(""), endOf("1")), range(endOf("1"), endOf("6")), range(endOf("6"), rp("")));
+        testGRRKeys(exBounds(rp(""), rp("")), range(rp(""), endOf("1")), range(endOf("1"), endOf("6")), exBounds(endOf("6"), rp("")));
+        testGRRKeys(incExBounds(rp(""), rp("")), bounds(rp(""), endOf("1")), range(endOf("1"), endOf("6")), exBounds(endOf("6"), rp("")));
     }
 }