You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ja...@apache.org on 2017/11/13 13:45:38 UTC

cassandra git commit: Introduce Leaf-only BTree Iterator

Repository: cassandra
Updated Branches:
  refs/heads/trunk 07258a96b -> 0eab80bf3


Introduce Leaf-only BTree Iterator

patch by Piotr Jastrzebski, Jay Zhuang; reviewed by jasobrown for CASSANDRA-9988


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

Branch: refs/heads/trunk
Commit: 0eab80bf389114be8d6f7627f72249bbc3c02e64
Parents: 07258a9
Author: Jay Zhuang <ja...@yahoo.com>
Authored: Sun Jan 15 16:52:58 2017 -0800
Committer: Jason Brown <ja...@gmail.com>
Committed: Mon Nov 13 05:43:22 2017 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../db/partitions/AbstractBTreePartition.java   |   3 +-
 .../org/apache/cassandra/utils/btree/BTree.java |  26 +-
 .../utils/btree/BTreeSearchIterator.java        | 137 +----------
 .../apache/cassandra/utils/btree/BTreeSet.java  |   3 +-
 .../utils/btree/FullBTreeSearchIterator.java    | 159 ++++++++++++
 .../utils/btree/LeafBTreeSearchIterator.java    | 113 +++++++++
 .../microbench/BTreeSearchIteratorBench.java    | 143 +++++++++++
 .../utils/btree/BTreeSearchIteratorTest.java    | 241 +++++++++++++++++++
 9 files changed, 683 insertions(+), 143 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0eab80bf/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f5951d6..494901c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0
+ * Introduce leaf-only iterator (CASSANDRA-9988)
  * Upgrade Guava to 23.3 and Airline to 0.8 (CASSANDRA-13997)
  * Allow only one concurrent call to StatusLogger (CASSANDRA-12182)
  * Refactoring to specialised functional interfaces (CASSANDRA-13982)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0eab80bf/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java b/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
index d913cb3..6dbaff5 100644
--- a/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
+++ b/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
@@ -26,7 +26,6 @@ import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.utils.SearchIterator;
 import org.apache.cassandra.utils.btree.BTree;
-import org.apache.cassandra.utils.btree.BTreeSearchIterator;
 
 import static org.apache.cassandra.utils.btree.BTree.Dir.desc;
 
