You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2014/03/13 00:30:28 UTC

[3/5] git commit: apply BTree update function correctly patch by Benedict Elliott Smith; reviewed by jbellis for CASSANDRA-6692

apply BTree update function correctly
patch by Benedict Elliott Smith; reviewed by jbellis for CASSANDRA-6692


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

Branch: refs/heads/trunk
Commit: 0cb1e3d248eb3d14f197217256d8ec29c4b3717c
Parents: 6041433
Author: Jonathan Ellis <jb...@apache.org>
Authored: Wed Mar 12 18:29:40 2014 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Wed Mar 12 18:30:04 2014 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +-
 .../org/apache/cassandra/utils/btree/BTree.java |  49 ++++++--
 .../apache/cassandra/utils/btree/Builder.java   |   8 +-
 .../cassandra/utils/btree/NodeBuilder.java      |   2 +-
 .../cassandra/utils/btree/UpdateFunction.java   |   1 -
 .../org/apache/cassandra/utils/BTreeTest.java   | 125 +++++++++++++++++++
 6 files changed, 171 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0cb1e3d2/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index d346499..912e6af 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -6,7 +6,7 @@
  * Add broadcast_rpc_address option to cassandra.yaml (CASSANDRA-5899)
  * Auto reload GossipingPropertyFileSnitch config (CASSANDRA-5897)
  * Fix overflow of memtable_total_space_in_mb (CASSANDRA-6573)
- * Fix ABTC NPE (CASSANDRA-6692)
+ * Fix ABTC NPE and apply update function correctly (CASSANDRA-6692)
  * Allow nodetool to use a file or prompt for password (CASSANDRA-6660)
  * Fix AIOOBE when concurrently accessing ABSC (CASSANDRA-6742)
  * Fix assertion error in ALTER TYPE RENAME (CASSANDRA-6705)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0cb1e3d2/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 69cf145..ad5065e 100644
--- a/src/java/org/apache/cassandra/utils/btree/BTree.java
+++ b/src/java/org/apache/cassandra/utils/btree/BTree.java
@@ -83,6 +83,11 @@ public class BTree
         return EMPTY_LEAF;
     }
 
+    public static <V> Object[] build(Collection<V> source, Comparator<V> comparator, boolean sorted, UpdateFunction<V> updateF)
+    {
+        return build(source, source.size(), comparator, sorted, updateF);
+    }
+
     /**
      * Creates a BTree containing all of the objects in the provided collection
      *
@@ -92,17 +97,23 @@ public class BTree
      * @param <V>
      * @return
      */
-    public static <V> Object[] build(Collection<V> source, Comparator<V> comparator, boolean sorted, UpdateFunction<V> updateF)
+    public static <V> Object[] build(Iterable<V> source, int size, Comparator<V> comparator, boolean sorted, UpdateFunction<V> updateF)
     {
-        int size = source.size();
-
         if (size < FAN_FACTOR)
         {
             // pad to even length to match contract that all leaf nodes are even
-            V[] values = source.toArray((V[]) new Object[size + (size & 1)]);
+            V[] values = (V[]) new Object[size + (size & 1)];
+            {
+                int i = 0;
+                for (V v : source)
+                    values[i++] = v;
+            }
+
             // inline sorting since we're already calling toArray
             if (!sorted)
                 Arrays.sort(values, 0, size, comparator);
+
+            // if updateF is specified
             if (updateF != null)
             {
                 for (int i = 0 ; i < size ; i++)
@@ -119,7 +130,7 @@ public class BTree
         Builder builder = queue.poll();
         if (builder == null)
             builder = new Builder();
-        Object[] btree = builder.build(source, size);
+        Object[] btree = builder.build(source, updateF, size);
         queue.add(builder);
         return btree;
     }
@@ -139,6 +150,15 @@ public class BTree
         return update(btree, comparator, updateWith, updateWithIsSorted, UpdateFunction.NoOp.<V>instance());
     }
 
+    public static <V> Object[] update(Object[] btree,
+                                      Comparator<V> comparator,
+                                      Collection<V> updateWith,
+                                      boolean updateWithIsSorted,
+                                      UpdateFunction<V> updateF)
+    {
+        return update(btree, comparator, updateWith, updateWith.size(), updateWithIsSorted, updateF);
+    }
+
     /**
      * Returns a new BTree with the provided set inserting/replacing as necessary any equal items
      *
@@ -152,15 +172,16 @@ public class BTree
      */
     public static <V> Object[] update(Object[] btree,
                                       Comparator<V> comparator,
-                                      Collection<V> updateWith,
+                                      Iterable<V> updateWith,
+                                      int updateWithLength,
                                       boolean updateWithIsSorted,
                                       UpdateFunction<V> updateF)
     {
         if (btree.length == 0)
-            return build(updateWith, comparator, updateWithIsSorted, updateF);
+            return build(updateWith, updateWithLength, comparator, updateWithIsSorted, updateF);
 
         if (!updateWithIsSorted)
-            updateWith = sorted(updateWith, comparator, updateWith.size());
+            updateWith = sorted(updateWith, comparator, updateWithLength);
 
         Queue<Builder> queue = modifier.get();
         Builder builder = queue.poll();
@@ -316,6 +337,11 @@ public class BTree
         return (node.length & 1) == 0;
     }
 
+    public static boolean isEmpty(Object[] tree)
+    {
+        return tree.length == 0;
+    }
+
     // Special class for making certain operations easier, so we can define a +/- Inf
     private static interface Special extends Comparable<Object> { }
     static final Special POSITIVE_INFINITY = new Special()
@@ -343,9 +369,12 @@ public class BTree
     };
 
     // return a sorted collection
-    private static <V> Collection<V> sorted(Collection<V> collection, Comparator<V> comparator, int size)
+    private static <V> Collection<V> sorted(Iterable<V> source, Comparator<V> comparator, int size)
     {
-        V[] vs = collection.toArray((V[]) new Object[size]);
+        V[] vs = (V[]) new Object[size];
+        int i = 0;
+        for (V v : source)
+            vs[i++] = v;
         Arrays.sort(vs, comparator);
         return Arrays.asList(vs);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0cb1e3d2/src/java/org/apache/cassandra/utils/btree/Builder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/btree/Builder.java b/src/java/org/apache/cassandra/utils/btree/Builder.java
index 03a7941..36325fe 100644
--- a/src/java/org/apache/cassandra/utils/btree/Builder.java
+++ b/src/java/org/apache/cassandra/utils/btree/Builder.java
@@ -55,7 +55,7 @@ final class Builder
      * we assume @param source has been sorted, e.g. by BTree.update, so the update of each key resumes where
      * the previous left off.
      */
-    public <V> Object[] update(Object[] btree, Comparator<V> comparator, Collection<V> source, UpdateFunction<V> updateF)
+    public <V> Object[] update(Object[] btree, Comparator<V> comparator, Iterable<V> source, UpdateFunction<V> updateF)
     {
         assert updateF != null;
 
@@ -97,15 +97,17 @@ final class Builder
         return r;
     }
 
-    public <V> Object[] build(Collection<V> source, int size)
+    public <V> Object[] build(Iterable<V> source, UpdateFunction<V> updateF, int size)
     {
+        assert updateF != null;
+
         NodeBuilder current = rootBuilder;
         // we descend only to avoid wasting memory; in update() we will often descend into existing trees
         // so here we want to descend also, so we don't have lg max(N) depth in both directions
         while ((size >>= FAN_SHIFT) > 0)
             current = current.ensureChild();
 
-        current.reset(EMPTY_LEAF, POSITIVE_INFINITY, UpdateFunction.NoOp.instance(), null);
+        current.reset(EMPTY_LEAF, POSITIVE_INFINITY, updateF, null);
         for (V key : source)
             current.addNewKey(key);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0cb1e3d2/src/java/org/apache/cassandra/utils/btree/NodeBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/btree/NodeBuilder.java b/src/java/org/apache/cassandra/utils/btree/NodeBuilder.java
index 759ffaa..7039380 100644
--- a/src/java/org/apache/cassandra/utils/btree/NodeBuilder.java
+++ b/src/java/org/apache/cassandra/utils/btree/NodeBuilder.java
@@ -280,7 +280,7 @@ final class NodeBuilder
     void addNewKey(Object key)
     {
         ensureRoom(buildKeyPosition + 1);
-        buildKeys[buildKeyPosition++] = key;
+        buildKeys[buildKeyPosition++] = updateFunction.apply(key);
     }
 
     // copies children from copyf to the builder, up to the provided index in copyf (exclusive)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0cb1e3d2/src/java/org/apache/cassandra/utils/btree/UpdateFunction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/btree/UpdateFunction.java b/src/java/org/apache/cassandra/utils/btree/UpdateFunction.java
index e4062a4..355028e 100644
--- a/src/java/org/apache/cassandra/utils/btree/UpdateFunction.java
+++ b/src/java/org/apache/cassandra/utils/btree/UpdateFunction.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.utils.btree;
 
 import com.google.common.base.Function;
-
 /**
  * An interface defining a function to be applied to both the object we are replacing in a BTree and
  * the object that is intended to replace it, returning the object to actually replace it.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0cb1e3d2/test/unit/org/apache/cassandra/utils/BTreeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/BTreeTest.java b/test/unit/org/apache/cassandra/utils/BTreeTest.java
new file mode 100644
index 0000000..b4a960b
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/BTreeTest.java
@@ -0,0 +1,125 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
+
+import org.junit.Test;
+
+import org.apache.cassandra.utils.btree.BTree;
+import org.apache.cassandra.utils.btree.BTreeSet;
+import org.apache.cassandra.utils.btree.UpdateFunction;
+
+public class BTreeTest
+{
+
+    static Integer[] ints = new Integer[20];
+    static
+    {
+        System.setProperty("cassandra.btree.fanfactor", "4");
+        for (int i = 0 ; i < ints.length ; i++)
+            ints[i] = new Integer(i);
+    }
+
+    static final UpdateFunction<Integer> updateF = new UpdateFunction<Integer>()
+    {
+        public Integer apply(Integer replacing, Integer update)
+        {
+            return ints[update];
+        }
+
+        public boolean abortEarly()
+        {
+            return false;
+        }
+
+        public void allocated(long heapSize)
+        {
+
+        }
+
+        public Integer apply(Integer integer)
+        {
+            return ints[integer];
+        }
+    };
+
+    private static List<Integer> seq(int count)
+    {
+        List<Integer> r = new ArrayList<>();
+        for (int i = 0 ; i < count ; i++)
+            r.add(i);
+        return r;
+    }
+
+    private static List<Integer> rand(int count)
+    {
+        Random rand = ThreadLocalRandom.current();
+        List<Integer> r = seq(count);
+        for (int i = 0 ; i < count - 1 ; i++)
+        {
+            int swap = i + rand.nextInt(count - i);
+            Integer tmp = r.get(i);
+            r.set(i, r.get(swap));
+            r.set(swap, tmp);
+        }
+        return r;
+    }
+
+    private static final Comparator<Integer> CMP = new Comparator<Integer>()
+    {
+        public int compare(Integer o1, Integer o2)
+        {
+            return Integer.compare(o1, o2);
+        }
+    };
+
+    @Test
+    public void testBuilding_UpdateFunctionReplacement()
+    {
+        for (int i = 0; i < 20 ; i++)
+        {
+            checkResult(i, BTree.build(seq(i), CMP, true, updateF));
+            checkResult(i, BTree.build(rand(i), CMP, false, updateF));
+        }
+    }
+
+    @Test
+    public void testUpdate_UpdateFunctionReplacement()
+    {
+        for (int i = 0; i < 20 ; i++)
+        {
+            checkResult(i, BTree.update(BTree.build(seq(i), CMP, true, UpdateFunction.NoOp.<Integer>instance()), CMP, seq(i), true, updateF));
+            checkResult(i, BTree.update(BTree.build(rand(i), CMP, false, UpdateFunction.NoOp.<Integer>instance()), CMP, rand(i), false, updateF));
+        }
+    }
+
+    private static void checkResult(int count, Object[] btree)
+    {
+        BTreeSet<Integer> vs = new BTreeSet<>(btree, CMP);
+        assert vs.size() == count;
+        int i = 0;
+        for (Integer j : vs)
+            assert j == ints[i++];
+    }
+
+}