@@ -131,7 +130,7 @@ public abstract class AbstractBTreePartition implements Partition, Iterable<Row>
         final Holder current = holder();
         return new SearchIterator<Clustering, Row>()
         {
-            private final SearchIterator<Clustering, Row> rawIter = new BTreeSearchIterator<>(current.tree, metadata().comparator, desc(reversed));
+            private final SearchIterator<Clustering, Row> rawIter = BTree.slice(current.tree, metadata().comparator, desc(reversed));
             private final DeletionTime partitionDeletion = current.deletionInfo.getPartitionDeletion();
 
             public Row next(Clustering clustering)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0eab80bf/src/java/org/apache/cassandra/utils/btree/BTree.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/btree/BTree.java b/src/java/org/apache/cassandra/utils/btree/BTree.java
index a4519b9..9ed7534 100644
--- a/src/java/org/apache/cassandra/utils/btree/BTree.java
+++ b/src/java/org/apache/cassandra/utils/btree/BTree.java
@@ -201,12 +201,14 @@ public class BTree
 
     public static <V> Iterator<V> iterator(Object[] btree, Dir dir)
     {
-        return new BTreeSearchIterator<>(btree, null, dir);
+        return isLeaf(btree) ? new LeafBTreeSearchIterator<>(btree, null, dir)
+                             : new FullBTreeSearchIterator<>(btree, null, dir);
     }
 
     public static <V> Iterator<V> iterator(Object[] btree, int lb, int ub, Dir dir)
     {
-        return new BTreeSearchIterator<>(btree, null, dir, lb, ub);
+        return isLeaf(btree) ? new LeafBTreeSearchIterator<>(btree, null, dir, lb, ub)
+                             : new FullBTreeSearchIterator<>(btree, null, dir, lb, ub);
     }
 
     public static <V> Iterable<V> iterable(Object[] btree)
@@ -234,7 +236,8 @@ public class BTree
      */
     public static <K, V> BTreeSearchIterator<K, V> slice(Object[] btree, Comparator<? super K> comparator, Dir dir)
     {
-        return new BTreeSearchIterator<>(btree, comparator, dir);
+        return isLeaf(btree) ? new LeafBTreeSearchIterator<>(btree, comparator, dir)
+                             : new FullBTreeSearchIterator<>(btree, comparator, dir);
     }
 
     /**
@@ -251,6 +254,20 @@ public class BTree
     }
 
     /**
+     * @param btree      the tree to iterate over
+     * @param comparator the comparator that defines the ordering over the items in the tree
+     * @param startIndex      the start index of the range to return, inclusive
+     * @param endIndex        the end index of the range to return, inclusive
+     * @param dir   if false, the iterator will start at the last item and move backwards
+     * @return           an Iterator over the defined sub-range of the tree
+     */
+    public static <K, V extends K> BTreeSearchIterator<K, V> slice(Object[] btree, Comparator<? super K> comparator, int startIndex, int endIndex, Dir dir)
+    {
+        return isLeaf(btree) ? new LeafBTreeSearchIterator<>(btree, comparator, dir, startIndex, endIndex)
+                             : new FullBTreeSearchIterator<>(btree, comparator, dir, startIndex, endIndex);
+    }
+
+    /**
      * @param btree          the tree to iterate over
      * @param comparator     the comparator that defines the ordering over the items in the tree
      * @param start          low bound of the range
@@ -270,7 +287,8 @@ public class BTree
                                       end == null ? Integer.MAX_VALUE
                                                   : endInclusive ? floorIndex(btree, comparator, end)
                                                                  : lowerIndex(btree, comparator, end));
-        return new BTreeSearchIterator<>(btree, comparator, dir, inclusiveLowerBound, inclusiveUpperBound);
+        return isLeaf(btree) ? new LeafBTreeSearchIterator<>(btree, comparator, dir, inclusiveLowerBound, inclusiveUpperBound)
+                             : new FullBTreeSearchIterator<>(btree, comparator, dir, inclusiveLowerBound, inclusiveUpperBound);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0eab80bf/src/java/org/apache/cassandra/utils/btree/BTreeSearchIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/btree/BTreeSearchIterator.java b/src/java/org/apache/cassandra/utils/btree/BTreeSearchIterator.java
index ec16a8e..2fcece6 100644
--- a/src/java/org/apache/cassandra/utils/btree/BTreeSearchIterator.java
+++ b/src/java/org/apache/cassandra/utils/btree/BTreeSearchIterator.java
@@ -18,146 +18,11 @@
 */
 package org.apache.cassandra.utils.btree;
 
-import java.util.Comparator;
 import java.util.Iterator;
-import java.util.NoSuchElementException;
 
 import org.apache.cassandra.utils.IndexedSearchIterator;
 
-import static org.apache.cassandra.utils.btree.BTree.size;
 
-public class BTreeSearchIterator<K, V> extends TreeCursor<K> implements IndexedSearchIterator<K, V>, Iterator<V>
+public interface BTreeSearchIterator<K, V> extends IndexedSearchIterator<K, V>, Iterator<V>
 {
-    private final boolean forwards;
-
-    // for simplicity, we just always use the index feature of the btree to maintain our bounds within the tree,
-    // whether or not they are constrained
-    private int index;
-    private byte state;
-    private final int lowerBound, upperBound; // inclusive
-
-    private static final int MIDDLE = 0; // only "exists" as an absence of other states
-    private static final int ON_ITEM = 1; // may only co-exist with LAST (or MIDDLE, which is 0)
-    private static final int BEFORE_FIRST = 2; // may not coexist with any other state
-    private static final int LAST = 4; // may co-exist with ON_ITEM, in which case we are also at END
-    private static final int END = 5; // equal to LAST | ON_ITEM
-
-    public BTreeSearchIterator(Object[] btree, Comparator<? super K> comparator, BTree.Dir dir)
-    {
-        this(btree, comparator, dir, 0, size(btree)-1);
-    }
-
-    BTreeSearchIterator(Object[] btree, Comparator<? super K> comparator, BTree.Dir dir, int lowerBound, int upperBound)
-    {
-        super(comparator, btree);
-        this.forwards = dir == BTree.Dir.ASC;
-        this.lowerBound = lowerBound;
-        this.upperBound = upperBound;
-        rewind();
-    }
-
-    /**
-     * @return 0 if we are on the last item, 1 if we are past the last item, and -1 if we are before it
-     */
-    private int compareToLast(int idx)
-    {
-        return forwards ? idx - upperBound : lowerBound - idx;
-    }
-
-    private int compareToFirst(int idx)
-    {
-        return forwards ? idx - lowerBound : upperBound - idx;
-    }
-
-    public boolean hasNext()
-    {
-        return state != END;
-    }
-
-    public V next()
-    {
-        switch (state)
-        {
-            case ON_ITEM:
-                if (compareToLast(index = moveOne(forwards)) >= 0)
-                    state = END;
-                break;
-            case BEFORE_FIRST:
-                seekTo(index = forwards ? lowerBound : upperBound);
-                state = (byte) (upperBound == lowerBound ? LAST : MIDDLE);
-            case LAST:
-            case MIDDLE:
-                state |= ON_ITEM;
-                break;
-            default:
-                throw new NoSuchElementException();
-        }
-
-        return current();
-    }
-
-    public V next(K target)
-    {
-        if (!hasNext())
-            return null;
-
-        int state = this.state;
-        boolean found = seekTo(target, forwards, (state & (ON_ITEM | BEFORE_FIRST)) != 0);
-        int index = cur.globalIndex();
-
-        V next = null;
-        if (state == BEFORE_FIRST && compareToFirst(index) < 0)
-            return null;
-
-        int compareToLast = compareToLast(index);
-        if ((compareToLast <= 0))
-        {
-            state = compareToLast < 0 ? MIDDLE : LAST;
-            if (found)
-            {
-                state |= ON_ITEM;
-                next = (V) currentValue();
-            }
-        }
-        else state = END;
-
-        this.state = (byte) state;
-        this.index = index;
-        return next;
-    }
-
-    /**
-     * Reset this Iterator to its starting position
-     */
-    public void rewind()
-    {
-        if (upperBound < lowerBound)
-        {
-            state = (byte) END;
-        }
-        else
-        {
-            // we don't move into the tree until the first request is made, so we know where to go
-            reset(forwards);
-            state = (byte) BEFORE_FIRST;
-        }
-    }
-
-    private void checkOnItem()
-    {
-        if ((state & ON_ITEM) != ON_ITEM)
-            throw new NoSuchElementException();
-    }
-
-    public V current()
-    {
-        checkOnItem();
-        return (V) currentValue();
-    }
-
-    public int indexOfCurrent()
-    {
-        checkOnItem();
-        return compareToFirst(index);
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0eab80bf/src/java/org/apache/cassandra/utils/btree/BTreeSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/btree/BTreeSet.java b/src/java/org/apache/cassandra/utils/btree/BTreeSet.java
index a59e481..1a28d78 100644
--- a/src/java/org/apache/cassandra/utils/btree/BTreeSet.java
+++ b/src/java/org/apache/cassandra/utils/btree/BTreeSet.java
@@ -27,6 +27,7 @@ import org.apache.cassandra.utils.btree.BTree.Dir;
 
 import static org.apache.cassandra.utils.btree.BTree.findIndex;
 
+
 public class BTreeSet<V> implements NavigableSet<V>, List<V>
 {
     protected final Comparator<? super V> comparator;
@@ -361,7 +362,7 @@ public class BTreeSet<V> implements NavigableSet<V>, List<V>
         @Override
         protected BTreeSearchIterator<V, V> slice(Dir dir)
         {
-            return new BTreeSearchIterator<>(tree, comparator, dir, lowerBound, upperBound);
+            return BTree.slice(tree, comparator, lowerBound, upperBound, dir);
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0eab80bf/src/java/org/apache/cassandra/utils/btree/FullBTreeSearchIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/btree/FullBTreeSearchIterator.java b/src/java/org/apache/cassandra/utils/btree/FullBTreeSearchIterator.java
new file mode 100644
index 0000000..c19d447
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/btree/FullBTreeSearchIterator.java
@@ -0,0 +1,159 @@
+/*
+ * 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.utils.btree;
+
+import static org.apache.cassandra.utils.btree.BTree.size;
+
+import java.util.Comparator;
+import java.util.NoSuchElementException;
+
+public class FullBTreeSearchIterator<K, V> extends TreeCursor<K> implements BTreeSearchIterator<K, V>
+{
+    private final boolean forwards;
+
+    // for simplicity, we just always use the index feature of the btree to maintain our bounds within the tree,
+    // whether or not they are constrained
+    private int index;
+    private byte state;
+    private final int lowerBound, upperBound; // inclusive
+
+    private static final int MIDDLE = 0; // only "exists" as an absence of other states
+    private static final int ON_ITEM = 1; // may only co-exist with LAST (or MIDDLE, which is 0)
+    private static final int BEFORE_FIRST = 2; // may not coexist with any other state
+    private static final int LAST = 4; // may co-exist with ON_ITEM, in which case we are also at END
+    private static final int END = 5; // equal to LAST | ON_ITEM
+
+    public FullBTreeSearchIterator(Object[] btree, Comparator<? super K> comparator, BTree.Dir dir)
+    {
+        this(btree, comparator, dir, 0, size(btree)-1);
+    }
+
+    FullBTreeSearchIterator(Object[] btree, Comparator<? super K> comparator, BTree.Dir dir, int lowerBound, int upperBound)
+    {
+        super(comparator, btree);
+        this.forwards = dir == BTree.Dir.ASC;
+        this.lowerBound = lowerBound;
+        this.upperBound = upperBound;
+        rewind();
+    }
+
+    /**
+     * @return 0 if we are on the last item, 1 if we are past the last item, and -1 if we are before it
+     */
+    private int compareToLast(int idx)
+    {
+        return forwards ? idx - upperBound : lowerBound - idx;
+    }
+
+    private int compareToFirst(int idx)
+    {
+        return forwards ? idx - lowerBound : upperBound - idx;
+    }
+
+    public boolean hasNext()
+    {
+        return state != END;
+    }
+
+    public V next()
+    {
+        switch (state)
+        {
+            case ON_ITEM:
+                if (compareToLast(index = moveOne(forwards)) >= 0)
+                    state = END;
+                break;
+            case BEFORE_FIRST:
+                seekTo(index = forwards ? lowerBound : upperBound);
+                state = (byte) (upperBound == lowerBound ? LAST : MIDDLE);
+            case LAST:
+            case MIDDLE:
+                state |= ON_ITEM;
+                break;
+            default:
+                throw new NoSuchElementException();
+        }
+
+        return current();
+    }
+
+    public V next(K target)
+    {
+        if (!hasNext())
+            return null;
+
+        int state = this.state;
+        boolean found = seekTo(target, forwards, (state & (ON_ITEM | BEFORE_FIRST)) != 0);
+        int index = cur.globalIndex();
+
+        V next = null;
+        if (state == BEFORE_FIRST && compareToFirst(index) < 0)
+            return null;
+
+        int compareToLast = compareToLast(index);
+        if ((compareToLast <= 0))
+        {
+            state = compareToLast < 0 ? MIDDLE : LAST;
+            if (found)
+            {
+                state |= ON_ITEM;
+                next = (V) currentValue();
+            }
+        }
+        else state = END;
+
+        this.state = (byte) state;
+        this.index = index;
+        return next;
+    }
+
+    /**
+     * Reset this Iterator to its starting position
+     */
+    public void rewind()
+    {
+        if (upperBound < lowerBound)
+        {
+            state = (byte) END;
+        }
+        else
+        {
+            // we don't move into the tree until the first request is made, so we know where to go
+            reset(forwards);
+            state = (byte) BEFORE_FIRST;
+        }
+    }
+
+    private void checkOnItem()
+    {
+        if ((state & ON_ITEM) != ON_ITEM)
+            throw new NoSuchElementException();
+    }
+
+    public V current()
+    {
+        checkOnItem();
+        return (V) currentValue();
+    }
+
+    public int indexOfCurrent()
+    {
+        checkOnItem();
+        return compareToFirst(index);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0eab80bf/src/java/org/apache/cassandra/utils/btree/LeafBTreeSearchIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/btree/LeafBTreeSearchIterator.java b/src/java/org/apache/cassandra/utils/btree/LeafBTreeSearchIterator.java
new file mode 100644
index 0000000..a6197f8
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/btree/LeafBTreeSearchIterator.java
@@ -0,0 +1,113 @@
+/*
+ * 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.utils.btree;
+
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.NoSuchElementException;
+
+import static org.apache.cassandra.utils.btree.BTree.size;
+
+public class LeafBTreeSearchIterator<K, V> implements BTreeSearchIterator<K, V>
+{
+    private final boolean forwards;
+    private final K[] keys;
+    private final Comparator<? super K> comparator;
+    private int nextPos;
+    private final int lowerBound, upperBound; // inclusive
+    private boolean hasNext;
+    private boolean hasCurrent;
+
+    public LeafBTreeSearchIterator(Object[] btree, Comparator<? super K> comparator, BTree.Dir dir)
+    {
+        this(btree, comparator, dir, 0, size(btree) -1);
+    }
+
+    LeafBTreeSearchIterator(Object[] btree, Comparator<? super K> comparator, BTree.Dir dir, int lowerBound, int upperBound)
+    {
+        this.keys = (K[]) btree;
+        this.forwards = dir == BTree.Dir.ASC;
+        this.comparator = comparator;
+        this.lowerBound = lowerBound;
+        this.upperBound = upperBound;
+        this.nextPos = forwards ? lowerBound : upperBound;
+        this.hasNext = nextPos >= lowerBound && nextPos <= upperBound;
+    }
+
+    public V next()
+    {
+        if (!hasNext)
+            throw new NoSuchElementException();
+        final V elem = (V) keys[nextPos];
+        nextPos += forwards ? 1 : -1;
+        hasNext = nextPos >= lowerBound && nextPos <= upperBound;
+        hasCurrent = true;
+        return elem;
+    }
+
+    public boolean hasNext()
+    {
+        return hasNext;
+    }
+
+    private int searchNext(K key)
+    {
+        int lb = forwards ? nextPos : lowerBound; // inclusive
+        int ub = forwards ? upperBound : nextPos; // inclusive
+
+        return Arrays.binarySearch(keys, lb, ub + 1, key, comparator);
+    }
+
+    public V next(K key)
+    {
+        if (!hasNext)
+            return null;
+        V result = null;
+
+        int find = searchNext(key);
+        if (find >= 0)
+        {
+            hasCurrent = true;
+            result = (V) keys[find];
+            nextPos = find + (forwards ? 1 : -1);
+        }
+        else
+        {
+            nextPos = (forwards ? -1 : -2) - find;
+            hasCurrent = false;
+        }
+        hasNext = nextPos >= lowerBound && nextPos <= upperBound;
+        return result;
+    }
+
+    public V current()
+    {
+        if (!hasCurrent)
+            throw new NoSuchElementException();
+        int current = forwards ? nextPos - 1 : nextPos + 1;
+        return (V) keys[current];
+    }
+
+    public int indexOfCurrent()
+    {
+        if (!hasCurrent)
+            throw new NoSuchElementException();
+        int current = forwards ? nextPos - 1 : nextPos + 1;
+        return forwards ? current - lowerBound : upperBound - current;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0eab80bf/test/microbench/org/apache/cassandra/test/microbench/BTreeSearchIteratorBench.java
----------------------------------------------------------------------
diff --git a/test/microbench/org/apache/cassandra/test/microbench/BTreeSearchIteratorBench.java b/test/microbench/org/apache/cassandra/test/microbench/BTreeSearchIteratorBench.java
new file mode 100644
index 0000000..400b297
--- /dev/null
+++ b/test/microbench/org/apache/cassandra/test/microbench/BTreeSearchIteratorBench.java
@@ -0,0 +1,143 @@
+/*
+ * 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.test.microbench;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.cassandra.utils.btree.BTreeSearchIterator;
+import org.apache.cassandra.utils.btree.BTree;
+import org.apache.cassandra.utils.btree.BTree.Dir;
+import org.apache.cassandra.utils.btree.FullBTreeSearchIterator;
+import org.apache.cassandra.utils.btree.LeafBTreeSearchIterator;
+import org.apache.cassandra.utils.btree.UpdateFunction;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OperationsPerInvocation;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Warmup;
+
+@BenchmarkMode(Mode.SampleTime)
+@OutputTimeUnit(TimeUnit.NANOSECONDS)
+@Warmup(iterations = 1, time = 1, timeUnit = TimeUnit.SECONDS)
+@Measurement(iterations = 2, time = 4, timeUnit = TimeUnit.SECONDS)
+@Fork(value = 2)
+@Threads(1)
+@State(Scope.Benchmark)
+public class BTreeSearchIteratorBench
+{
+    private final int btreeSize = 32;
+
+    @Param({"TREE", "LEAF"})
+    private String iteratorType;
+
+    @Param({ "0",  "1",  "2",  "3", "16", "17", "18", "19",
+            "24", "25", "26", "30", "31" })
+    private int targetIdx;
+
+    private final int cellSize = 1000;
+
+    @Param({"ASC", "DESC"})
+    private String dirParam;
+
+    private Dir dir;
+    private Object[] btree;
+    private ArrayList<String> data;
+    private ArrayList<String> nonExistData;
+
+    private static ArrayList<String> seq(int count, int minCellSize)
+    {
+        ArrayList<String> ret = new ArrayList<>();
+        for (int i = 0 ; i < count ; i++)
+        {
+            StringBuilder sb = new StringBuilder();
+            while (sb.length() < minCellSize)
+            {
+                String uuid = UUID.randomUUID().toString();
+                sb.append(uuid);
+            }
+            ret.add(sb.toString());
+        }
+        Collections.sort(ret);
+        return ret;
+    }
+
+    private static final Comparator<String> CMP = new Comparator<String>()
+    {
+        public int compare(String s1, String s2)
+        {
+            return s1.compareTo(s2);
+        }
+    };
+
+    @Setup(Level.Trial)
+    public void setup() throws Throwable
+    {
+        data = seq(btreeSize, cellSize);
+        nonExistData = new ArrayList<>();
+        btree = BTree.build(data, UpdateFunction.noOp());
+        for (String d : data)
+        {
+            nonExistData.add(d.substring(0, d.length() - 1) + "!");
+        }
+        dir = Dir.valueOf(dirParam);
+
+    }
+
+    @Benchmark
+    public void searchFound()
+    {
+        BTreeSearchIterator<String, String> iter = getIterator();
+        String val = iter.next(data.get(targetIdx));
+        assert(val != null);
+    }
+
+    private BTreeSearchIterator<String,String> getIterator()
+    {
+        switch (iteratorType)
+        {
+            case "LEAF":
+                return new LeafBTreeSearchIterator<>(btree, CMP, dir);
+            case "TREE":
+                return new FullBTreeSearchIterator<>(btree, CMP, dir);
+            default:
+                throw new IllegalArgumentException("unknown btree iterator type: " + iteratorType);
+        }
+    }
+
+    @Benchmark
+    public void searchNotFound()
+    {
+        BTreeSearchIterator<String, String> iter = getIterator();
+        String val = iter.next(nonExistData.get(targetIdx));
+        assert(val == null);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0eab80bf/test/unit/org/apache/cassandra/utils/btree/BTreeSearchIteratorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/btree/BTreeSearchIteratorTest.java b/test/unit/org/apache/cassandra/utils/btree/BTreeSearchIteratorTest.java
new file mode 100644
index 0000000..69ca93c
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/btree/BTreeSearchIteratorTest.java
@@ -0,0 +1,241 @@
+/*
+ * 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.utils.btree;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.utils.btree.BTree.Dir;
+import org.junit.Test;
+
+public class BTreeSearchIteratorTest
+{
+
+    private static List<Integer> seq(int count)
+    {
+        return seq(count, 0, 1);
+    }
+
+    private static List<Integer> seq(int count, int base, int multi)
+    {
+        List<Integer> r = new ArrayList<>();
+        for (int i = 0 ; i < count ; i++)
+            r.add(i * multi + base);
+        return r;
+    }
+
+    private static final Comparator<Integer> CMP = new Comparator<Integer>()
+    {
+        public int compare(Integer o1, Integer o2)
+        {
+            return Integer.compare(o1, o2);
+        }
+    };
+
+    private static void assertIteratorExceptionBegin(final BTreeSearchIterator<Integer, Integer> iter)
+    {
+        try
+        {
+            iter.current();
+            fail("Should throw NoSuchElementException");
+        }
+        catch (NoSuchElementException ex)
+        {
+        }
+        try
+        {
+            iter.indexOfCurrent();
+            fail("Should throw NoSuchElementException");
+        }
+        catch (NoSuchElementException ex)
+        {
+        }
+    }
+
+    private static void assertIteratorExceptionEnd(final BTreeSearchIterator<Integer, Integer> iter)
+    {
+        assertFalse(iter.hasNext());
+        try
+        {
+            iter.next();
+            fail("Should throw NoSuchElementException");
+        }
+        catch (NoSuchElementException ex)
+        {
+        }
+    }
+
+    private static void assertBTreeSearchIteratorEquals(final BTreeSearchIterator<Integer, Integer> iter1,
+                                                        final BTreeSearchIterator<Integer, Integer> iter2)
+    {
+        assertIteratorExceptionBegin(iter1);
+        assertIteratorExceptionBegin(iter2);
+        while (iter1.hasNext())
+        {
+            assertTrue(iter2.hasNext());
+            assertEquals(iter1.next(), iter2.next());
+            assertEquals(iter1.current(), iter2.current());
+            assertEquals(iter1.indexOfCurrent(), iter2.indexOfCurrent());
+        }
+        assertIteratorExceptionEnd(iter1);
+        assertIteratorExceptionEnd(iter2);
+    }
+
+    private static void assertBTreeSearchIteratorEquals(final BTreeSearchIterator<Integer, Integer> iter1,
+                                                        final BTreeSearchIterator<Integer, Integer> iter2,
+                                                        int... targets)
+    {
+        assertIteratorExceptionBegin(iter1);
+        assertIteratorExceptionBegin(iter2);
+        for (int i : targets)
+        {
+            Integer val1 = iter1.next(i);
+            Integer val2 = iter2.next(i);
+            assertEquals(val1, val2);
+            if (val1 != null)
+            {
+                assertEquals(iter1.current(), iter2.current());
+                assertEquals(iter1.indexOfCurrent(), iter2.indexOfCurrent());
+            }
+        }
+
+        while (iter1.hasNext())
+        {
+            assertTrue(iter2.hasNext());
+            assertEquals(iter1.next(), iter2.next());
+            assertEquals(iter1.current(), iter2.current());
+            assertEquals(iter1.indexOfCurrent(), iter2.indexOfCurrent());
+        }
+        assertIteratorExceptionEnd(iter1);
+        assertIteratorExceptionEnd(iter2);
+    }
+
+    @Test
+    public void testTreeIteratorNormal()
+    {
+        Object[] btree = BTree.build(seq(30), UpdateFunction.noOp());
+        BTreeSearchIterator fullIter = new FullBTreeSearchIterator<>(btree, CMP, Dir.ASC);
+        BTreeSearchIterator leafIter = new LeafBTreeSearchIterator<>(btree, CMP, Dir.ASC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter);
+        fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.ASC);
+        leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.ASC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter, -8);
+        fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.ASC);
+        leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.ASC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter, 100);
+        fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.ASC);
+        leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.ASC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter, 3, 4);
+        fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.ASC);
+        leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.ASC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter, 4, 3);
+        fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.ASC);
+        leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.ASC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter, -8, 3, 100);
+        fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.ASC);
+        leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.ASC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter, 0, 29, 30, 0);
+
+        fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.DESC);
+        leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.DESC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter);
+        fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.DESC);
+        leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.DESC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter, 100);
+        fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.DESC);
+        leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.DESC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter, -8);
+        fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.DESC);
+        leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.DESC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter, 4, 3);
+        fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.DESC);
+        leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.DESC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter, 100, 3, -8);
+    }
+
+    @Test
+    public void testTreeIteratorOneElem()
+    {
+        Object[] btree = BTree.build(seq(1), UpdateFunction.noOp());
+        BTreeSearchIterator fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.ASC);
+        BTreeSearchIterator leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.ASC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter);
+        fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.ASC);
+        leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.ASC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter, 0);
+        fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.ASC);
+        leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.ASC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter, 3);
+        fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.ASC);
+        leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.ASC);
+
+        fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.DESC);
+        leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.DESC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter);
+        fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.DESC);
+        leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.DESC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter, 0);
+        fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.DESC);
+        leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.DESC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter, 3);
+    }
+
+    @Test
+    public void testTreeIteratorEmpty()
+    {
+        BTreeSearchIterator leafIter = new LeafBTreeSearchIterator(BTree.empty(), CMP, Dir.ASC);
+        assertFalse(leafIter.hasNext());
+        leafIter = new LeafBTreeSearchIterator(BTree.empty(), CMP, Dir.DESC);
+        assertFalse(leafIter.hasNext());
+    }
+
+    @Test
+    public void testTreeIteratorNotFound()
+    {
+        Object[] btree = BTree.build(seq(31, 0, 3), UpdateFunction.noOp());
+        BTreeSearchIterator fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.ASC);
+        BTreeSearchIterator leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.ASC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter, 3 * 5 + 1);
+        fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.ASC);
+        leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.ASC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter, 3 * 5 + 1, 3 * 7);
+        fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.ASC);
+        leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.ASC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter, 3 * 5 + 1, 3 * 7 + 1);
+
+        fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.DESC);
+        leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.DESC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter, 3 * 5 + 1);
+        fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.DESC);
+        leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.DESC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter, 3 * 5 + 1, 3 * 2);
+        fullIter = new FullBTreeSearchIterator(btree, CMP, Dir.DESC);
+        leafIter = new LeafBTreeSearchIterator(btree, CMP, Dir.DESC);
+        assertBTreeSearchIteratorEquals(fullIter, leafIter, 3 * 5 + 1, 3 * 2 + 1);
+    }
+}


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