You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2022/03/09 13:25:59 UTC

[GitHub] [cassandra] blambov commented on a change in pull request #1462: CASSANDRA-15510 (4.0): Optimise BTree.{build,update,transform}

blambov commented on a change in pull request #1462:
URL: https://github.com/apache/cassandra/pull/1462#discussion_r820599686



##########
File path: src/java/org/apache/cassandra/db/partitions/AtomicBTreePartition.java
##########
@@ -127,22 +127,22 @@ protected boolean canHaveShadowedData()
                 updater.inputDeletionInfoCopy = update.deletionInfo().copy(HeapAllocator.instance);
 
             deletionInfo = current.deletionInfo.mutableCopy().add(updater.inputDeletionInfoCopy);
-            updater.allocated(deletionInfo.unsharedHeapSize() - current.deletionInfo.unsharedHeapSize());
+            updater.onAllocated(deletionInfo.unsharedHeapSize() - current.deletionInfo.unsharedHeapSize());

Review comment:
       Maybe be clearer here by calling this `addOnHeapMemoryUsage` or something similar?

##########
File path: src/java/org/apache/cassandra/utils/btree/BTree.java
##########
@@ -1232,63 +1761,6 @@ else if (quickResolver != null)
         return cmp.compare((V) a, (V) b);

Review comment:
       This method is no longer used (replaced by `compareWellFormed`).

##########
File path: src/java/org/apache/cassandra/utils/btree/BTree.java
##########
@@ -20,283 +20,547 @@
 
 import java.util.*;
 import java.util.function.BiConsumer;
+import java.util.function.BiFunction;
 import java.util.function.Consumer;
+import java.util.function.Function;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
-import com.google.common.base.Predicate;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Ordering;
 
 import org.apache.cassandra.utils.BiLongAccumulator;
+import org.apache.cassandra.utils.BulkIterator;
 import org.apache.cassandra.utils.LongAccumulator;
 import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.utils.caching.TinyThreadLocalPool;
 
-import static com.google.common.collect.Iterables.concat;
-import static com.google.common.collect.Iterables.filter;
-import static com.google.common.collect.Iterables.transform;
 import static java.lang.Math.max;
 import static java.lang.Math.min;
-import static java.util.Comparator.naturalOrder;
 
 public class BTree
 {
     /**
-     * Leaf Nodes are a raw array of values: Object[V1, V1, ...,].
+     * The {@code BRANCH_FACTOR} is defined as the maximum number of children of each branch, with between
+     * BRANCH_FACTOR/2-1 and BRANCH_FACTOR-1 keys being stored in every node. This yields a minimum tree size of
+     * {@code (BRANCH_FACTOR/2)^height - 1} and a maximum tree size of {@code BRANCH_FACTOR^height - 1}.
      *
-     * Branch Nodes: Object[V1, V2, ..., child[&lt;V1.key], child[&lt;V2.key], ..., child[&lt; Inf], size], where
-     * each child is another node, i.e., an Object[].  Thus, the value elements in a branch node are the
-     * first half of the array (minus one).  In our implementation, each value must include its own key;
-     * we access these via Comparator, rather than directly. 
+     * Branches differ from leaves only in that they contain a suffix region containing the child nodes that occur
+     * either side of the keys, and a sizeMap in the last position, permitting seeking by index within the tree.
+     * Nodes are disambiguated by the length of the array that represents them: an even number is a branch, odd a leaf.
      *
-     * So we can quickly distinguish between leaves and branches, we require that leaf nodes are always an odd number
-     * of elements (padded with a null, if necessary), and branches are always an even number of elements.
+     * Leaf Nodes are represented by an odd-length array of keys, with the final element possibly null, i.e.
+     * Object[V1, V2, ...,null?]
+     *
+     * Branch nodes: Object[V1, V2, ..., child[&lt;V1.key], child[&lt;V2.key], ..., child[&lt; Inf], sizeMap]
+     * Each child is either a branch or leaf, i.e., always an Object[].
+     * The key elements in a branch node occupy the first half of the array (minus one)
      *
      * BTrees are immutable; updating one returns a new tree that reuses unmodified nodes.
      *
-     * There are no references back to a parent node from its children.  (This would make it impossible to re-use
-     * subtrees when modifying the tree, since the modified tree would need new parent references.)
+     * There are no references back to a parent node from its children (this would make it impossible to re-use
+     * subtrees when modifying the tree, since the modified tree would need new parent references).
      * Instead, we store these references in a Path as needed when navigating the tree.
      */
+    public static final int BRANCH_SHIFT = Integer.getInteger("cassandra.btree.branchshift", 5);
 
-    // The maximum fan factor used for B-Trees
-    static final int FAN_SHIFT;
-
-    // The maximun tree size for certain heigth of tree
-    static final int[] TREE_SIZE;
-
-    // NB we encode Path indexes as Bytes, so this needs to be less than Byte.MAX_VALUE / 2
-    static final int FAN_FACTOR;
-
-    static final int MAX_TREE_SIZE = Integer.MAX_VALUE;
-
-    static
-    {
-        int fanfactor = Integer.parseInt(System.getProperty("cassandra.btree.fanfactor", "32"));
-        assert fanfactor >= 2 : "the minimal btree fanfactor is 2";
-        int shift = 1;
-        while (1 << shift < fanfactor)
-            shift += 1;
-
-        FAN_SHIFT = shift;
-        FAN_FACTOR = 1 << FAN_SHIFT;
-
-        // For current FAN_FACTOR, calculate the maximum height of the tree we could build
-        int maxHeight = 0;
-        for (long maxSize = 0; maxSize < MAX_TREE_SIZE; maxHeight++)
-            // each tree node could have (FAN_FACTOR + 1) children,
-            // plus current node could have FAN_FACTOR number of values
-            maxSize = maxSize * (FAN_FACTOR + 1) + FAN_FACTOR;
-
-        TREE_SIZE = new int[maxHeight];
-
-        TREE_SIZE[0] = FAN_FACTOR;
-        for (int i = 1; i < maxHeight - 1; i++)
-            TREE_SIZE[i] = TREE_SIZE[i - 1] * (FAN_FACTOR + 1) + FAN_FACTOR;
-
-        TREE_SIZE[maxHeight - 1] = MAX_TREE_SIZE;
-    }
-
-
-    static final int MINIMAL_NODE_SIZE = FAN_FACTOR >> 1;
+    private static final int BRANCH_FACTOR = 1 << BRANCH_SHIFT;
+    public static final int MIN_KEYS = BRANCH_FACTOR/2 - 1;
+    public static final int MAX_KEYS = BRANCH_FACTOR - 1;
 
     // An empty BTree Leaf - which is the same as an empty BTree
-    static final Object[] EMPTY_LEAF = new Object[1];
+    private static final Object[] EMPTY_LEAF = new Object[1];
 
-    // An empty BTree branch - used only for internal purposes in Modifier
-    static final Object[] EMPTY_BRANCH = new Object[] { null, new int[0] };
-
-    // direction of iteration
-    public static enum Dir
-    {
-        ASC, DESC;
-        public Dir invert() { return this == ASC ? DESC : ASC; }
-        public static Dir asc(boolean asc) { return asc ? ASC : DESC; }
-        public static Dir desc(boolean desc) { return desc ? DESC : ASC; }
-    }
+    private static final int[][] DENSE_SIZE_MAPS = buildBalancedSizeMaps(BRANCH_SHIFT);
+    private static final int[][] SPARSE_SIZE_MAPS = buildBalancedSizeMaps(BRANCH_SHIFT - 1);
+    private static final long[] PERFECT_DENSE_SIZE_ON_HEAP = sizeOnHeapOfPerfectTrees(BRANCH_SHIFT);
+    private static final long[] PERFECT_SPARSE_SIZE_ON_HEAP = sizeOnHeapOfPerfectTrees(BRANCH_SHIFT - 1);
 
     /**
-     * Enables methods to consume the contents of iterators, collections, or arrays without duplicating code or
-     * allocating intermediate objects. Instead of taking an argument that implements an interface, a method takes
-     * an opaque object as the input, and a singleton helper object it uses as an intermediary to access it's contents.
-     * The purpose of doing things this way is to avoid memory allocations on hot paths.
+     * Represents the direction of iteration.
      */
-    private interface IteratingFunction<T>
+    public enum Dir
     {
-        /**
-         * Returns the next object at the given index. This method  must be called with sequentially increasing index
-         * values, starting at 0, and must only be called once per index value. The results of calling this method
-         * without following these rules are undefined.
-         */
-        <K> K nextAt(T input, int idx);
-    }
+        ASC, DESC;
 
-    private static final IteratingFunction<Iterator> ITERATOR_FUNCTION = new IteratingFunction<Iterator>()
-    {
-        public <K> K nextAt(Iterator input, int idx)
+        public Dir invert()
         {
-            return (K) input.next();
+            return this == ASC ? DESC : ASC;
         }
-    };
 
-    private static final IteratingFunction<Object[]> ARRAY_FUNCTION = new IteratingFunction<Object[]>()
-    {
-        public <K> K nextAt(Object[] input, int idx)
+        public static Dir desc(boolean desc)
         {
-            return (K) input[idx];
+            return desc ? DESC : ASC;
         }
-    };
+    }
 
+    /**
+     * Returns an empty BTree
+     *
+     * @return an empty BTree
+     */
     public static Object[] empty()
     {
         return EMPTY_LEAF;
     }
 
+    /**
+     * Create a BTree containing only the specified object
+     *
+     * @return an new BTree containing only the specified object
+     */
     public static Object[] singleton(Object value)
     {
         return new Object[] { value };
     }
 
-    public static <C, K extends C, V extends C> Object[] build(Collection<K> source, UpdateFunction<K, V> updateF)
+    @Deprecated
+    public static <C, K extends C, V extends C> Object[] build(Collection<K> source)
     {
-        return buildInternal(source.iterator(), ITERATOR_FUNCTION, source.size(), updateF);
+        return build(source, UpdateFunction.noOp());
     }
 
-    /**
-     * Creates a BTree containing all of the objects in the provided collection
-     *
-     * @param source  the items to build the tree with. MUST BE IN STRICTLY ASCENDING ORDER.
-     * @param size    the size of the source iterable
-     * @return        a btree representing the contents of the provided iterable
-     */
-    public static <C, K extends C, V extends C> Object[] build(Iterable<K> source, int size, UpdateFunction<K, V> updateF)
+    @Deprecated
+    public static <C, K extends C, V extends C> Object[] build(Collection<K> source, UpdateFunction<K, V> updateF)
     {
-        if (size < 0)
-            throw new IllegalArgumentException(Integer.toString(size));
-        return buildInternal(source.iterator(), ITERATOR_FUNCTION, size, updateF);
+        return build(BulkIterator.of(source.iterator()), source.size(), updateF);
     }
 
-    public static <C, K extends C, V extends C> Object[] build(Object[] source, int size, UpdateFunction<K, V> updateF)
+    public static <C, I extends C, O extends C> Object[] build(BulkIterator<I> source, int size, UpdateFunction<I, O> updateF)
     {
-        if (size < 0)
-            throw new IllegalArgumentException(Integer.toString(size));
-        return buildInternal(source, ARRAY_FUNCTION, size, updateF);
+        assert size >= 0;
+        if (size == 0)
+            return EMPTY_LEAF;
+
+        if (size <= MAX_KEYS)
+            return buildLeaf(source, size, updateF);
+
+        return buildRoot(source, size, updateF);
     }
 
-    private static <C, K extends C, V extends C, S> Object[] buildLeaf(S source, IteratingFunction<S> iterFunc, int size, int startIdx, UpdateFunction<K, V> updateF)
+    /**
+     * Build a leaf with {@code size} elements taken in bulk from {@code insert}, and apply {@code updateF} to these elements
+     */
+    private static <C, I extends C, O extends C> Object[] buildLeaf(BulkIterator<I> insert,
+                                                                    int size,
+                                                                    UpdateFunction<I, O> updateF)
     {
-        V[] values = (V[]) new Object[size | 1];
+        Object[] values = new Object[size | 1]; // ensure that we have an odd-length array
+        insert.fetch(values, 0, size);
+        if (!isSimple(updateF))
+        {
+            updateF.onAllocated(ObjectSizes.sizeOfReferenceArray(values.length));
+            for (int i = 0; i < size; i++)
+                values[i] = updateF.apply((I) values[i]);
+        }
+        return values;
+    }
 
-        int idx = startIdx;
-        for (int i = 0; i < size; i++)
+    /**
+     * Build a leaf with {@code size} elements taken in bulk from {@code insert}, and apply {@code updateF} to these elements
+     * Do not invoke {@code updateF.onAllocated}.  Used by {@link #buildMaximallyDense} and {@link #buildPerfect} which
+     * track the size for the entire tree they build in order to save on work.
+     */
+    private static <C, I extends C, O extends C> Object[] buildLeafWithoutSizeTracking(BulkIterator<I> insert, int size, UpdateFunction<I, O> updateF)
+    {
+        Object[] values = new Object[size | 1]; // ensure that we have an odd-length array
+        insert.fetch(values, 0, size);
+        if (!isSimple(updateF))
         {
-            K k = iterFunc.nextAt(source, idx);
-            values[i] = updateF.apply(k);
-            idx++;
+            for (int i = 0; i < size; i++)
+                values[i] = updateF.apply((I) values[i]);
         }
-        if (updateF != UpdateFunction.<K>noOp())
-            updateF.allocated(ObjectSizes.sizeOfArray(values));
         return values;
     }
 
-    private static <C, K extends C, V extends C, S> Object[] buildInternal(S source, IteratingFunction<S> iterFunc, int size, int level, int startIdx, UpdateFunction<K, V> updateF)
+    /**
+     * Build a root node from the first {@code size} elements from {@code source}, applying {@code updateF} to those elements.
+     * A root node is permitted to have as few as two children, if a branch (i.e. if {@code size > MAX_SIZE}.
+     */
+    private static <C, I extends C, O extends C> Object[] buildRoot(BulkIterator<I> source, int size, UpdateFunction<I, O> updateF)
     {
-        assert size > 0;
-        assert level >= 0;
-        if (level == 0)
-            return buildLeaf(source, iterFunc, size, startIdx, updateF);
+        // first calculate the minimum height needed for this size of tree
+        int height = minHeight(size);
+        assert height > 1;
 
-        // calcuate child num: (size - (childNum - 1)) / maxChildSize <= childNum
-        int childNum = size / (TREE_SIZE[level - 1] + 1) + 1;
+        int denseChildSize = denseSize(height - 1);
+        int childCount = size / (denseChildSize + 1) + 1;
 
-        V[] values = (V[]) new Object[childNum * 2];
-        if (updateF != UpdateFunction.<K>noOp())
-            updateF.allocated(ObjectSizes.sizeOfArray(values));
+        return buildMaximallyDense(source, childCount, size, height, updateF);
+    }
+
+    /**
+     * Build a tree containing some initial quantity of dense nodes, up to a single node of arbitrary size (between
+     * dense and sparse), and the remainder of nodes all sparse.

Review comment:
       It would be great if we had a statement what this shape optimizes for. Appends? Structure size? Efficiency of the construction procedure?
   
   AFAICS the main advantage is the structure size, because fully dense and sparse trees can have shared size maps.

##########
File path: src/java/org/apache/cassandra/utils/btree/BTree.java
##########
@@ -20,283 +20,547 @@
 
 import java.util.*;
 import java.util.function.BiConsumer;
+import java.util.function.BiFunction;
 import java.util.function.Consumer;
+import java.util.function.Function;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
-import com.google.common.base.Predicate;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Ordering;
 
 import org.apache.cassandra.utils.BiLongAccumulator;
+import org.apache.cassandra.utils.BulkIterator;
 import org.apache.cassandra.utils.LongAccumulator;
 import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.utils.caching.TinyThreadLocalPool;
 
-import static com.google.common.collect.Iterables.concat;
-import static com.google.common.collect.Iterables.filter;
-import static com.google.common.collect.Iterables.transform;
 import static java.lang.Math.max;
 import static java.lang.Math.min;
-import static java.util.Comparator.naturalOrder;
 
 public class BTree
 {
     /**
-     * Leaf Nodes are a raw array of values: Object[V1, V1, ...,].
+     * The {@code BRANCH_FACTOR} is defined as the maximum number of children of each branch, with between
+     * BRANCH_FACTOR/2-1 and BRANCH_FACTOR-1 keys being stored in every node. This yields a minimum tree size of
+     * {@code (BRANCH_FACTOR/2)^height - 1} and a maximum tree size of {@code BRANCH_FACTOR^height - 1}.
      *
-     * Branch Nodes: Object[V1, V2, ..., child[&lt;V1.key], child[&lt;V2.key], ..., child[&lt; Inf], size], where
-     * each child is another node, i.e., an Object[].  Thus, the value elements in a branch node are the
-     * first half of the array (minus one).  In our implementation, each value must include its own key;
-     * we access these via Comparator, rather than directly. 
+     * Branches differ from leaves only in that they contain a suffix region containing the child nodes that occur
+     * either side of the keys, and a sizeMap in the last position, permitting seeking by index within the tree.
+     * Nodes are disambiguated by the length of the array that represents them: an even number is a branch, odd a leaf.
      *
-     * So we can quickly distinguish between leaves and branches, we require that leaf nodes are always an odd number
-     * of elements (padded with a null, if necessary), and branches are always an even number of elements.
+     * Leaf Nodes are represented by an odd-length array of keys, with the final element possibly null, i.e.
+     * Object[V1, V2, ...,null?]
+     *
+     * Branch nodes: Object[V1, V2, ..., child[&lt;V1.key], child[&lt;V2.key], ..., child[&lt; Inf], sizeMap]
+     * Each child is either a branch or leaf, i.e., always an Object[].
+     * The key elements in a branch node occupy the first half of the array (minus one)
      *
      * BTrees are immutable; updating one returns a new tree that reuses unmodified nodes.
      *
-     * There are no references back to a parent node from its children.  (This would make it impossible to re-use
-     * subtrees when modifying the tree, since the modified tree would need new parent references.)
+     * There are no references back to a parent node from its children (this would make it impossible to re-use
+     * subtrees when modifying the tree, since the modified tree would need new parent references).
      * Instead, we store these references in a Path as needed when navigating the tree.
      */
+    public static final int BRANCH_SHIFT = Integer.getInteger("cassandra.btree.branchshift", 5);
 
-    // The maximum fan factor used for B-Trees
-    static final int FAN_SHIFT;
-
-    // The maximun tree size for certain heigth of tree
-    static final int[] TREE_SIZE;
-
-    // NB we encode Path indexes as Bytes, so this needs to be less than Byte.MAX_VALUE / 2
-    static final int FAN_FACTOR;
-
-    static final int MAX_TREE_SIZE = Integer.MAX_VALUE;
-
-    static
-    {
-        int fanfactor = Integer.parseInt(System.getProperty("cassandra.btree.fanfactor", "32"));
-        assert fanfactor >= 2 : "the minimal btree fanfactor is 2";
-        int shift = 1;
-        while (1 << shift < fanfactor)
-            shift += 1;
-
-        FAN_SHIFT = shift;
-        FAN_FACTOR = 1 << FAN_SHIFT;
-
-        // For current FAN_FACTOR, calculate the maximum height of the tree we could build
-        int maxHeight = 0;
-        for (long maxSize = 0; maxSize < MAX_TREE_SIZE; maxHeight++)
-            // each tree node could have (FAN_FACTOR + 1) children,
-            // plus current node could have FAN_FACTOR number of values
-            maxSize = maxSize * (FAN_FACTOR + 1) + FAN_FACTOR;
-
-        TREE_SIZE = new int[maxHeight];
-
-        TREE_SIZE[0] = FAN_FACTOR;
-        for (int i = 1; i < maxHeight - 1; i++)
-            TREE_SIZE[i] = TREE_SIZE[i - 1] * (FAN_FACTOR + 1) + FAN_FACTOR;
-
-        TREE_SIZE[maxHeight - 1] = MAX_TREE_SIZE;
-    }
-
-
-    static final int MINIMAL_NODE_SIZE = FAN_FACTOR >> 1;
+    private static final int BRANCH_FACTOR = 1 << BRANCH_SHIFT;
+    public static final int MIN_KEYS = BRANCH_FACTOR/2 - 1;
+    public static final int MAX_KEYS = BRANCH_FACTOR - 1;
 
     // An empty BTree Leaf - which is the same as an empty BTree
-    static final Object[] EMPTY_LEAF = new Object[1];
+    private static final Object[] EMPTY_LEAF = new Object[1];
 
-    // An empty BTree branch - used only for internal purposes in Modifier
-    static final Object[] EMPTY_BRANCH = new Object[] { null, new int[0] };
-
-    // direction of iteration
-    public static enum Dir
-    {
-        ASC, DESC;
-        public Dir invert() { return this == ASC ? DESC : ASC; }
-        public static Dir asc(boolean asc) { return asc ? ASC : DESC; }
-        public static Dir desc(boolean desc) { return desc ? DESC : ASC; }
-    }
+    private static final int[][] DENSE_SIZE_MAPS = buildBalancedSizeMaps(BRANCH_SHIFT);
+    private static final int[][] SPARSE_SIZE_MAPS = buildBalancedSizeMaps(BRANCH_SHIFT - 1);
+    private static final long[] PERFECT_DENSE_SIZE_ON_HEAP = sizeOnHeapOfPerfectTrees(BRANCH_SHIFT);
+    private static final long[] PERFECT_SPARSE_SIZE_ON_HEAP = sizeOnHeapOfPerfectTrees(BRANCH_SHIFT - 1);
 
     /**
-     * Enables methods to consume the contents of iterators, collections, or arrays without duplicating code or
-     * allocating intermediate objects. Instead of taking an argument that implements an interface, a method takes
-     * an opaque object as the input, and a singleton helper object it uses as an intermediary to access it's contents.
-     * The purpose of doing things this way is to avoid memory allocations on hot paths.
+     * Represents the direction of iteration.
      */
-    private interface IteratingFunction<T>
+    public enum Dir
     {
-        /**
-         * Returns the next object at the given index. This method  must be called with sequentially increasing index
-         * values, starting at 0, and must only be called once per index value. The results of calling this method
-         * without following these rules are undefined.
-         */
-        <K> K nextAt(T input, int idx);
-    }
+        ASC, DESC;
 
-    private static final IteratingFunction<Iterator> ITERATOR_FUNCTION = new IteratingFunction<Iterator>()
-    {
-        public <K> K nextAt(Iterator input, int idx)
+        public Dir invert()
         {
-            return (K) input.next();
+            return this == ASC ? DESC : ASC;
         }
-    };
 
-    private static final IteratingFunction<Object[]> ARRAY_FUNCTION = new IteratingFunction<Object[]>()
-    {
-        public <K> K nextAt(Object[] input, int idx)
+        public static Dir desc(boolean desc)
         {
-            return (K) input[idx];
+            return desc ? DESC : ASC;
         }
-    };
+    }
 
+    /**
+     * Returns an empty BTree
+     *
+     * @return an empty BTree
+     */
     public static Object[] empty()
     {
         return EMPTY_LEAF;
     }
 
+    /**
+     * Create a BTree containing only the specified object
+     *
+     * @return an new BTree containing only the specified object
+     */
     public static Object[] singleton(Object value)
     {
         return new Object[] { value };
     }
 
-    public static <C, K extends C, V extends C> Object[] build(Collection<K> source, UpdateFunction<K, V> updateF)
+    @Deprecated
+    public static <C, K extends C, V extends C> Object[] build(Collection<K> source)
     {
-        return buildInternal(source.iterator(), ITERATOR_FUNCTION, source.size(), updateF);
+        return build(source, UpdateFunction.noOp());
     }
 
-    /**
-     * Creates a BTree containing all of the objects in the provided collection
-     *
-     * @param source  the items to build the tree with. MUST BE IN STRICTLY ASCENDING ORDER.
-     * @param size    the size of the source iterable
-     * @return        a btree representing the contents of the provided iterable
-     */
-    public static <C, K extends C, V extends C> Object[] build(Iterable<K> source, int size, UpdateFunction<K, V> updateF)
+    @Deprecated
+    public static <C, K extends C, V extends C> Object[] build(Collection<K> source, UpdateFunction<K, V> updateF)
     {
-        if (size < 0)
-            throw new IllegalArgumentException(Integer.toString(size));
-        return buildInternal(source.iterator(), ITERATOR_FUNCTION, size, updateF);
+        return build(BulkIterator.of(source.iterator()), source.size(), updateF);
     }
 
-    public static <C, K extends C, V extends C> Object[] build(Object[] source, int size, UpdateFunction<K, V> updateF)
+    public static <C, I extends C, O extends C> Object[] build(BulkIterator<I> source, int size, UpdateFunction<I, O> updateF)
     {
-        if (size < 0)
-            throw new IllegalArgumentException(Integer.toString(size));
-        return buildInternal(source, ARRAY_FUNCTION, size, updateF);
+        assert size >= 0;
+        if (size == 0)
+            return EMPTY_LEAF;
+
+        if (size <= MAX_KEYS)
+            return buildLeaf(source, size, updateF);
+
+        return buildRoot(source, size, updateF);
     }
 
-    private static <C, K extends C, V extends C, S> Object[] buildLeaf(S source, IteratingFunction<S> iterFunc, int size, int startIdx, UpdateFunction<K, V> updateF)
+    /**
+     * Build a leaf with {@code size} elements taken in bulk from {@code insert}, and apply {@code updateF} to these elements
+     */
+    private static <C, I extends C, O extends C> Object[] buildLeaf(BulkIterator<I> insert,
+                                                                    int size,
+                                                                    UpdateFunction<I, O> updateF)
     {
-        V[] values = (V[]) new Object[size | 1];
+        Object[] values = new Object[size | 1]; // ensure that we have an odd-length array
+        insert.fetch(values, 0, size);
+        if (!isSimple(updateF))
+        {
+            updateF.onAllocated(ObjectSizes.sizeOfReferenceArray(values.length));
+            for (int i = 0; i < size; i++)
+                values[i] = updateF.apply((I) values[i]);
+        }
+        return values;
+    }
 
-        int idx = startIdx;
-        for (int i = 0; i < size; i++)
+    /**
+     * Build a leaf with {@code size} elements taken in bulk from {@code insert}, and apply {@code updateF} to these elements
+     * Do not invoke {@code updateF.onAllocated}.  Used by {@link #buildMaximallyDense} and {@link #buildPerfect} which
+     * track the size for the entire tree they build in order to save on work.
+     */
+    private static <C, I extends C, O extends C> Object[] buildLeafWithoutSizeTracking(BulkIterator<I> insert, int size, UpdateFunction<I, O> updateF)
+    {
+        Object[] values = new Object[size | 1]; // ensure that we have an odd-length array
+        insert.fetch(values, 0, size);
+        if (!isSimple(updateF))
         {
-            K k = iterFunc.nextAt(source, idx);
-            values[i] = updateF.apply(k);
-            idx++;
+            for (int i = 0; i < size; i++)
+                values[i] = updateF.apply((I) values[i]);
         }
-        if (updateF != UpdateFunction.<K>noOp())
-            updateF.allocated(ObjectSizes.sizeOfArray(values));
         return values;
     }
 
-    private static <C, K extends C, V extends C, S> Object[] buildInternal(S source, IteratingFunction<S> iterFunc, int size, int level, int startIdx, UpdateFunction<K, V> updateF)
+    /**
+     * Build a root node from the first {@code size} elements from {@code source}, applying {@code updateF} to those elements.
+     * A root node is permitted to have as few as two children, if a branch (i.e. if {@code size > MAX_SIZE}.
+     */
+    private static <C, I extends C, O extends C> Object[] buildRoot(BulkIterator<I> source, int size, UpdateFunction<I, O> updateF)
     {
-        assert size > 0;
-        assert level >= 0;
-        if (level == 0)
-            return buildLeaf(source, iterFunc, size, startIdx, updateF);
+        // first calculate the minimum height needed for this size of tree
+        int height = minHeight(size);
+        assert height > 1;
 
-        // calcuate child num: (size - (childNum - 1)) / maxChildSize <= childNum
-        int childNum = size / (TREE_SIZE[level - 1] + 1) + 1;
+        int denseChildSize = denseSize(height - 1);
+        int childCount = size / (denseChildSize + 1) + 1;
 
-        V[] values = (V[]) new Object[childNum * 2];
-        if (updateF != UpdateFunction.<K>noOp())
-            updateF.allocated(ObjectSizes.sizeOfArray(values));
+        return buildMaximallyDense(source, childCount, size, height, updateF);
+    }
+
+    /**
+     * Build a tree containing some initial quantity of dense nodes, up to a single node of arbitrary size (between
+     * dense and sparse), and the remainder of nodes all sparse.
+     *
+     * This permits us to build any size of tree:
+     * 1) A root node can have any number of children, so for a given height we can build our smallest tree with two
+     *    sparse nodes.  For a given childCount we can vary up to a whole dense child's size with this scheme,
+     *    and any more difference would require a change in child count anyway. We have at most two sparse nodes.
+     * 2) An internal node of size > 1/2 max can be constructed in exactly the same way, the extra constraint only
+     *    imposed because we must have at least BRANCH_FACTOR/2 children.
+     * 3) A smaller internal node simply uses precisely BRANCH_FACTOR/2 children, and applies the same algorithm.
+     *    This permits up to all children to be sparse, if necessary.
+     *
+     * For branches just above the leaf level, we split more evenly, since there's no advantage to imbalance.
+     */
+    private static <C, I extends C, O extends C> Object[] buildMaximallyDense(BulkIterator<I> source,
+                                                                              int childCount,
+                                                                              int size,
+                                                                              int height,
+                                                                              UpdateFunction<I, O> updateF)
+    {
+        assert height * BRANCH_SHIFT < 32;
 
-        int[] indexOffsets = new int[childNum];
-        int childPos = childNum - 1;
+        int keyCount = childCount - 1;
+        int[] sizeMap = new int[childCount];
+        Object[] branch = new Object[childCount * 2];
 
-        int index = 0;
-        for (int i = 0; i < childNum - 1; i++)
+        if (height == 2)
+        {
+            // we use the _exact same logic_ as below, only we invoke buildLeaf
+            // denseCount: subtract the minimal (sparse) size of the tree and
+            // divide the remainder by the amount extra needed extra per dense

Review comment:
       Very unclear. Perhaps "divide the remainder by the number of additional items per dense"?

##########
File path: src/java/org/apache/cassandra/utils/btree/BTree.java
##########
@@ -1448,4 +1920,2355 @@ private static boolean isStopSentinel(long v)
     {
         return accumulate(btree, accumulator, arg, null, null, initialValue);
     }
+
+    /**
+     * Calculate the minimum height needed for this size of tree
+     *
+     * @param size the tree size
+     * @return the minimum height needed for this size of tree
+     */
+    private static int minHeight(int size)
+    {
+        return heightAtSize2n(size, BRANCH_SHIFT);
+    }
+
+    private static int heightAtSize2n(int size, int branchShift)
+    {
+        //    branch factor                 =  1 <<  branchShift
+        // => full size at height            = (1 << (branchShift * height)) - 1
+        // => full size at height + 1        =  1 << (branchShift * height)
+        // => shift(full size at height + 1) = branchShift * height
+        // => shift(full size at height + 1) / branchShift = height
+        int indexOfHighestBit = 63 - Long.numberOfLeadingZeros(size + 1);
+        indexOfHighestBit += (1L << indexOfHighestBit) == (size + 1) ? 0 : 1;

Review comment:
       The two lines above are the same as `indexOfHighestBit = 64 - Long.numberOfLeadingZeros(size);` which corresponds to the number of bits needed to represent `size` correctly in binary. What we are looking for here is the number of bits needed to represent `size` correctly in base `1<<branchshift` which is obtained from that by the next line (upwards rounded division by `branchShift`).

##########
File path: src/java/org/apache/cassandra/utils/btree/BTree.java
##########
@@ -20,283 +20,547 @@
 
 import java.util.*;
 import java.util.function.BiConsumer;
+import java.util.function.BiFunction;
 import java.util.function.Consumer;
+import java.util.function.Function;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
-import com.google.common.base.Predicate;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Ordering;
 
 import org.apache.cassandra.utils.BiLongAccumulator;
+import org.apache.cassandra.utils.BulkIterator;
 import org.apache.cassandra.utils.LongAccumulator;
 import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.utils.caching.TinyThreadLocalPool;
 
-import static com.google.common.collect.Iterables.concat;
-import static com.google.common.collect.Iterables.filter;
-import static com.google.common.collect.Iterables.transform;
 import static java.lang.Math.max;
 import static java.lang.Math.min;
-import static java.util.Comparator.naturalOrder;
 
 public class BTree
 {
     /**
-     * Leaf Nodes are a raw array of values: Object[V1, V1, ...,].
+     * The {@code BRANCH_FACTOR} is defined as the maximum number of children of each branch, with between
+     * BRANCH_FACTOR/2-1 and BRANCH_FACTOR-1 keys being stored in every node. This yields a minimum tree size of
+     * {@code (BRANCH_FACTOR/2)^height - 1} and a maximum tree size of {@code BRANCH_FACTOR^height - 1}.
      *
-     * Branch Nodes: Object[V1, V2, ..., child[&lt;V1.key], child[&lt;V2.key], ..., child[&lt; Inf], size], where
-     * each child is another node, i.e., an Object[].  Thus, the value elements in a branch node are the
-     * first half of the array (minus one).  In our implementation, each value must include its own key;
-     * we access these via Comparator, rather than directly. 
+     * Branches differ from leaves only in that they contain a suffix region containing the child nodes that occur
+     * either side of the keys, and a sizeMap in the last position, permitting seeking by index within the tree.
+     * Nodes are disambiguated by the length of the array that represents them: an even number is a branch, odd a leaf.
      *
-     * So we can quickly distinguish between leaves and branches, we require that leaf nodes are always an odd number
-     * of elements (padded with a null, if necessary), and branches are always an even number of elements.
+     * Leaf Nodes are represented by an odd-length array of keys, with the final element possibly null, i.e.
+     * Object[V1, V2, ...,null?]
+     *
+     * Branch nodes: Object[V1, V2, ..., child[&lt;V1.key], child[&lt;V2.key], ..., child[&lt; Inf], sizeMap]
+     * Each child is either a branch or leaf, i.e., always an Object[].
+     * The key elements in a branch node occupy the first half of the array (minus one)
      *
      * BTrees are immutable; updating one returns a new tree that reuses unmodified nodes.
      *
-     * There are no references back to a parent node from its children.  (This would make it impossible to re-use
-     * subtrees when modifying the tree, since the modified tree would need new parent references.)
+     * There are no references back to a parent node from its children (this would make it impossible to re-use
+     * subtrees when modifying the tree, since the modified tree would need new parent references).
      * Instead, we store these references in a Path as needed when navigating the tree.
      */
+    public static final int BRANCH_SHIFT = Integer.getInteger("cassandra.btree.branchshift", 5);
 
-    // The maximum fan factor used for B-Trees
-    static final int FAN_SHIFT;
-
-    // The maximun tree size for certain heigth of tree
-    static final int[] TREE_SIZE;
-
-    // NB we encode Path indexes as Bytes, so this needs to be less than Byte.MAX_VALUE / 2
-    static final int FAN_FACTOR;
-
-    static final int MAX_TREE_SIZE = Integer.MAX_VALUE;
-
-    static
-    {
-        int fanfactor = Integer.parseInt(System.getProperty("cassandra.btree.fanfactor", "32"));
-        assert fanfactor >= 2 : "the minimal btree fanfactor is 2";
-        int shift = 1;
-        while (1 << shift < fanfactor)
-            shift += 1;
-
-        FAN_SHIFT = shift;
-        FAN_FACTOR = 1 << FAN_SHIFT;
-
-        // For current FAN_FACTOR, calculate the maximum height of the tree we could build
-        int maxHeight = 0;
-        for (long maxSize = 0; maxSize < MAX_TREE_SIZE; maxHeight++)
-            // each tree node could have (FAN_FACTOR + 1) children,
-            // plus current node could have FAN_FACTOR number of values
-            maxSize = maxSize * (FAN_FACTOR + 1) + FAN_FACTOR;
-
-        TREE_SIZE = new int[maxHeight];
-
-        TREE_SIZE[0] = FAN_FACTOR;
-        for (int i = 1; i < maxHeight - 1; i++)
-            TREE_SIZE[i] = TREE_SIZE[i - 1] * (FAN_FACTOR + 1) + FAN_FACTOR;
-
-        TREE_SIZE[maxHeight - 1] = MAX_TREE_SIZE;
-    }
-
-
-    static final int MINIMAL_NODE_SIZE = FAN_FACTOR >> 1;
+    private static final int BRANCH_FACTOR = 1 << BRANCH_SHIFT;
+    public static final int MIN_KEYS = BRANCH_FACTOR/2 - 1;
+    public static final int MAX_KEYS = BRANCH_FACTOR - 1;
 
     // An empty BTree Leaf - which is the same as an empty BTree
-    static final Object[] EMPTY_LEAF = new Object[1];
+    private static final Object[] EMPTY_LEAF = new Object[1];
 
-    // An empty BTree branch - used only for internal purposes in Modifier
-    static final Object[] EMPTY_BRANCH = new Object[] { null, new int[0] };
-
-    // direction of iteration
-    public static enum Dir
-    {
-        ASC, DESC;
-        public Dir invert() { return this == ASC ? DESC : ASC; }
-        public static Dir asc(boolean asc) { return asc ? ASC : DESC; }
-        public static Dir desc(boolean desc) { return desc ? DESC : ASC; }
-    }
+    private static final int[][] DENSE_SIZE_MAPS = buildBalancedSizeMaps(BRANCH_SHIFT);
+    private static final int[][] SPARSE_SIZE_MAPS = buildBalancedSizeMaps(BRANCH_SHIFT - 1);
+    private static final long[] PERFECT_DENSE_SIZE_ON_HEAP = sizeOnHeapOfPerfectTrees(BRANCH_SHIFT);
+    private static final long[] PERFECT_SPARSE_SIZE_ON_HEAP = sizeOnHeapOfPerfectTrees(BRANCH_SHIFT - 1);
 
     /**
-     * Enables methods to consume the contents of iterators, collections, or arrays without duplicating code or
-     * allocating intermediate objects. Instead of taking an argument that implements an interface, a method takes
-     * an opaque object as the input, and a singleton helper object it uses as an intermediary to access it's contents.
-     * The purpose of doing things this way is to avoid memory allocations on hot paths.
+     * Represents the direction of iteration.
      */
-    private interface IteratingFunction<T>
+    public enum Dir
     {
-        /**
-         * Returns the next object at the given index. This method  must be called with sequentially increasing index
-         * values, starting at 0, and must only be called once per index value. The results of calling this method
-         * without following these rules are undefined.
-         */
-        <K> K nextAt(T input, int idx);
-    }
+        ASC, DESC;
 
-    private static final IteratingFunction<Iterator> ITERATOR_FUNCTION = new IteratingFunction<Iterator>()
-    {
-        public <K> K nextAt(Iterator input, int idx)
+        public Dir invert()
         {
-            return (K) input.next();
+            return this == ASC ? DESC : ASC;
         }
-    };
 
-    private static final IteratingFunction<Object[]> ARRAY_FUNCTION = new IteratingFunction<Object[]>()
-    {
-        public <K> K nextAt(Object[] input, int idx)
+        public static Dir desc(boolean desc)
         {
-            return (K) input[idx];
+            return desc ? DESC : ASC;
         }
-    };
+    }
 
+    /**
+     * Returns an empty BTree
+     *
+     * @return an empty BTree
+     */
     public static Object[] empty()
     {
         return EMPTY_LEAF;
     }
 
+    /**
+     * Create a BTree containing only the specified object
+     *
+     * @return an new BTree containing only the specified object
+     */
     public static Object[] singleton(Object value)
     {
         return new Object[] { value };
     }
 
-    public static <C, K extends C, V extends C> Object[] build(Collection<K> source, UpdateFunction<K, V> updateF)
+    @Deprecated
+    public static <C, K extends C, V extends C> Object[] build(Collection<K> source)
     {
-        return buildInternal(source.iterator(), ITERATOR_FUNCTION, source.size(), updateF);
+        return build(source, UpdateFunction.noOp());
     }
 
-    /**
-     * Creates a BTree containing all of the objects in the provided collection
-     *
-     * @param source  the items to build the tree with. MUST BE IN STRICTLY ASCENDING ORDER.
-     * @param size    the size of the source iterable
-     * @return        a btree representing the contents of the provided iterable
-     */
-    public static <C, K extends C, V extends C> Object[] build(Iterable<K> source, int size, UpdateFunction<K, V> updateF)
+    @Deprecated
+    public static <C, K extends C, V extends C> Object[] build(Collection<K> source, UpdateFunction<K, V> updateF)
     {
-        if (size < 0)
-            throw new IllegalArgumentException(Integer.toString(size));
-        return buildInternal(source.iterator(), ITERATOR_FUNCTION, size, updateF);
+        return build(BulkIterator.of(source.iterator()), source.size(), updateF);
     }
 
-    public static <C, K extends C, V extends C> Object[] build(Object[] source, int size, UpdateFunction<K, V> updateF)
+    public static <C, I extends C, O extends C> Object[] build(BulkIterator<I> source, int size, UpdateFunction<I, O> updateF)
     {
-        if (size < 0)
-            throw new IllegalArgumentException(Integer.toString(size));
-        return buildInternal(source, ARRAY_FUNCTION, size, updateF);
+        assert size >= 0;
+        if (size == 0)
+            return EMPTY_LEAF;
+
+        if (size <= MAX_KEYS)
+            return buildLeaf(source, size, updateF);
+
+        return buildRoot(source, size, updateF);
     }
 
-    private static <C, K extends C, V extends C, S> Object[] buildLeaf(S source, IteratingFunction<S> iterFunc, int size, int startIdx, UpdateFunction<K, V> updateF)
+    /**
+     * Build a leaf with {@code size} elements taken in bulk from {@code insert}, and apply {@code updateF} to these elements
+     */
+    private static <C, I extends C, O extends C> Object[] buildLeaf(BulkIterator<I> insert,
+                                                                    int size,
+                                                                    UpdateFunction<I, O> updateF)
     {
-        V[] values = (V[]) new Object[size | 1];
+        Object[] values = new Object[size | 1]; // ensure that we have an odd-length array
+        insert.fetch(values, 0, size);
+        if (!isSimple(updateF))
+        {
+            updateF.onAllocated(ObjectSizes.sizeOfReferenceArray(values.length));
+            for (int i = 0; i < size; i++)
+                values[i] = updateF.apply((I) values[i]);
+        }
+        return values;
+    }
 
-        int idx = startIdx;
-        for (int i = 0; i < size; i++)
+    /**
+     * Build a leaf with {@code size} elements taken in bulk from {@code insert}, and apply {@code updateF} to these elements
+     * Do not invoke {@code updateF.onAllocated}.  Used by {@link #buildMaximallyDense} and {@link #buildPerfect} which
+     * track the size for the entire tree they build in order to save on work.
+     */
+    private static <C, I extends C, O extends C> Object[] buildLeafWithoutSizeTracking(BulkIterator<I> insert, int size, UpdateFunction<I, O> updateF)
+    {
+        Object[] values = new Object[size | 1]; // ensure that we have an odd-length array
+        insert.fetch(values, 0, size);
+        if (!isSimple(updateF))
         {
-            K k = iterFunc.nextAt(source, idx);
-            values[i] = updateF.apply(k);
-            idx++;
+            for (int i = 0; i < size; i++)
+                values[i] = updateF.apply((I) values[i]);
         }
-        if (updateF != UpdateFunction.<K>noOp())
-            updateF.allocated(ObjectSizes.sizeOfArray(values));
         return values;
     }
 
-    private static <C, K extends C, V extends C, S> Object[] buildInternal(S source, IteratingFunction<S> iterFunc, int size, int level, int startIdx, UpdateFunction<K, V> updateF)
+    /**
+     * Build a root node from the first {@code size} elements from {@code source}, applying {@code updateF} to those elements.
+     * A root node is permitted to have as few as two children, if a branch (i.e. if {@code size > MAX_SIZE}.
+     */
+    private static <C, I extends C, O extends C> Object[] buildRoot(BulkIterator<I> source, int size, UpdateFunction<I, O> updateF)
     {
-        assert size > 0;
-        assert level >= 0;
-        if (level == 0)
-            return buildLeaf(source, iterFunc, size, startIdx, updateF);
+        // first calculate the minimum height needed for this size of tree
+        int height = minHeight(size);
+        assert height > 1;
 
-        // calcuate child num: (size - (childNum - 1)) / maxChildSize <= childNum
-        int childNum = size / (TREE_SIZE[level - 1] + 1) + 1;
+        int denseChildSize = denseSize(height - 1);
+        int childCount = size / (denseChildSize + 1) + 1;
 
-        V[] values = (V[]) new Object[childNum * 2];
-        if (updateF != UpdateFunction.<K>noOp())
-            updateF.allocated(ObjectSizes.sizeOfArray(values));
+        return buildMaximallyDense(source, childCount, size, height, updateF);
+    }
+
+    /**
+     * Build a tree containing some initial quantity of dense nodes, up to a single node of arbitrary size (between
+     * dense and sparse), and the remainder of nodes all sparse.
+     *
+     * This permits us to build any size of tree:
+     * 1) A root node can have any number of children, so for a given height we can build our smallest tree with two
+     *    sparse nodes.  For a given childCount we can vary up to a whole dense child's size with this scheme,
+     *    and any more difference would require a change in child count anyway. We have at most two sparse nodes.
+     * 2) An internal node of size > 1/2 max can be constructed in exactly the same way, the extra constraint only
+     *    imposed because we must have at least BRANCH_FACTOR/2 children.
+     * 3) A smaller internal node simply uses precisely BRANCH_FACTOR/2 children, and applies the same algorithm.
+     *    This permits up to all children to be sparse, if necessary.
+     *
+     * For branches just above the leaf level, we split more evenly, since there's no advantage to imbalance.
+     */
+    private static <C, I extends C, O extends C> Object[] buildMaximallyDense(BulkIterator<I> source,
+                                                                              int childCount,
+                                                                              int size,
+                                                                              int height,
+                                                                              UpdateFunction<I, O> updateF)
+    {
+        assert height * BRANCH_SHIFT < 32;
 
-        int[] indexOffsets = new int[childNum];
-        int childPos = childNum - 1;
+        int keyCount = childCount - 1;
+        int[] sizeMap = new int[childCount];
+        Object[] branch = new Object[childCount * 2];
 
-        int index = 0;
-        for (int i = 0; i < childNum - 1; i++)
+        if (height == 2)
+        {
+            // we use the _exact same logic_ as below, only we invoke buildLeaf
+            // denseCount: subtract the minimal (sparse) size of the tree and
+            // divide the remainder by the amount extra needed extra per dense
+            int denseCount = (size - (childCount * MIN_KEYS + keyCount + 1)) / (1+MIN_KEYS);
+            int i = 0;
+            while (i < denseCount)
+            {
+                sizeMap[i] = i * (1+MAX_KEYS) + MAX_KEYS;
+                branch[keyCount + i] = buildLeafWithoutSizeTracking(source, MAX_KEYS, updateF);
+                branch[i++] = isSimple(updateF) ? source.next() : updateF.apply(source.next());
+            }
+            int sparseCount = childCount - (1 + denseCount);
+            int remainderSize = size - (denseCount * (1+MAX_KEYS) + sparseCount * (1+MIN_KEYS));
+            {
+                branch[keyCount + i] = buildLeafWithoutSizeTracking(source, remainderSize, updateF);
+                sizeMap[i] = size = i * (1+MAX_KEYS) + remainderSize;

Review comment:
       We shouldn't reuse `size` for this.
   
   While we're at it, this code can be simplified if we keep track of the constructed size throughout.

##########
File path: src/java/org/apache/cassandra/utils/btree/BTree.java
##########
@@ -1448,4 +1920,2355 @@ private static boolean isStopSentinel(long v)
     {
         return accumulate(btree, accumulator, arg, null, null, initialValue);
     }
+
+    /**
+     * Calculate the minimum height needed for this size of tree
+     *
+     * @param size the tree size
+     * @return the minimum height needed for this size of tree
+     */
+    private static int minHeight(int size)
+    {
+        return heightAtSize2n(size, BRANCH_SHIFT);
+    }
+
+    private static int heightAtSize2n(int size, int branchShift)
+    {
+        //    branch factor                 =  1 <<  branchShift
+        // => full size at height            = (1 << (branchShift * height)) - 1
+        // => full size at height + 1        =  1 << (branchShift * height)
+        // => shift(full size at height + 1) = branchShift * height
+        // => shift(full size at height + 1) / branchShift = height
+        int indexOfHighestBit = 63 - Long.numberOfLeadingZeros(size + 1);
+        indexOfHighestBit += (1L << indexOfHighestBit) == (size + 1) ? 0 : 1;
+        return (branchShift - 1 + indexOfHighestBit) / branchShift;
+    }
+
+    private static int[][] buildBalancedSizeMaps(int branchShift)
+    {
+        int count = (32 / branchShift) - 1;
+        int childCount = 1 << branchShift;
+        int[][] sizeMaps = new int[count][childCount];
+        for (int height = 0 ; height < count ; ++height)
+        {
+            int childSize = treeSize2n(height + 1, branchShift);
+            int size = 0;
+            int[] sizeMap = sizeMaps[height];
+            for (int i = 0 ; i < childCount ; ++i)
+            {
+                sizeMap[i] = size += childSize;
+                size += 1;
+            }
+        }
+        return sizeMaps;
+    }
+
+    // simply utility to reverse the contents of array[from..to)
+    private static void reverse(Object[] array, int from, int to)
+    {
+        int mid = (from + to) / 2;
+        for (int i = from ; i < mid ; i++)
+        {
+            int j = to - (1 + i - from);
+            Object tmp = array[i];
+            array[i] = array[j];
+            array[j] = tmp;
+        }
+    }
+
+    // simply utility to reverse the contents of array[from..to)
+    private static void reverse(int[] array, int from, int to)
+    {
+        int mid = (from + to) / 2;
+        for (int i = from ; i < mid ; i++)
+        {
+            int j = to - (1 + i - from);
+            int tmp = array[i];
+            array[i] = array[j];
+            array[j] = tmp;
+        }
+    }
+
+    /**
+     * Mutate an array of child sizes into a cumulative sizeMap, returning the total size
+     */
+    private static int sizesToSizeMap(int[] sizeMap)
+    {
+        int total = sizeMap[0];
+        for (int i = 1 ; i < sizeMap.length ; ++i)
+            sizeMap[i] = total += 1 + sizeMap[i];
+        return total;
+    }
+
+    private static int sizesToSizeMap(int[] sizes, int count)
+    {
+        int total = sizes[0];
+        for (int i = 1 ; i < count ; ++i)
+            sizes[i] = total += 1 + sizes[i];
+        return total;
+    }
+
+    /**
+     * Mutate an array of child sizes into a cumulative sizeMap, returning the total size
+     */
+    private static void sizeMapToSizes(int[] sizeMap)
+    {
+        for (int i = sizeMap.length ; i > 1 ; --i)
+            sizeMap[i] -= 1 + sizeMap[i - 1];
+    }
+
+    /**
+     * A simple utility method to handle a null upper bound that we treat as infinity
+     */
+    private static <Compare> int compareWithMaybeInfinity(Comparator<? super Compare> comparator, Compare key, Compare ub)
+    {
+        if (ub == null)
+            return -1;
+        return comparator.compare(key, ub);
+    }
+
+    /**
+     * Perform {@link #exponentialSearch} on {@code in[from..to)}, treating a {@code find} of {@code null} as infinity.
+     */
+    static <Compare> int exponentialSearchForMaybeInfinity(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare find)
+    {
+        if (find == null)
+            return -(1 + to);
+        return exponentialSearch(comparator, in, from, to, find);
+    }
+
+    /**
+     * Equivalent to {@link Arrays#binarySearch}, only more efficient algorithmically for linear merges.
+     * Binary search has worst case complexity {@code O(n.lg n)} for a linear merge, whereas exponential search
+     * has a worst case of {@code O(n)}. However compared to a simple linear merge, the best case for exponential
+     * search is {@code O(lg(n))} instead of {@code O(n)}.
+     */
+    private static <Compare> int exponentialSearch(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare find)
+    {
+        int step = 0;
+        while (from + step < to)
+        {
+            int i = from + step;
+            int c = comparator.compare(find, (Compare) in[i]);
+            if (c < 0)
+            {
+                to = i;
+                break;
+            }
+            if (c == 0)
+                return i;
+            from = i + 1;
+            step = step * 2 + 1; // jump in perfect binary search increments
+        }
+        return Arrays.binarySearch((Compare[]) in, from, to, find, comparator);
+    }
+
+    /**
+     * Perform {@link #exponentialSearch} on {@code in[from..to)}; if the value falls outside of the range of these
+     * elements, test against {@code ub} as though it occurred at position {@code to}
+     *
+     * @return same as {@link Arrays#binarySearch} if {@code find} occurs in the range {@code [in[from]..in[to])};
+     * otherwise the insertion position {@code -(1+to)} if {@code find} is less than {@code ub}, and {@code -(2+t)}
+     * if it is greater than or equal to.
+     *
+     * {@code ub} may be {@code null}, representing infinity.
+     */
+    static <Compare> int exponentialSearchWithUpperBound(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare ub, Compare find)
+    {
+        int step = 0;
+        while (true)
+        {
+            int i = from + step;
+            if (i >= to)
+            {
+                int c = compareWithMaybeInfinity(comparator, find, ub);
+                if (c >= 0)
+                    return -(2 + to);
+                break;
+            }
+            int c = comparator.compare(find, (Compare) in[i]);
+            if (c < 0)
+            {
+                to = i;
+                break;
+            }
+            if (c == 0)
+                return i;
+            from = i + 1;
+            step = step * 2 + 1; // jump in perfect binary search increments
+        }
+        return Arrays.binarySearch((Compare[]) in, from, to, find, comparator);
+    }
+    
+    /**
+     * Compute the size-in-bytes of full trees of cardinality {@code branchFactor^height - 1}
+     */
+    private static long[] sizeOnHeapOfPerfectTrees(int branchShift)
+    {
+        long[] result = new long[heightAtSize2n(Integer.MAX_VALUE, branchShift)];
+        int branchFactor = 1 << branchShift;
+        result[0] = branchFactor - 1;
+        for (int i = 1 ; i < result.length ; ++i)
+            result[i] = sizeOnHeapOfPerfectTree(i + 1, branchFactor);
+        return result;
+    }
+
+    /**
+     * Compute the size-in-bytes of a full tree of cardinality {@code branchFactor^height - 1}
+     * TODO: test
+     */
+    private static long sizeOnHeapOfPerfectTree(int height, int branchShift)
+    {
+        int branchFactor = 1 << branchShift;
+        long branchSize = ObjectSizes.sizeOfReferenceArray(branchFactor * 2);
+        int branchCount = height == 2 ? 1 : 2 + treeSize2n(height - 2, branchShift);
+        long leafSize = ObjectSizes.sizeOfReferenceArray((branchFactor - 1)|1);
+        int leafCount = 1 + treeSize2n(height - 1, branchShift);
+        return (branchSize * branchCount) + (leafSize * leafCount);
+    }
+
+    /**
+     * @return the actual height of {@code tree}
+     */
+    public static int height(Object[] tree)
+    {
+        if (isLeaf(tree))
+            return 1;
+
+        int height = 1;
+        while (!isLeaf(tree))
+        {
+            height++;
+            tree = (Object[]) tree[shallowSizeOfBranch(tree)];
+        }
+        return height;
+    }
+
+    /**
+     * @return the minimum representable size at {@code height}, for an internal (non-root) node.
+     */
+    private static int sparseSize(int height)
+    {
+        return treeSize2n(height, BRANCH_SHIFT - 1);
+    }
+
+    /**
+     * @return the maximum representable size at {@code height}.
+     */
+    private static int denseSize(int height)
+    {
+        return treeSize2n(height, BRANCH_SHIFT);
+    }
+
+    /**
+     * @return the maximum representable size at {@code height}.
+     */
+    private static int checkedDenseSize(int height)
+    {
+        assert height * BRANCH_SHIFT < 32;
+        return denseSize(height);
+    }
+
+    /**
+     * Computes the number of nodes in a full tree of height {@code height}
+     * and with {@code 2^branchShift} branch factor.
+     * i.e. computes {@code (2^branchShift)^height - 1}
+     */
+    private static int treeSize2n(int height, int branchShift)
+    {
+        return (1 << (branchShift * height)) - 1;
+    }
+
+    // TODO: test
+    private static int maxRootHeight(int size)
+    {
+        if (size <= BRANCH_FACTOR)
+            return 1;
+        return 1 + heightAtSize2n((size - 1) / 2, BRANCH_SHIFT - 1);
+    }
+    
+    private static int sizeOfBranch(Object[] branch)
+    {
+        int length = branch.length;
+        // length - 1 == getChildEnd == getPositionOfSizeMap
+        // (length / 2) - 1 == getChildCount - 1 == position of full tree size
+        // hard code this, as will be used often;
+        return ((int[]) branch[length - 1])[(length / 2) - 1];
+    }
+
+    /**
+     * Checks if the UpdateFunction is an instance of {@code UpdateFunction.Simple}.
+     */
+    private static boolean isSimple(UpdateFunction<?, ?> updateF)
+    {
+        return updateF instanceof UpdateFunction.Simple;
+    }
+
+    /**
+     * @return the size map for the branch node
+     */
+    static int[] sizeMap(Object[] branch)
+    {
+        return (int[]) branch[branch.length - 1];
+    }
+
+    public static long sizeOnHeapOf(Object[] tree)
+    {
+        long size = ObjectSizes.sizeOfArray(tree);
+        if (isLeaf(tree))
+            return size;
+        for (int i = childOffset(tree); i < childEndOffset(tree) ; i++)
+            size += sizeOnHeapOf((Object[]) tree[i]);
+        size += ObjectSizes.sizeOfArray(sizeMap(tree)); // may overcount, since we share size maps
+        return size;
+    }
+
+    // Arbitrary boundaries
+    private static Object POSITIVE_INFINITY = new Object();
+    private static Object NEGATIVE_INFINITY = new Object();
+
+    /** simple static wrapper to calls to cmp.compare() which checks if either a or b are Special (i.e. represent an infinity) */
+    private static <V> int compareWellFormed(Comparator<V> cmp, Object a, Object b)
+    {
+        if (a == b)
+            return 0;
+        if (a == NEGATIVE_INFINITY | b == POSITIVE_INFINITY)
+            return -1;
+        if (b == NEGATIVE_INFINITY | a == POSITIVE_INFINITY)
+            return 1;
+        return cmp.compare((V) a, (V) b);
+    }
+
+    public static boolean isWellFormed(Object[] btree, Comparator<?> cmp)
+    {
+        return isWellFormed(cmp, btree, true, NEGATIVE_INFINITY, POSITIVE_INFINITY);
+    }
+
+    private static boolean isWellFormed(Comparator<?> cmp, Object[] node, boolean isRoot, Object min, Object max)
+    {
+        if (isEmpty(node))
+            return true;
+
+        if (cmp != null && !isNodeWellFormed(cmp, node, min, max))
+            return false;
+
+        int keyCount = shallowSize(node);
+        if (keyCount < 1)
+            return false;
+        if (!isRoot && keyCount < BRANCH_FACTOR/2 - 1)
+            return false;
+        if (keyCount >= BRANCH_FACTOR)
+            return false;
+
+        if (isLeaf(node))
+            return true;
+
+        int[] sizeMap = sizeMap(node);
+        int type = 0;
+        int size = 0;
+        // compare each child node with the branch element at the head of this node it corresponds with
+        for (int i = childOffset(node); i < childEndOffset(node) ; i++)
+        {
+            Object[] child = (Object[]) node[i];
+            Object localmax = i < node.length - 2 ? node[i - childOffset(node)] : max;
+            if (!isWellFormed(cmp, child, false, min, localmax))
+                return false;
+
+            type |= isLeaf(child) ? 1 : 2;
+            min = localmax;
+            size += size(child);
+            if (sizeMap[i - childOffset(node)] != size)
+                return false;
+            size += 1;
+        }
+
+        if (type >= 3)
+            return false;
+        return true; // either all leaves or all branches but not a mix
+    }
+
+    private static boolean isNodeWellFormed(Comparator<?> cmp, Object[] node, Object min, Object max)
+    {
+        Object previous = min;
+        int end = shallowSize(node);
+        for (int i = 0; i < end; i++)
+        {
+            Object current = node[i];
+            if (compareWellFormed(cmp, previous, current) >= 0)
+                return false;
+
+            previous = current;
+        }
+        return compareWellFormed(cmp, previous, max) < 0;
+    }
+
+    /**
+     * Build a tree of unknown size, in order.
+     *
+     * Can be used with {@link #reverseInSitu} to build a tree in reverse.
+     */
+    public static <V> FastBuilder<V> fastBuilder()
+    {
+        TinyThreadLocalPool.TinyPool<FastBuilder<?>> pool = FastBuilder.POOL.get();
+        FastBuilder<V> builder = (FastBuilder<V>) pool.poll();
+        if (builder == null)
+            builder = new FastBuilder<>();
+        builder.pool = pool;
+        return builder;
+    }
+
+    /**
+     * Base class for AbstractFastBuilder.BranchBuilder, LeafBuilder and AbstractFastBuilder,
+     * containing shared behaviour and declaring some useful abstract methods.
+     */
+    private static abstract class LeafOrBranchBuilder
+    {
+        final int height;
+        final LeafOrBranchBuilder child;
+        BranchBuilder parent;
+
+        /**
+         * The current buffer contents (if any) of the leaf or branch - always sized to contain a complete
+         * node of the form being constructed.  Always non-null, except briefly during overflow.
+         */
+        Object[] buffer;
+        /**
+         * The number of keys in our buffer, whether or not we are building a leaf or branch; if we are building
+         * a branch, we will ordinarily have the same number of children as well, except temporarily when finishing
+         * the construction of the node.
+         */
+        int count;
+
+        /**
+         * either
+         *  1) an empty leftover buffer from a past usage, which can be used when we exhaust {@code buffer}; or
+         *  2) a full {@code buffer} that has been parked until we next overflow, so we can steal some back
+         *     if we finish before reaching MIN_KEYS in {@code buffer}
+         */
+        Object[] savedBuffer;
+        /**
+         * The key we overflowed on when populating savedBuffer.  If null, {@link #savedBuffer} is logically empty.
+         */
+        Object savedNextKey;
+
+        LeafOrBranchBuilder(LeafOrBranchBuilder child)
+        {
+            this.height = child == null ? 1 : 1 + child.height;
+            this.child = child;
+        }
+
+        /**
+         * Do we have enough keys in the builder to construct at least one balanced node?
+         * We could have enough to build two.
+         */
+        final boolean isSufficient()
+        {
+            return hasOverflow() || count >= MIN_KEYS;
+        }
+
+        /**
+         * Do we have an already constructed node saved, that we can propagate or redistribute?
+         * This implies we are building two nodes, since {@link #savedNextKey} would overflow {@link #savedBuffer}
+         */
+        final boolean hasOverflow()
+        {
+            return savedNextKey != null;
+        }
+
+        /**
+         * Do we have an already constructed node saved AND insufficient keys in our buffer, so
+         * that we need to share the contents of {@link #savedBuffer} with {@link #buffer} to construct
+         * our results?
+         */
+        final boolean mustRedistribute()
+        {
+            return hasOverflow() && count < MIN_KEYS;
+        }
+
+        /**
+         * Are we empty, i.e. we have no contents in either {@link #buffer} or {@link #savedBuffer}
+         */
+        final boolean isEmpty()
+        {
+            return count == 0 && savedNextKey == null;
+        }
+
+        /**
+         * Drain the contents of this builder and build up to two nodes, as necessary.
+         * If {@code unode != null} and we are building a single node that is identical to it, use {@code unode} instead.
+         * If {@code propagateTo != null} propagate any nodes we build to it.
+         * @return the last node we construct
+         */
+        abstract Object[] drainAndPropagate(Object[] unode, int usz, BranchBuilder propagateTo);
+
+        /**
+         * Drain the contents of this builder and build at most one node.
+         * Requires {@code !hasOverflow()}
+         * @return the node we construct
+         */
+        abstract Object[] drain();
+
+        /**
+         * Takes a node that would logically occur directly preceding the current buffer contents,
+         * and the key that would separate them in a parent node, and prepends their contents
+         * to the current buffer's contents.  This can be used to redistribute already-propagated
+         * contents to a parent in cases where this is convenient (i.e. when transforming)
+         *
+         * @param predecessor directly preceding node
+         * @param predecessorNextKey key that would have separated predecessor from buffer contents
+         */
+        abstract void prepend(Object[] predecessor, Object predecessorNextKey);
+
+        /**
+         * Indicates if this builder produces dense nodes, i.e. those that are populated with MAX_KEYS
+         * at every level.  Only the last two children of any branch may be non-dense, and in some cases only
+         * the last two nodes in any tier of the tree.
+         *
+         * This flag switches whether or not we maintain a buffer of sizes, or use the globally shared contents of
+         * DENSE_SIZE_MAPS.
+         */
+        abstract boolean producesOnlyDense();
+
+        /**
+         * Ensure there is a {@code branch.parent}, and return it
+         */
+        final BranchBuilder ensureParent()
+        {
+            if (parent == null)
+                parent = new BranchBuilder(this);
+            parent.touched = true;
+            return parent;
+        }
+
+        /**
+         * Mark a branch builder as utilised, so that we must clear it when resetting any {@link AbstractFastBuilder}
+         * @return {@code branch}
+         */
+        static BranchBuilder touch(BranchBuilder branch)
+        {
+            branch.touched = true;
+            return branch;
+        }
+
+        /**
+         * @return the parent builder iff we will overflow or the parent has already been touched
+         */
+        final BranchBuilder parentIfInUse()
+        {
+            // if we have enough data to propagate, ensure there is a parent and return it
+            if (hasOverflow())
+                return ensureParent();
+
+            // otherwise return it only if it's already in use
+            return parent == null || !parent.touched ? null : parent;
+        }
+
+        /**
+         * A utility method for comparing a range of two arrays
+         */
+        static boolean areIdentical(Object[] a, int aOffset, Object[] b, int bOffset, int count)
+        {
+            for (int i = 0 ; i < count ; ++i)
+            {
+                if (a[i + aOffset] != b[i + bOffset])
+                    return false;
+            }
+            return true;
+        }
+
+        /**
+         * A utility method for comparing a range of two arrays
+         */
+        static boolean areIdentical(int[] a, int aOffset, int[] b, int bOffset, int count)
+        {
+            for (int i = 0 ; i < count ; ++i)
+            {
+                if (a[i + aOffset] != b[i + bOffset])
+                    return false;
+            }
+            return true;
+        }
+    }
+
+    /**
+     * LeafBuilder for methods pertaining specifically to building a leaf in an {@link AbstractFastBuilder}.
+     * Note that {@link AbstractFastBuilder} extends this class directly, however it is convenient to maintain
+     * distinct classes in the hierarchy for clarity of behaviour and intent.
+     */
+    private static abstract class LeafBuilder extends LeafOrBranchBuilder
+    {
+        long allocated;
+
+        LeafBuilder()
+        {
+            super(null);
+            buffer = new Object[MAX_KEYS];
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer, overflowing if necessary
+         */
+        public void addKey(Object nextKey)
+        {
+            if (count == MAX_KEYS)
+                overflow(nextKey);
+            else
+                buffer[count++] = nextKey;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void addKeyNoOverflow(Object nextKey)
+        {
+            buffer[count++] = nextKey;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void maybeAddKeyNoOverflow(Object nextKey)
+        {
+            buffer[count] = nextKey;
+            count += nextKey != null ? 1 : 0;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void maybeAddKey(Object nextKey)
+        {
+            if (count == MAX_KEYS)
+            {
+                if (nextKey != null)
+                    overflow(nextKey);
+            }
+            else
+            {
+                buffer[count] = nextKey;
+                count += nextKey != null ? 1 : 0;
+            }
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}, overflowing as necessary.
+         */
+        void copy(Object[] source, int offset, int length)
+        {
+            if (count + length > MAX_KEYS)
+            {
+                int copy = MAX_KEYS - count;
+                System.arraycopy(source, offset, buffer, count, copy);
+                offset += copy;
+//              implicitly:  count = MAX_KEYS;
+                overflow(source[offset++]);
+                length -= 1 + copy;
+            }
+            System.arraycopy(source, offset, buffer, count, length);
+            count += length;
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}; the caller specifying overflow is unnecessary
+         */
+        void copyNoOverflow(Object[] source, int offset, int length)
+        {
+            System.arraycopy(source, offset, buffer, count, length);
+            count += length;
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}, overflowing as necessary.
+         * Applies {@code updateF} to the contents before insertion.
+         */
+        <Insert, Existing> void copy(Object[] source, int offset, int length, UpdateFunction<Insert, Existing> updateF)
+        {
+            if (isSimple(updateF))
+            {
+                copy(source, offset, length);
+                return;
+            }
+
+            if (count + length > MAX_KEYS)
+            {
+                int copy = MAX_KEYS - count;
+                for (int i = 0 ; i < copy ; ++i)
+                    buffer[count + i] = updateF.apply((Insert) source[offset + i]);
+                offset += copy;
+//              implicitly:  leaf().count = MAX_KEYS;
+                overflow(updateF.apply((Insert) source[offset++]));
+                length -= 1 + copy;
+            }
+            for (int i = 0 ; i < length ; ++i)
+                buffer[count + i] = updateF.apply((Insert) source[offset + i]);
+            count += length;
+        }
+
+        /**
+         * {@link #buffer} is full, and we need to make room either by populating {@link #savedBuffer},
+         * propagating its current contents, if any, to {@link #parent}
+         */
+        void overflow(Object nextKey)
+        {
+            if (hasOverflow())
+                propagateOverflow();
+
+            // precondition: count == MAX_KEYS and savedNextKey == null
+
+            Object[] newBuffer = savedBuffer;
+            if (newBuffer == null)
+                newBuffer = new Object[MAX_KEYS];
+
+            savedBuffer = buffer;
+            savedNextKey = nextKey;
+            buffer = newBuffer;
+            count = 0;
+        }
+
+        /**
+         * Redistribute the contents of {@link #savedBuffer} into {@link #buffer}, finalise {@link #savedBuffer} and flush upwards.
+         * Invoked when we are building from {@link #buffer}, have insufficient values but a complete leaf in {@link #savedBuffer}
+         * @return the size of the leaf we flushed to our parent from {@link #savedBuffer}
+         */
+        Object[] redistributeOverflowAndDrain()
+        {
+            Object[] newLeaf = redistributeAndDrain(savedBuffer, MAX_KEYS, savedNextKey);
+            savedNextKey = null;
+            return newLeaf;
+        }
+
+        /**
+         * Redistribute the contents of {@link #buffer} and an immediate predecessor into a new leaf,
+         * then construct a new predecessor with the remaining contents and propagate up to our parent
+         * Invoked when we are building from {@link #buffer}, have insufficient values but either a complete
+         * leaf in {@link #savedBuffer} or can exfiltrate one from our parent to redistribute.
+         *
+         * @return the second of the two new leaves
+         */
+        Object[] redistributeAndDrain(Object[] pred, int predSize, Object predNextKey)
+        {
+            // precondition: savedLeafCount == MAX_KEYS && leaf().count < MIN_KEYS
+            // ensure we have at least MIN_KEYS in leaf().buffer
+            // first shift leaf().buffer and steal some keys from leaf().savedBuffer and leaf().savedBufferNextKey
+            int steal = MIN_KEYS - count;
+            Object[] newLeaf = new Object[MIN_KEYS];
+            System.arraycopy(pred,  predSize - (steal - 1), newLeaf, 0,     steal - 1);
+                                                            newLeaf [steal - 1] = predNextKey;
+            System.arraycopy(buffer, 0,                     newLeaf, steal, count);
+
+            // then create a leaf out of the remainder of savedBuffer
+            int newPredecessorCount = predSize - steal;
+            Object[] newPredecessor = new Object[newPredecessorCount | 1];
+            System.arraycopy(pred, 0, newPredecessor, 0, newPredecessorCount);
+            if (allocated >= 0)
+                allocated += ObjectSizes.sizeOfReferenceArray(newPredecessorCount | 1);
+            ensureParent().addChildAndNextKey(newPredecessor, newPredecessorCount, pred[newPredecessorCount]);
+            return newLeaf;
+        }
+
+        /**
+         * Invoked to fill our {@link #buffer} to >= MIN_KEYS with data ocurring before {@link #buffer};
+         * possibly instead fills {@link #savedBuffer}
+         *
+         * @param pred directly preceding node
+         * @param predNextKey key that would have separated predecessor from buffer contents
+         */
+        void prepend(Object[] pred, Object predNextKey)

Review comment:
       What are the semantics if there already is overflow? Shouldn't we add "not already overflowed" as a precondition?

##########
File path: src/java/org/apache/cassandra/utils/btree/BTree.java
##########
@@ -1448,4 +1920,2355 @@ private static boolean isStopSentinel(long v)
     {
         return accumulate(btree, accumulator, arg, null, null, initialValue);
     }
+
+    /**
+     * Calculate the minimum height needed for this size of tree
+     *
+     * @param size the tree size
+     * @return the minimum height needed for this size of tree
+     */
+    private static int minHeight(int size)
+    {
+        return heightAtSize2n(size, BRANCH_SHIFT);
+    }
+
+    private static int heightAtSize2n(int size, int branchShift)
+    {
+        //    branch factor                 =  1 <<  branchShift
+        // => full size at height            = (1 << (branchShift * height)) - 1
+        // => full size at height + 1        =  1 << (branchShift * height)
+        // => shift(full size at height + 1) = branchShift * height
+        // => shift(full size at height + 1) / branchShift = height
+        int indexOfHighestBit = 63 - Long.numberOfLeadingZeros(size + 1);
+        indexOfHighestBit += (1L << indexOfHighestBit) == (size + 1) ? 0 : 1;
+        return (branchShift - 1 + indexOfHighestBit) / branchShift;
+    }
+
+    private static int[][] buildBalancedSizeMaps(int branchShift)
+    {
+        int count = (32 / branchShift) - 1;
+        int childCount = 1 << branchShift;
+        int[][] sizeMaps = new int[count][childCount];
+        for (int height = 0 ; height < count ; ++height)
+        {
+            int childSize = treeSize2n(height + 1, branchShift);
+            int size = 0;
+            int[] sizeMap = sizeMaps[height];
+            for (int i = 0 ; i < childCount ; ++i)
+            {
+                sizeMap[i] = size += childSize;
+                size += 1;
+            }
+        }
+        return sizeMaps;
+    }
+
+    // simply utility to reverse the contents of array[from..to)
+    private static void reverse(Object[] array, int from, int to)
+    {
+        int mid = (from + to) / 2;
+        for (int i = from ; i < mid ; i++)
+        {
+            int j = to - (1 + i - from);
+            Object tmp = array[i];
+            array[i] = array[j];
+            array[j] = tmp;
+        }
+    }
+
+    // simply utility to reverse the contents of array[from..to)
+    private static void reverse(int[] array, int from, int to)
+    {
+        int mid = (from + to) / 2;
+        for (int i = from ; i < mid ; i++)
+        {
+            int j = to - (1 + i - from);
+            int tmp = array[i];
+            array[i] = array[j];
+            array[j] = tmp;
+        }
+    }
+
+    /**
+     * Mutate an array of child sizes into a cumulative sizeMap, returning the total size
+     */
+    private static int sizesToSizeMap(int[] sizeMap)
+    {
+        int total = sizeMap[0];
+        for (int i = 1 ; i < sizeMap.length ; ++i)
+            sizeMap[i] = total += 1 + sizeMap[i];
+        return total;
+    }
+
+    private static int sizesToSizeMap(int[] sizes, int count)
+    {
+        int total = sizes[0];
+        for (int i = 1 ; i < count ; ++i)
+            sizes[i] = total += 1 + sizes[i];
+        return total;
+    }
+
+    /**
+     * Mutate an array of child sizes into a cumulative sizeMap, returning the total size
+     */
+    private static void sizeMapToSizes(int[] sizeMap)
+    {
+        for (int i = sizeMap.length ; i > 1 ; --i)
+            sizeMap[i] -= 1 + sizeMap[i - 1];
+    }
+
+    /**
+     * A simple utility method to handle a null upper bound that we treat as infinity
+     */
+    private static <Compare> int compareWithMaybeInfinity(Comparator<? super Compare> comparator, Compare key, Compare ub)
+    {
+        if (ub == null)
+            return -1;
+        return comparator.compare(key, ub);
+    }
+
+    /**
+     * Perform {@link #exponentialSearch} on {@code in[from..to)}, treating a {@code find} of {@code null} as infinity.
+     */
+    static <Compare> int exponentialSearchForMaybeInfinity(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare find)
+    {
+        if (find == null)
+            return -(1 + to);
+        return exponentialSearch(comparator, in, from, to, find);
+    }
+
+    /**
+     * Equivalent to {@link Arrays#binarySearch}, only more efficient algorithmically for linear merges.
+     * Binary search has worst case complexity {@code O(n.lg n)} for a linear merge, whereas exponential search
+     * has a worst case of {@code O(n)}. However compared to a simple linear merge, the best case for exponential
+     * search is {@code O(lg(n))} instead of {@code O(n)}.
+     */
+    private static <Compare> int exponentialSearch(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare find)
+    {
+        int step = 0;
+        while (from + step < to)
+        {
+            int i = from + step;
+            int c = comparator.compare(find, (Compare) in[i]);
+            if (c < 0)
+            {
+                to = i;
+                break;
+            }
+            if (c == 0)
+                return i;
+            from = i + 1;
+            step = step * 2 + 1; // jump in perfect binary search increments
+        }
+        return Arrays.binarySearch((Compare[]) in, from, to, find, comparator);
+    }
+
+    /**
+     * Perform {@link #exponentialSearch} on {@code in[from..to)}; if the value falls outside of the range of these
+     * elements, test against {@code ub} as though it occurred at position {@code to}
+     *
+     * @return same as {@link Arrays#binarySearch} if {@code find} occurs in the range {@code [in[from]..in[to])};
+     * otherwise the insertion position {@code -(1+to)} if {@code find} is less than {@code ub}, and {@code -(2+t)}
+     * if it is greater than or equal to.
+     *
+     * {@code ub} may be {@code null}, representing infinity.
+     */
+    static <Compare> int exponentialSearchWithUpperBound(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare ub, Compare find)
+    {
+        int step = 0;
+        while (true)
+        {
+            int i = from + step;
+            if (i >= to)
+            {
+                int c = compareWithMaybeInfinity(comparator, find, ub);
+                if (c >= 0)
+                    return -(2 + to);
+                break;
+            }
+            int c = comparator.compare(find, (Compare) in[i]);
+            if (c < 0)
+            {
+                to = i;
+                break;
+            }
+            if (c == 0)
+                return i;
+            from = i + 1;
+            step = step * 2 + 1; // jump in perfect binary search increments
+        }
+        return Arrays.binarySearch((Compare[]) in, from, to, find, comparator);
+    }
+    
+    /**
+     * Compute the size-in-bytes of full trees of cardinality {@code branchFactor^height - 1}
+     */
+    private static long[] sizeOnHeapOfPerfectTrees(int branchShift)
+    {
+        long[] result = new long[heightAtSize2n(Integer.MAX_VALUE, branchShift)];
+        int branchFactor = 1 << branchShift;
+        result[0] = branchFactor - 1;
+        for (int i = 1 ; i < result.length ; ++i)
+            result[i] = sizeOnHeapOfPerfectTree(i + 1, branchFactor);
+        return result;
+    }
+
+    /**
+     * Compute the size-in-bytes of a full tree of cardinality {@code branchFactor^height - 1}
+     * TODO: test
+     */
+    private static long sizeOnHeapOfPerfectTree(int height, int branchShift)
+    {
+        int branchFactor = 1 << branchShift;
+        long branchSize = ObjectSizes.sizeOfReferenceArray(branchFactor * 2);
+        int branchCount = height == 2 ? 1 : 2 + treeSize2n(height - 2, branchShift);
+        long leafSize = ObjectSizes.sizeOfReferenceArray((branchFactor - 1)|1);
+        int leafCount = 1 + treeSize2n(height - 1, branchShift);
+        return (branchSize * branchCount) + (leafSize * leafCount);
+    }
+
+    /**
+     * @return the actual height of {@code tree}
+     */
+    public static int height(Object[] tree)
+    {
+        if (isLeaf(tree))
+            return 1;
+
+        int height = 1;
+        while (!isLeaf(tree))
+        {
+            height++;
+            tree = (Object[]) tree[shallowSizeOfBranch(tree)];
+        }
+        return height;
+    }
+
+    /**
+     * @return the minimum representable size at {@code height}, for an internal (non-root) node.
+     */
+    private static int sparseSize(int height)
+    {
+        return treeSize2n(height, BRANCH_SHIFT - 1);
+    }
+
+    /**
+     * @return the maximum representable size at {@code height}.
+     */
+    private static int denseSize(int height)
+    {
+        return treeSize2n(height, BRANCH_SHIFT);
+    }
+
+    /**
+     * @return the maximum representable size at {@code height}.
+     */
+    private static int checkedDenseSize(int height)
+    {
+        assert height * BRANCH_SHIFT < 32;
+        return denseSize(height);
+    }
+
+    /**
+     * Computes the number of nodes in a full tree of height {@code height}
+     * and with {@code 2^branchShift} branch factor.
+     * i.e. computes {@code (2^branchShift)^height - 1}
+     */
+    private static int treeSize2n(int height, int branchShift)
+    {
+        return (1 << (branchShift * height)) - 1;
+    }
+
+    // TODO: test
+    private static int maxRootHeight(int size)
+    {
+        if (size <= BRANCH_FACTOR)
+            return 1;
+        return 1 + heightAtSize2n((size - 1) / 2, BRANCH_SHIFT - 1);
+    }
+    
+    private static int sizeOfBranch(Object[] branch)
+    {
+        int length = branch.length;
+        // length - 1 == getChildEnd == getPositionOfSizeMap
+        // (length / 2) - 1 == getChildCount - 1 == position of full tree size
+        // hard code this, as will be used often;
+        return ((int[]) branch[length - 1])[(length / 2) - 1];
+    }
+
+    /**
+     * Checks if the UpdateFunction is an instance of {@code UpdateFunction.Simple}.
+     */
+    private static boolean isSimple(UpdateFunction<?, ?> updateF)
+    {
+        return updateF instanceof UpdateFunction.Simple;
+    }
+
+    /**
+     * @return the size map for the branch node
+     */
+    static int[] sizeMap(Object[] branch)
+    {
+        return (int[]) branch[branch.length - 1];
+    }
+
+    public static long sizeOnHeapOf(Object[] tree)
+    {
+        long size = ObjectSizes.sizeOfArray(tree);
+        if (isLeaf(tree))
+            return size;
+        for (int i = childOffset(tree); i < childEndOffset(tree) ; i++)
+            size += sizeOnHeapOf((Object[]) tree[i]);
+        size += ObjectSizes.sizeOfArray(sizeMap(tree)); // may overcount, since we share size maps
+        return size;
+    }
+
+    // Arbitrary boundaries
+    private static Object POSITIVE_INFINITY = new Object();
+    private static Object NEGATIVE_INFINITY = new Object();
+
+    /** simple static wrapper to calls to cmp.compare() which checks if either a or b are Special (i.e. represent an infinity) */
+    private static <V> int compareWellFormed(Comparator<V> cmp, Object a, Object b)
+    {
+        if (a == b)
+            return 0;
+        if (a == NEGATIVE_INFINITY | b == POSITIVE_INFINITY)
+            return -1;
+        if (b == NEGATIVE_INFINITY | a == POSITIVE_INFINITY)
+            return 1;
+        return cmp.compare((V) a, (V) b);
+    }
+
+    public static boolean isWellFormed(Object[] btree, Comparator<?> cmp)
+    {
+        return isWellFormed(cmp, btree, true, NEGATIVE_INFINITY, POSITIVE_INFINITY);
+    }
+
+    private static boolean isWellFormed(Comparator<?> cmp, Object[] node, boolean isRoot, Object min, Object max)
+    {
+        if (isEmpty(node))
+            return true;
+
+        if (cmp != null && !isNodeWellFormed(cmp, node, min, max))
+            return false;
+
+        int keyCount = shallowSize(node);
+        if (keyCount < 1)
+            return false;
+        if (!isRoot && keyCount < BRANCH_FACTOR/2 - 1)
+            return false;
+        if (keyCount >= BRANCH_FACTOR)
+            return false;
+
+        if (isLeaf(node))
+            return true;
+
+        int[] sizeMap = sizeMap(node);
+        int type = 0;
+        int size = 0;
+        // compare each child node with the branch element at the head of this node it corresponds with
+        for (int i = childOffset(node); i < childEndOffset(node) ; i++)
+        {
+            Object[] child = (Object[]) node[i];
+            Object localmax = i < node.length - 2 ? node[i - childOffset(node)] : max;
+            if (!isWellFormed(cmp, child, false, min, localmax))
+                return false;
+
+            type |= isLeaf(child) ? 1 : 2;
+            min = localmax;
+            size += size(child);
+            if (sizeMap[i - childOffset(node)] != size)
+                return false;
+            size += 1;
+        }
+
+        if (type >= 3)
+            return false;
+        return true; // either all leaves or all branches but not a mix
+    }
+
+    private static boolean isNodeWellFormed(Comparator<?> cmp, Object[] node, Object min, Object max)
+    {
+        Object previous = min;
+        int end = shallowSize(node);
+        for (int i = 0; i < end; i++)
+        {
+            Object current = node[i];
+            if (compareWellFormed(cmp, previous, current) >= 0)
+                return false;
+
+            previous = current;
+        }
+        return compareWellFormed(cmp, previous, max) < 0;
+    }
+
+    /**
+     * Build a tree of unknown size, in order.
+     *
+     * Can be used with {@link #reverseInSitu} to build a tree in reverse.
+     */
+    public static <V> FastBuilder<V> fastBuilder()
+    {
+        TinyThreadLocalPool.TinyPool<FastBuilder<?>> pool = FastBuilder.POOL.get();
+        FastBuilder<V> builder = (FastBuilder<V>) pool.poll();
+        if (builder == null)
+            builder = new FastBuilder<>();
+        builder.pool = pool;
+        return builder;
+    }
+
+    /**
+     * Base class for AbstractFastBuilder.BranchBuilder, LeafBuilder and AbstractFastBuilder,
+     * containing shared behaviour and declaring some useful abstract methods.
+     */
+    private static abstract class LeafOrBranchBuilder
+    {
+        final int height;
+        final LeafOrBranchBuilder child;
+        BranchBuilder parent;
+
+        /**
+         * The current buffer contents (if any) of the leaf or branch - always sized to contain a complete
+         * node of the form being constructed.  Always non-null, except briefly during overflow.
+         */
+        Object[] buffer;
+        /**
+         * The number of keys in our buffer, whether or not we are building a leaf or branch; if we are building
+         * a branch, we will ordinarily have the same number of children as well, except temporarily when finishing
+         * the construction of the node.
+         */
+        int count;
+
+        /**
+         * either
+         *  1) an empty leftover buffer from a past usage, which can be used when we exhaust {@code buffer}; or
+         *  2) a full {@code buffer} that has been parked until we next overflow, so we can steal some back
+         *     if we finish before reaching MIN_KEYS in {@code buffer}
+         */
+        Object[] savedBuffer;
+        /**
+         * The key we overflowed on when populating savedBuffer.  If null, {@link #savedBuffer} is logically empty.
+         */
+        Object savedNextKey;
+
+        LeafOrBranchBuilder(LeafOrBranchBuilder child)
+        {
+            this.height = child == null ? 1 : 1 + child.height;
+            this.child = child;
+        }
+
+        /**
+         * Do we have enough keys in the builder to construct at least one balanced node?
+         * We could have enough to build two.
+         */
+        final boolean isSufficient()
+        {
+            return hasOverflow() || count >= MIN_KEYS;
+        }
+
+        /**
+         * Do we have an already constructed node saved, that we can propagate or redistribute?
+         * This implies we are building two nodes, since {@link #savedNextKey} would overflow {@link #savedBuffer}
+         */
+        final boolean hasOverflow()
+        {
+            return savedNextKey != null;
+        }
+
+        /**
+         * Do we have an already constructed node saved AND insufficient keys in our buffer, so
+         * that we need to share the contents of {@link #savedBuffer} with {@link #buffer} to construct
+         * our results?
+         */
+        final boolean mustRedistribute()
+        {
+            return hasOverflow() && count < MIN_KEYS;
+        }
+
+        /**
+         * Are we empty, i.e. we have no contents in either {@link #buffer} or {@link #savedBuffer}
+         */
+        final boolean isEmpty()
+        {
+            return count == 0 && savedNextKey == null;
+        }
+
+        /**
+         * Drain the contents of this builder and build up to two nodes, as necessary.
+         * If {@code unode != null} and we are building a single node that is identical to it, use {@code unode} instead.
+         * If {@code propagateTo != null} propagate any nodes we build to it.
+         * @return the last node we construct
+         */
+        abstract Object[] drainAndPropagate(Object[] unode, int usz, BranchBuilder propagateTo);
+
+        /**
+         * Drain the contents of this builder and build at most one node.
+         * Requires {@code !hasOverflow()}
+         * @return the node we construct
+         */
+        abstract Object[] drain();
+
+        /**
+         * Takes a node that would logically occur directly preceding the current buffer contents,
+         * and the key that would separate them in a parent node, and prepends their contents
+         * to the current buffer's contents.  This can be used to redistribute already-propagated
+         * contents to a parent in cases where this is convenient (i.e. when transforming)
+         *
+         * @param predecessor directly preceding node
+         * @param predecessorNextKey key that would have separated predecessor from buffer contents
+         */
+        abstract void prepend(Object[] predecessor, Object predecessorNextKey);
+
+        /**
+         * Indicates if this builder produces dense nodes, i.e. those that are populated with MAX_KEYS
+         * at every level.  Only the last two children of any branch may be non-dense, and in some cases only
+         * the last two nodes in any tier of the tree.
+         *
+         * This flag switches whether or not we maintain a buffer of sizes, or use the globally shared contents of
+         * DENSE_SIZE_MAPS.
+         */
+        abstract boolean producesOnlyDense();
+
+        /**
+         * Ensure there is a {@code branch.parent}, and return it
+         */
+        final BranchBuilder ensureParent()
+        {
+            if (parent == null)
+                parent = new BranchBuilder(this);
+            parent.touched = true;
+            return parent;
+        }
+
+        /**
+         * Mark a branch builder as utilised, so that we must clear it when resetting any {@link AbstractFastBuilder}
+         * @return {@code branch}
+         */
+        static BranchBuilder touch(BranchBuilder branch)
+        {
+            branch.touched = true;
+            return branch;
+        }
+
+        /**
+         * @return the parent builder iff we will overflow or the parent has already been touched
+         */
+        final BranchBuilder parentIfInUse()
+        {
+            // if we have enough data to propagate, ensure there is a parent and return it
+            if (hasOverflow())
+                return ensureParent();
+
+            // otherwise return it only if it's already in use
+            return parent == null || !parent.touched ? null : parent;
+        }
+
+        /**
+         * A utility method for comparing a range of two arrays
+         */
+        static boolean areIdentical(Object[] a, int aOffset, Object[] b, int bOffset, int count)
+        {
+            for (int i = 0 ; i < count ; ++i)
+            {
+                if (a[i + aOffset] != b[i + bOffset])
+                    return false;
+            }
+            return true;
+        }
+
+        /**
+         * A utility method for comparing a range of two arrays
+         */
+        static boolean areIdentical(int[] a, int aOffset, int[] b, int bOffset, int count)
+        {
+            for (int i = 0 ; i < count ; ++i)
+            {
+                if (a[i + aOffset] != b[i + bOffset])
+                    return false;
+            }
+            return true;
+        }
+    }
+
+    /**
+     * LeafBuilder for methods pertaining specifically to building a leaf in an {@link AbstractFastBuilder}.
+     * Note that {@link AbstractFastBuilder} extends this class directly, however it is convenient to maintain
+     * distinct classes in the hierarchy for clarity of behaviour and intent.
+     */
+    private static abstract class LeafBuilder extends LeafOrBranchBuilder
+    {
+        long allocated;
+
+        LeafBuilder()
+        {
+            super(null);
+            buffer = new Object[MAX_KEYS];
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer, overflowing if necessary
+         */
+        public void addKey(Object nextKey)
+        {
+            if (count == MAX_KEYS)
+                overflow(nextKey);
+            else
+                buffer[count++] = nextKey;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void addKeyNoOverflow(Object nextKey)
+        {
+            buffer[count++] = nextKey;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void maybeAddKeyNoOverflow(Object nextKey)
+        {
+            buffer[count] = nextKey;
+            count += nextKey != null ? 1 : 0;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void maybeAddKey(Object nextKey)
+        {
+            if (count == MAX_KEYS)
+            {
+                if (nextKey != null)
+                    overflow(nextKey);
+            }
+            else
+            {
+                buffer[count] = nextKey;
+                count += nextKey != null ? 1 : 0;
+            }
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}, overflowing as necessary.
+         */
+        void copy(Object[] source, int offset, int length)
+        {
+            if (count + length > MAX_KEYS)
+            {
+                int copy = MAX_KEYS - count;
+                System.arraycopy(source, offset, buffer, count, copy);
+                offset += copy;
+//              implicitly:  count = MAX_KEYS;
+                overflow(source[offset++]);
+                length -= 1 + copy;
+            }
+            System.arraycopy(source, offset, buffer, count, length);
+            count += length;
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}; the caller specifying overflow is unnecessary
+         */
+        void copyNoOverflow(Object[] source, int offset, int length)
+        {
+            System.arraycopy(source, offset, buffer, count, length);
+            count += length;
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}, overflowing as necessary.
+         * Applies {@code updateF} to the contents before insertion.
+         */
+        <Insert, Existing> void copy(Object[] source, int offset, int length, UpdateFunction<Insert, Existing> updateF)
+        {
+            if (isSimple(updateF))
+            {
+                copy(source, offset, length);
+                return;
+            }
+
+            if (count + length > MAX_KEYS)
+            {
+                int copy = MAX_KEYS - count;
+                for (int i = 0 ; i < copy ; ++i)
+                    buffer[count + i] = updateF.apply((Insert) source[offset + i]);
+                offset += copy;
+//              implicitly:  leaf().count = MAX_KEYS;
+                overflow(updateF.apply((Insert) source[offset++]));
+                length -= 1 + copy;
+            }
+            for (int i = 0 ; i < length ; ++i)
+                buffer[count + i] = updateF.apply((Insert) source[offset + i]);
+            count += length;
+        }
+
+        /**
+         * {@link #buffer} is full, and we need to make room either by populating {@link #savedBuffer},
+         * propagating its current contents, if any, to {@link #parent}
+         */
+        void overflow(Object nextKey)
+        {
+            if (hasOverflow())
+                propagateOverflow();
+
+            // precondition: count == MAX_KEYS and savedNextKey == null
+
+            Object[] newBuffer = savedBuffer;
+            if (newBuffer == null)
+                newBuffer = new Object[MAX_KEYS];
+
+            savedBuffer = buffer;
+            savedNextKey = nextKey;
+            buffer = newBuffer;
+            count = 0;
+        }
+
+        /**
+         * Redistribute the contents of {@link #savedBuffer} into {@link #buffer}, finalise {@link #savedBuffer} and flush upwards.
+         * Invoked when we are building from {@link #buffer}, have insufficient values but a complete leaf in {@link #savedBuffer}
+         * @return the size of the leaf we flushed to our parent from {@link #savedBuffer}
+         */
+        Object[] redistributeOverflowAndDrain()
+        {
+            Object[] newLeaf = redistributeAndDrain(savedBuffer, MAX_KEYS, savedNextKey);
+            savedNextKey = null;
+            return newLeaf;
+        }
+
+        /**
+         * Redistribute the contents of {@link #buffer} and an immediate predecessor into a new leaf,
+         * then construct a new predecessor with the remaining contents and propagate up to our parent
+         * Invoked when we are building from {@link #buffer}, have insufficient values but either a complete
+         * leaf in {@link #savedBuffer} or can exfiltrate one from our parent to redistribute.
+         *
+         * @return the second of the two new leaves
+         */
+        Object[] redistributeAndDrain(Object[] pred, int predSize, Object predNextKey)
+        {
+            // precondition: savedLeafCount == MAX_KEYS && leaf().count < MIN_KEYS
+            // ensure we have at least MIN_KEYS in leaf().buffer
+            // first shift leaf().buffer and steal some keys from leaf().savedBuffer and leaf().savedBufferNextKey
+            int steal = MIN_KEYS - count;
+            Object[] newLeaf = new Object[MIN_KEYS];
+            System.arraycopy(pred,  predSize - (steal - 1), newLeaf, 0,     steal - 1);
+                                                            newLeaf [steal - 1] = predNextKey;
+            System.arraycopy(buffer, 0,                     newLeaf, steal, count);
+
+            // then create a leaf out of the remainder of savedBuffer
+            int newPredecessorCount = predSize - steal;
+            Object[] newPredecessor = new Object[newPredecessorCount | 1];
+            System.arraycopy(pred, 0, newPredecessor, 0, newPredecessorCount);
+            if (allocated >= 0)
+                allocated += ObjectSizes.sizeOfReferenceArray(newPredecessorCount | 1);
+            ensureParent().addChildAndNextKey(newPredecessor, newPredecessorCount, pred[newPredecessorCount]);
+            return newLeaf;
+        }
+
+        /**
+         * Invoked to fill our {@link #buffer} to >= MIN_KEYS with data ocurring before {@link #buffer};
+         * possibly instead fills {@link #savedBuffer}
+         *
+         * @param pred directly preceding node
+         * @param predNextKey key that would have separated predecessor from buffer contents
+         */
+        void prepend(Object[] pred, Object predNextKey)
+        {
+            int predSize = sizeOfLeaf(pred);
+            int newKeys = 1 + predSize;
+            if (newKeys + count <= MAX_KEYS)
+            {
+                System.arraycopy(buffer, 0, buffer, newKeys, count);
+                System.arraycopy(pred,   0, buffer, 0,       predSize);
+                                            buffer [predSize] = predNextKey;
+                count += newKeys;
+            }
+            else
+            {
+                if (savedBuffer == null)
+                    savedBuffer = new Object[MAX_KEYS];
+                System.arraycopy(pred, 0, savedBuffer, 0, predSize);
+                if (predSize == MAX_KEYS)
+                {
+                    savedNextKey = predNextKey;
+                }
+                else
+                {
+                    int removeKeys = MAX_KEYS - predSize;
+                    count -= removeKeys;
+                                                         savedBuffer [predSize] = predNextKey;
+                    System.arraycopy(buffer, 0,          savedBuffer, predSize + 1, MAX_KEYS - newKeys);
+                                                         savedNextKey      = buffer[MAX_KEYS - newKeys];
+                    System.arraycopy(buffer, removeKeys, buffer,      0,            count);
+                }
+            }
+        }
+
+        /**
+         * Invoked when we want to add a key to the leaf buffer, but it is full
+         */
+        void propagateOverflow()
+        {
+            // propagate the leaf we have saved in altLeafBuffer

Review comment:
       Shouldn't this be `savedBuffer`?

##########
File path: src/java/org/apache/cassandra/utils/btree/BTree.java
##########
@@ -1448,4 +1920,2355 @@ private static boolean isStopSentinel(long v)
     {
         return accumulate(btree, accumulator, arg, null, null, initialValue);
     }
+
+    /**
+     * Calculate the minimum height needed for this size of tree
+     *
+     * @param size the tree size
+     * @return the minimum height needed for this size of tree
+     */
+    private static int minHeight(int size)
+    {
+        return heightAtSize2n(size, BRANCH_SHIFT);
+    }
+
+    private static int heightAtSize2n(int size, int branchShift)
+    {
+        //    branch factor                 =  1 <<  branchShift
+        // => full size at height            = (1 << (branchShift * height)) - 1
+        // => full size at height + 1        =  1 << (branchShift * height)
+        // => shift(full size at height + 1) = branchShift * height
+        // => shift(full size at height + 1) / branchShift = height
+        int indexOfHighestBit = 63 - Long.numberOfLeadingZeros(size + 1);
+        indexOfHighestBit += (1L << indexOfHighestBit) == (size + 1) ? 0 : 1;
+        return (branchShift - 1 + indexOfHighestBit) / branchShift;
+    }
+
+    private static int[][] buildBalancedSizeMaps(int branchShift)
+    {
+        int count = (32 / branchShift) - 1;
+        int childCount = 1 << branchShift;
+        int[][] sizeMaps = new int[count][childCount];
+        for (int height = 0 ; height < count ; ++height)
+        {
+            int childSize = treeSize2n(height + 1, branchShift);
+            int size = 0;
+            int[] sizeMap = sizeMaps[height];
+            for (int i = 0 ; i < childCount ; ++i)
+            {
+                sizeMap[i] = size += childSize;
+                size += 1;
+            }
+        }
+        return sizeMaps;
+    }
+
+    // simply utility to reverse the contents of array[from..to)
+    private static void reverse(Object[] array, int from, int to)
+    {
+        int mid = (from + to) / 2;
+        for (int i = from ; i < mid ; i++)
+        {
+            int j = to - (1 + i - from);
+            Object tmp = array[i];
+            array[i] = array[j];
+            array[j] = tmp;
+        }
+    }
+
+    // simply utility to reverse the contents of array[from..to)
+    private static void reverse(int[] array, int from, int to)
+    {
+        int mid = (from + to) / 2;
+        for (int i = from ; i < mid ; i++)
+        {
+            int j = to - (1 + i - from);
+            int tmp = array[i];
+            array[i] = array[j];
+            array[j] = tmp;
+        }
+    }
+
+    /**
+     * Mutate an array of child sizes into a cumulative sizeMap, returning the total size
+     */
+    private static int sizesToSizeMap(int[] sizeMap)
+    {
+        int total = sizeMap[0];
+        for (int i = 1 ; i < sizeMap.length ; ++i)
+            sizeMap[i] = total += 1 + sizeMap[i];
+        return total;
+    }
+
+    private static int sizesToSizeMap(int[] sizes, int count)
+    {
+        int total = sizes[0];
+        for (int i = 1 ; i < count ; ++i)
+            sizes[i] = total += 1 + sizes[i];
+        return total;
+    }
+
+    /**
+     * Mutate an array of child sizes into a cumulative sizeMap, returning the total size
+     */
+    private static void sizeMapToSizes(int[] sizeMap)
+    {
+        for (int i = sizeMap.length ; i > 1 ; --i)
+            sizeMap[i] -= 1 + sizeMap[i - 1];
+    }
+
+    /**
+     * A simple utility method to handle a null upper bound that we treat as infinity
+     */
+    private static <Compare> int compareWithMaybeInfinity(Comparator<? super Compare> comparator, Compare key, Compare ub)
+    {
+        if (ub == null)
+            return -1;
+        return comparator.compare(key, ub);
+    }
+
+    /**
+     * Perform {@link #exponentialSearch} on {@code in[from..to)}, treating a {@code find} of {@code null} as infinity.
+     */
+    static <Compare> int exponentialSearchForMaybeInfinity(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare find)
+    {
+        if (find == null)
+            return -(1 + to);
+        return exponentialSearch(comparator, in, from, to, find);
+    }
+
+    /**
+     * Equivalent to {@link Arrays#binarySearch}, only more efficient algorithmically for linear merges.
+     * Binary search has worst case complexity {@code O(n.lg n)} for a linear merge, whereas exponential search
+     * has a worst case of {@code O(n)}. However compared to a simple linear merge, the best case for exponential
+     * search is {@code O(lg(n))} instead of {@code O(n)}.
+     */
+    private static <Compare> int exponentialSearch(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare find)
+    {
+        int step = 0;
+        while (from + step < to)
+        {
+            int i = from + step;
+            int c = comparator.compare(find, (Compare) in[i]);
+            if (c < 0)
+            {
+                to = i;
+                break;
+            }
+            if (c == 0)
+                return i;
+            from = i + 1;
+            step = step * 2 + 1; // jump in perfect binary search increments
+        }
+        return Arrays.binarySearch((Compare[]) in, from, to, find, comparator);
+    }
+
+    /**
+     * Perform {@link #exponentialSearch} on {@code in[from..to)}; if the value falls outside of the range of these
+     * elements, test against {@code ub} as though it occurred at position {@code to}
+     *
+     * @return same as {@link Arrays#binarySearch} if {@code find} occurs in the range {@code [in[from]..in[to])};
+     * otherwise the insertion position {@code -(1+to)} if {@code find} is less than {@code ub}, and {@code -(2+t)}
+     * if it is greater than or equal to.
+     *
+     * {@code ub} may be {@code null}, representing infinity.
+     */
+    static <Compare> int exponentialSearchWithUpperBound(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare ub, Compare find)
+    {
+        int step = 0;
+        while (true)
+        {
+            int i = from + step;
+            if (i >= to)
+            {
+                int c = compareWithMaybeInfinity(comparator, find, ub);
+                if (c >= 0)
+                    return -(2 + to);
+                break;
+            }
+            int c = comparator.compare(find, (Compare) in[i]);
+            if (c < 0)
+            {
+                to = i;
+                break;
+            }
+            if (c == 0)
+                return i;
+            from = i + 1;
+            step = step * 2 + 1; // jump in perfect binary search increments
+        }
+        return Arrays.binarySearch((Compare[]) in, from, to, find, comparator);
+    }
+    
+    /**
+     * Compute the size-in-bytes of full trees of cardinality {@code branchFactor^height - 1}
+     */
+    private static long[] sizeOnHeapOfPerfectTrees(int branchShift)
+    {
+        long[] result = new long[heightAtSize2n(Integer.MAX_VALUE, branchShift)];
+        int branchFactor = 1 << branchShift;
+        result[0] = branchFactor - 1;
+        for (int i = 1 ; i < result.length ; ++i)
+            result[i] = sizeOnHeapOfPerfectTree(i + 1, branchFactor);
+        return result;
+    }
+
+    /**
+     * Compute the size-in-bytes of a full tree of cardinality {@code branchFactor^height - 1}
+     * TODO: test
+     */
+    private static long sizeOnHeapOfPerfectTree(int height, int branchShift)
+    {
+        int branchFactor = 1 << branchShift;
+        long branchSize = ObjectSizes.sizeOfReferenceArray(branchFactor * 2);
+        int branchCount = height == 2 ? 1 : 2 + treeSize2n(height - 2, branchShift);
+        long leafSize = ObjectSizes.sizeOfReferenceArray((branchFactor - 1)|1);
+        int leafCount = 1 + treeSize2n(height - 1, branchShift);
+        return (branchSize * branchCount) + (leafSize * leafCount);
+    }
+
+    /**
+     * @return the actual height of {@code tree}
+     */
+    public static int height(Object[] tree)
+    {
+        if (isLeaf(tree))
+            return 1;
+
+        int height = 1;
+        while (!isLeaf(tree))
+        {
+            height++;
+            tree = (Object[]) tree[shallowSizeOfBranch(tree)];
+        }
+        return height;
+    }
+
+    /**
+     * @return the minimum representable size at {@code height}, for an internal (non-root) node.
+     */
+    private static int sparseSize(int height)
+    {
+        return treeSize2n(height, BRANCH_SHIFT - 1);
+    }
+
+    /**
+     * @return the maximum representable size at {@code height}.
+     */
+    private static int denseSize(int height)
+    {
+        return treeSize2n(height, BRANCH_SHIFT);
+    }
+
+    /**
+     * @return the maximum representable size at {@code height}.
+     */
+    private static int checkedDenseSize(int height)
+    {
+        assert height * BRANCH_SHIFT < 32;
+        return denseSize(height);
+    }
+
+    /**
+     * Computes the number of nodes in a full tree of height {@code height}
+     * and with {@code 2^branchShift} branch factor.
+     * i.e. computes {@code (2^branchShift)^height - 1}
+     */
+    private static int treeSize2n(int height, int branchShift)
+    {
+        return (1 << (branchShift * height)) - 1;
+    }
+
+    // TODO: test
+    private static int maxRootHeight(int size)
+    {
+        if (size <= BRANCH_FACTOR)
+            return 1;
+        return 1 + heightAtSize2n((size - 1) / 2, BRANCH_SHIFT - 1);
+    }
+    
+    private static int sizeOfBranch(Object[] branch)
+    {
+        int length = branch.length;
+        // length - 1 == getChildEnd == getPositionOfSizeMap
+        // (length / 2) - 1 == getChildCount - 1 == position of full tree size
+        // hard code this, as will be used often;
+        return ((int[]) branch[length - 1])[(length / 2) - 1];
+    }
+
+    /**
+     * Checks if the UpdateFunction is an instance of {@code UpdateFunction.Simple}.
+     */
+    private static boolean isSimple(UpdateFunction<?, ?> updateF)
+    {
+        return updateF instanceof UpdateFunction.Simple;
+    }
+
+    /**
+     * @return the size map for the branch node
+     */
+    static int[] sizeMap(Object[] branch)
+    {
+        return (int[]) branch[branch.length - 1];
+    }
+
+    public static long sizeOnHeapOf(Object[] tree)
+    {
+        long size = ObjectSizes.sizeOfArray(tree);
+        if (isLeaf(tree))
+            return size;
+        for (int i = childOffset(tree); i < childEndOffset(tree) ; i++)
+            size += sizeOnHeapOf((Object[]) tree[i]);
+        size += ObjectSizes.sizeOfArray(sizeMap(tree)); // may overcount, since we share size maps
+        return size;
+    }
+
+    // Arbitrary boundaries
+    private static Object POSITIVE_INFINITY = new Object();
+    private static Object NEGATIVE_INFINITY = new Object();
+
+    /** simple static wrapper to calls to cmp.compare() which checks if either a or b are Special (i.e. represent an infinity) */
+    private static <V> int compareWellFormed(Comparator<V> cmp, Object a, Object b)
+    {
+        if (a == b)
+            return 0;
+        if (a == NEGATIVE_INFINITY | b == POSITIVE_INFINITY)
+            return -1;
+        if (b == NEGATIVE_INFINITY | a == POSITIVE_INFINITY)
+            return 1;
+        return cmp.compare((V) a, (V) b);
+    }
+
+    public static boolean isWellFormed(Object[] btree, Comparator<?> cmp)
+    {
+        return isWellFormed(cmp, btree, true, NEGATIVE_INFINITY, POSITIVE_INFINITY);
+    }
+
+    private static boolean isWellFormed(Comparator<?> cmp, Object[] node, boolean isRoot, Object min, Object max)
+    {
+        if (isEmpty(node))
+            return true;
+
+        if (cmp != null && !isNodeWellFormed(cmp, node, min, max))
+            return false;
+
+        int keyCount = shallowSize(node);
+        if (keyCount < 1)
+            return false;
+        if (!isRoot && keyCount < BRANCH_FACTOR/2 - 1)
+            return false;
+        if (keyCount >= BRANCH_FACTOR)
+            return false;
+
+        if (isLeaf(node))
+            return true;
+
+        int[] sizeMap = sizeMap(node);
+        int type = 0;
+        int size = 0;
+        // compare each child node with the branch element at the head of this node it corresponds with
+        for (int i = childOffset(node); i < childEndOffset(node) ; i++)
+        {
+            Object[] child = (Object[]) node[i];
+            Object localmax = i < node.length - 2 ? node[i - childOffset(node)] : max;
+            if (!isWellFormed(cmp, child, false, min, localmax))
+                return false;
+
+            type |= isLeaf(child) ? 1 : 2;
+            min = localmax;
+            size += size(child);
+            if (sizeMap[i - childOffset(node)] != size)
+                return false;
+            size += 1;
+        }
+
+        if (type >= 3)
+            return false;
+        return true; // either all leaves or all branches but not a mix
+    }
+
+    private static boolean isNodeWellFormed(Comparator<?> cmp, Object[] node, Object min, Object max)
+    {
+        Object previous = min;
+        int end = shallowSize(node);
+        for (int i = 0; i < end; i++)
+        {
+            Object current = node[i];
+            if (compareWellFormed(cmp, previous, current) >= 0)
+                return false;
+
+            previous = current;
+        }
+        return compareWellFormed(cmp, previous, max) < 0;
+    }
+
+    /**
+     * Build a tree of unknown size, in order.
+     *
+     * Can be used with {@link #reverseInSitu} to build a tree in reverse.
+     */
+    public static <V> FastBuilder<V> fastBuilder()
+    {
+        TinyThreadLocalPool.TinyPool<FastBuilder<?>> pool = FastBuilder.POOL.get();
+        FastBuilder<V> builder = (FastBuilder<V>) pool.poll();
+        if (builder == null)
+            builder = new FastBuilder<>();
+        builder.pool = pool;
+        return builder;
+    }
+
+    /**
+     * Base class for AbstractFastBuilder.BranchBuilder, LeafBuilder and AbstractFastBuilder,
+     * containing shared behaviour and declaring some useful abstract methods.
+     */
+    private static abstract class LeafOrBranchBuilder
+    {
+        final int height;
+        final LeafOrBranchBuilder child;
+        BranchBuilder parent;
+
+        /**
+         * The current buffer contents (if any) of the leaf or branch - always sized to contain a complete
+         * node of the form being constructed.  Always non-null, except briefly during overflow.
+         */
+        Object[] buffer;
+        /**
+         * The number of keys in our buffer, whether or not we are building a leaf or branch; if we are building
+         * a branch, we will ordinarily have the same number of children as well, except temporarily when finishing
+         * the construction of the node.
+         */
+        int count;
+
+        /**
+         * either
+         *  1) an empty leftover buffer from a past usage, which can be used when we exhaust {@code buffer}; or
+         *  2) a full {@code buffer} that has been parked until we next overflow, so we can steal some back
+         *     if we finish before reaching MIN_KEYS in {@code buffer}
+         */
+        Object[] savedBuffer;
+        /**
+         * The key we overflowed on when populating savedBuffer.  If null, {@link #savedBuffer} is logically empty.
+         */
+        Object savedNextKey;
+
+        LeafOrBranchBuilder(LeafOrBranchBuilder child)
+        {
+            this.height = child == null ? 1 : 1 + child.height;
+            this.child = child;
+        }
+
+        /**
+         * Do we have enough keys in the builder to construct at least one balanced node?
+         * We could have enough to build two.
+         */
+        final boolean isSufficient()
+        {
+            return hasOverflow() || count >= MIN_KEYS;
+        }
+
+        /**
+         * Do we have an already constructed node saved, that we can propagate or redistribute?
+         * This implies we are building two nodes, since {@link #savedNextKey} would overflow {@link #savedBuffer}
+         */
+        final boolean hasOverflow()
+        {
+            return savedNextKey != null;
+        }
+
+        /**
+         * Do we have an already constructed node saved AND insufficient keys in our buffer, so
+         * that we need to share the contents of {@link #savedBuffer} with {@link #buffer} to construct
+         * our results?
+         */
+        final boolean mustRedistribute()
+        {
+            return hasOverflow() && count < MIN_KEYS;
+        }
+
+        /**
+         * Are we empty, i.e. we have no contents in either {@link #buffer} or {@link #savedBuffer}
+         */
+        final boolean isEmpty()
+        {
+            return count == 0 && savedNextKey == null;
+        }
+
+        /**
+         * Drain the contents of this builder and build up to two nodes, as necessary.
+         * If {@code unode != null} and we are building a single node that is identical to it, use {@code unode} instead.
+         * If {@code propagateTo != null} propagate any nodes we build to it.
+         * @return the last node we construct
+         */
+        abstract Object[] drainAndPropagate(Object[] unode, int usz, BranchBuilder propagateTo);
+
+        /**
+         * Drain the contents of this builder and build at most one node.
+         * Requires {@code !hasOverflow()}
+         * @return the node we construct
+         */
+        abstract Object[] drain();
+
+        /**
+         * Takes a node that would logically occur directly preceding the current buffer contents,
+         * and the key that would separate them in a parent node, and prepends their contents
+         * to the current buffer's contents.  This can be used to redistribute already-propagated
+         * contents to a parent in cases where this is convenient (i.e. when transforming)
+         *
+         * @param predecessor directly preceding node
+         * @param predecessorNextKey key that would have separated predecessor from buffer contents
+         */
+        abstract void prepend(Object[] predecessor, Object predecessorNextKey);
+
+        /**
+         * Indicates if this builder produces dense nodes, i.e. those that are populated with MAX_KEYS
+         * at every level.  Only the last two children of any branch may be non-dense, and in some cases only
+         * the last two nodes in any tier of the tree.
+         *
+         * This flag switches whether or not we maintain a buffer of sizes, or use the globally shared contents of
+         * DENSE_SIZE_MAPS.
+         */
+        abstract boolean producesOnlyDense();
+
+        /**
+         * Ensure there is a {@code branch.parent}, and return it
+         */
+        final BranchBuilder ensureParent()
+        {
+            if (parent == null)
+                parent = new BranchBuilder(this);
+            parent.touched = true;
+            return parent;
+        }
+
+        /**
+         * Mark a branch builder as utilised, so that we must clear it when resetting any {@link AbstractFastBuilder}
+         * @return {@code branch}
+         */
+        static BranchBuilder touch(BranchBuilder branch)
+        {
+            branch.touched = true;
+            return branch;
+        }
+
+        /**
+         * @return the parent builder iff we will overflow or the parent has already been touched
+         */
+        final BranchBuilder parentIfInUse()
+        {
+            // if we have enough data to propagate, ensure there is a parent and return it
+            if (hasOverflow())
+                return ensureParent();
+
+            // otherwise return it only if it's already in use
+            return parent == null || !parent.touched ? null : parent;
+        }
+
+        /**
+         * A utility method for comparing a range of two arrays
+         */
+        static boolean areIdentical(Object[] a, int aOffset, Object[] b, int bOffset, int count)
+        {
+            for (int i = 0 ; i < count ; ++i)
+            {
+                if (a[i + aOffset] != b[i + bOffset])
+                    return false;
+            }
+            return true;
+        }
+
+        /**
+         * A utility method for comparing a range of two arrays
+         */
+        static boolean areIdentical(int[] a, int aOffset, int[] b, int bOffset, int count)
+        {
+            for (int i = 0 ; i < count ; ++i)
+            {
+                if (a[i + aOffset] != b[i + bOffset])
+                    return false;
+            }
+            return true;
+        }
+    }
+
+    /**
+     * LeafBuilder for methods pertaining specifically to building a leaf in an {@link AbstractFastBuilder}.
+     * Note that {@link AbstractFastBuilder} extends this class directly, however it is convenient to maintain
+     * distinct classes in the hierarchy for clarity of behaviour and intent.
+     */
+    private static abstract class LeafBuilder extends LeafOrBranchBuilder
+    {
+        long allocated;
+
+        LeafBuilder()
+        {
+            super(null);
+            buffer = new Object[MAX_KEYS];
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer, overflowing if necessary
+         */
+        public void addKey(Object nextKey)
+        {
+            if (count == MAX_KEYS)
+                overflow(nextKey);
+            else
+                buffer[count++] = nextKey;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void addKeyNoOverflow(Object nextKey)
+        {
+            buffer[count++] = nextKey;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void maybeAddKeyNoOverflow(Object nextKey)
+        {
+            buffer[count] = nextKey;
+            count += nextKey != null ? 1 : 0;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void maybeAddKey(Object nextKey)
+        {
+            if (count == MAX_KEYS)
+            {
+                if (nextKey != null)
+                    overflow(nextKey);
+            }
+            else
+            {
+                buffer[count] = nextKey;
+                count += nextKey != null ? 1 : 0;
+            }
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}, overflowing as necessary.
+         */
+        void copy(Object[] source, int offset, int length)
+        {
+            if (count + length > MAX_KEYS)
+            {
+                int copy = MAX_KEYS - count;
+                System.arraycopy(source, offset, buffer, count, copy);
+                offset += copy;
+//              implicitly:  count = MAX_KEYS;
+                overflow(source[offset++]);
+                length -= 1 + copy;
+            }
+            System.arraycopy(source, offset, buffer, count, length);
+            count += length;
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}; the caller specifying overflow is unnecessary
+         */
+        void copyNoOverflow(Object[] source, int offset, int length)
+        {
+            System.arraycopy(source, offset, buffer, count, length);
+            count += length;
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}, overflowing as necessary.
+         * Applies {@code updateF} to the contents before insertion.
+         */
+        <Insert, Existing> void copy(Object[] source, int offset, int length, UpdateFunction<Insert, Existing> updateF)
+        {
+            if (isSimple(updateF))
+            {
+                copy(source, offset, length);
+                return;
+            }
+
+            if (count + length > MAX_KEYS)
+            {
+                int copy = MAX_KEYS - count;
+                for (int i = 0 ; i < copy ; ++i)
+                    buffer[count + i] = updateF.apply((Insert) source[offset + i]);
+                offset += copy;
+//              implicitly:  leaf().count = MAX_KEYS;
+                overflow(updateF.apply((Insert) source[offset++]));
+                length -= 1 + copy;
+            }
+            for (int i = 0 ; i < length ; ++i)
+                buffer[count + i] = updateF.apply((Insert) source[offset + i]);
+            count += length;
+        }
+
+        /**
+         * {@link #buffer} is full, and we need to make room either by populating {@link #savedBuffer},
+         * propagating its current contents, if any, to {@link #parent}
+         */
+        void overflow(Object nextKey)
+        {
+            if (hasOverflow())
+                propagateOverflow();
+
+            // precondition: count == MAX_KEYS and savedNextKey == null
+
+            Object[] newBuffer = savedBuffer;
+            if (newBuffer == null)
+                newBuffer = new Object[MAX_KEYS];
+
+            savedBuffer = buffer;
+            savedNextKey = nextKey;
+            buffer = newBuffer;
+            count = 0;
+        }
+
+        /**
+         * Redistribute the contents of {@link #savedBuffer} into {@link #buffer}, finalise {@link #savedBuffer} and flush upwards.
+         * Invoked when we are building from {@link #buffer}, have insufficient values but a complete leaf in {@link #savedBuffer}
+         * @return the size of the leaf we flushed to our parent from {@link #savedBuffer}
+         */
+        Object[] redistributeOverflowAndDrain()
+        {
+            Object[] newLeaf = redistributeAndDrain(savedBuffer, MAX_KEYS, savedNextKey);
+            savedNextKey = null;
+            return newLeaf;
+        }
+
+        /**
+         * Redistribute the contents of {@link #buffer} and an immediate predecessor into a new leaf,
+         * then construct a new predecessor with the remaining contents and propagate up to our parent
+         * Invoked when we are building from {@link #buffer}, have insufficient values but either a complete
+         * leaf in {@link #savedBuffer} or can exfiltrate one from our parent to redistribute.
+         *
+         * @return the second of the two new leaves
+         */
+        Object[] redistributeAndDrain(Object[] pred, int predSize, Object predNextKey)
+        {
+            // precondition: savedLeafCount == MAX_KEYS && leaf().count < MIN_KEYS
+            // ensure we have at least MIN_KEYS in leaf().buffer
+            // first shift leaf().buffer and steal some keys from leaf().savedBuffer and leaf().savedBufferNextKey
+            int steal = MIN_KEYS - count;
+            Object[] newLeaf = new Object[MIN_KEYS];
+            System.arraycopy(pred,  predSize - (steal - 1), newLeaf, 0,     steal - 1);
+                                                            newLeaf [steal - 1] = predNextKey;
+            System.arraycopy(buffer, 0,                     newLeaf, steal, count);
+
+            // then create a leaf out of the remainder of savedBuffer
+            int newPredecessorCount = predSize - steal;
+            Object[] newPredecessor = new Object[newPredecessorCount | 1];
+            System.arraycopy(pred, 0, newPredecessor, 0, newPredecessorCount);
+            if (allocated >= 0)
+                allocated += ObjectSizes.sizeOfReferenceArray(newPredecessorCount | 1);
+            ensureParent().addChildAndNextKey(newPredecessor, newPredecessorCount, pred[newPredecessorCount]);
+            return newLeaf;
+        }
+
+        /**
+         * Invoked to fill our {@link #buffer} to >= MIN_KEYS with data ocurring before {@link #buffer};
+         * possibly instead fills {@link #savedBuffer}
+         *
+         * @param pred directly preceding node
+         * @param predNextKey key that would have separated predecessor from buffer contents
+         */
+        void prepend(Object[] pred, Object predNextKey)
+        {
+            int predSize = sizeOfLeaf(pred);
+            int newKeys = 1 + predSize;
+            if (newKeys + count <= MAX_KEYS)
+            {
+                System.arraycopy(buffer, 0, buffer, newKeys, count);
+                System.arraycopy(pred,   0, buffer, 0,       predSize);
+                                            buffer [predSize] = predNextKey;
+                count += newKeys;
+            }
+            else
+            {
+                if (savedBuffer == null)
+                    savedBuffer = new Object[MAX_KEYS];
+                System.arraycopy(pred, 0, savedBuffer, 0, predSize);
+                if (predSize == MAX_KEYS)
+                {
+                    savedNextKey = predNextKey;
+                }
+                else
+                {
+                    int removeKeys = MAX_KEYS - predSize;
+                    count -= removeKeys;
+                                                         savedBuffer [predSize] = predNextKey;

Review comment:
       Could we get rid of this alignment throughout? It's inconsistent, out of spec and not helpful.

##########
File path: src/java/org/apache/cassandra/utils/btree/BTree.java
##########
@@ -1448,4 +1920,2355 @@ private static boolean isStopSentinel(long v)
     {
         return accumulate(btree, accumulator, arg, null, null, initialValue);
     }
+
+    /**
+     * Calculate the minimum height needed for this size of tree
+     *
+     * @param size the tree size
+     * @return the minimum height needed for this size of tree
+     */
+    private static int minHeight(int size)
+    {
+        return heightAtSize2n(size, BRANCH_SHIFT);
+    }
+
+    private static int heightAtSize2n(int size, int branchShift)
+    {
+        //    branch factor                 =  1 <<  branchShift
+        // => full size at height            = (1 << (branchShift * height)) - 1
+        // => full size at height + 1        =  1 << (branchShift * height)
+        // => shift(full size at height + 1) = branchShift * height
+        // => shift(full size at height + 1) / branchShift = height
+        int indexOfHighestBit = 63 - Long.numberOfLeadingZeros(size + 1);
+        indexOfHighestBit += (1L << indexOfHighestBit) == (size + 1) ? 0 : 1;
+        return (branchShift - 1 + indexOfHighestBit) / branchShift;
+    }
+
+    private static int[][] buildBalancedSizeMaps(int branchShift)
+    {
+        int count = (32 / branchShift) - 1;
+        int childCount = 1 << branchShift;
+        int[][] sizeMaps = new int[count][childCount];
+        for (int height = 0 ; height < count ; ++height)
+        {
+            int childSize = treeSize2n(height + 1, branchShift);
+            int size = 0;
+            int[] sizeMap = sizeMaps[height];
+            for (int i = 0 ; i < childCount ; ++i)
+            {
+                sizeMap[i] = size += childSize;
+                size += 1;
+            }
+        }
+        return sizeMaps;
+    }
+
+    // simply utility to reverse the contents of array[from..to)
+    private static void reverse(Object[] array, int from, int to)
+    {
+        int mid = (from + to) / 2;
+        for (int i = from ; i < mid ; i++)
+        {
+            int j = to - (1 + i - from);
+            Object tmp = array[i];
+            array[i] = array[j];
+            array[j] = tmp;
+        }
+    }
+
+    // simply utility to reverse the contents of array[from..to)
+    private static void reverse(int[] array, int from, int to)
+    {
+        int mid = (from + to) / 2;
+        for (int i = from ; i < mid ; i++)
+        {
+            int j = to - (1 + i - from);
+            int tmp = array[i];
+            array[i] = array[j];
+            array[j] = tmp;
+        }
+    }
+
+    /**
+     * Mutate an array of child sizes into a cumulative sizeMap, returning the total size
+     */
+    private static int sizesToSizeMap(int[] sizeMap)
+    {
+        int total = sizeMap[0];
+        for (int i = 1 ; i < sizeMap.length ; ++i)
+            sizeMap[i] = total += 1 + sizeMap[i];
+        return total;
+    }
+
+    private static int sizesToSizeMap(int[] sizes, int count)
+    {
+        int total = sizes[0];
+        for (int i = 1 ; i < count ; ++i)
+            sizes[i] = total += 1 + sizes[i];
+        return total;
+    }
+
+    /**
+     * Mutate an array of child sizes into a cumulative sizeMap, returning the total size
+     */
+    private static void sizeMapToSizes(int[] sizeMap)
+    {
+        for (int i = sizeMap.length ; i > 1 ; --i)
+            sizeMap[i] -= 1 + sizeMap[i - 1];
+    }
+
+    /**
+     * A simple utility method to handle a null upper bound that we treat as infinity
+     */
+    private static <Compare> int compareWithMaybeInfinity(Comparator<? super Compare> comparator, Compare key, Compare ub)
+    {
+        if (ub == null)
+            return -1;
+        return comparator.compare(key, ub);
+    }
+
+    /**
+     * Perform {@link #exponentialSearch} on {@code in[from..to)}, treating a {@code find} of {@code null} as infinity.
+     */
+    static <Compare> int exponentialSearchForMaybeInfinity(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare find)
+    {
+        if (find == null)
+            return -(1 + to);
+        return exponentialSearch(comparator, in, from, to, find);
+    }
+
+    /**
+     * Equivalent to {@link Arrays#binarySearch}, only more efficient algorithmically for linear merges.
+     * Binary search has worst case complexity {@code O(n.lg n)} for a linear merge, whereas exponential search
+     * has a worst case of {@code O(n)}. However compared to a simple linear merge, the best case for exponential
+     * search is {@code O(lg(n))} instead of {@code O(n)}.
+     */
+    private static <Compare> int exponentialSearch(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare find)
+    {
+        int step = 0;
+        while (from + step < to)
+        {
+            int i = from + step;
+            int c = comparator.compare(find, (Compare) in[i]);
+            if (c < 0)
+            {
+                to = i;
+                break;
+            }
+            if (c == 0)
+                return i;
+            from = i + 1;
+            step = step * 2 + 1; // jump in perfect binary search increments
+        }
+        return Arrays.binarySearch((Compare[]) in, from, to, find, comparator);
+    }
+
+    /**
+     * Perform {@link #exponentialSearch} on {@code in[from..to)}; if the value falls outside of the range of these
+     * elements, test against {@code ub} as though it occurred at position {@code to}
+     *
+     * @return same as {@link Arrays#binarySearch} if {@code find} occurs in the range {@code [in[from]..in[to])};
+     * otherwise the insertion position {@code -(1+to)} if {@code find} is less than {@code ub}, and {@code -(2+t)}
+     * if it is greater than or equal to.
+     *
+     * {@code ub} may be {@code null}, representing infinity.
+     */
+    static <Compare> int exponentialSearchWithUpperBound(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare ub, Compare find)
+    {
+        int step = 0;
+        while (true)
+        {
+            int i = from + step;
+            if (i >= to)
+            {
+                int c = compareWithMaybeInfinity(comparator, find, ub);
+                if (c >= 0)
+                    return -(2 + to);
+                break;
+            }
+            int c = comparator.compare(find, (Compare) in[i]);
+            if (c < 0)
+            {
+                to = i;
+                break;
+            }
+            if (c == 0)
+                return i;
+            from = i + 1;
+            step = step * 2 + 1; // jump in perfect binary search increments
+        }
+        return Arrays.binarySearch((Compare[]) in, from, to, find, comparator);
+    }
+    
+    /**
+     * Compute the size-in-bytes of full trees of cardinality {@code branchFactor^height - 1}
+     */
+    private static long[] sizeOnHeapOfPerfectTrees(int branchShift)
+    {
+        long[] result = new long[heightAtSize2n(Integer.MAX_VALUE, branchShift)];
+        int branchFactor = 1 << branchShift;
+        result[0] = branchFactor - 1;
+        for (int i = 1 ; i < result.length ; ++i)
+            result[i] = sizeOnHeapOfPerfectTree(i + 1, branchFactor);
+        return result;
+    }
+
+    /**
+     * Compute the size-in-bytes of a full tree of cardinality {@code branchFactor^height - 1}
+     * TODO: test
+     */
+    private static long sizeOnHeapOfPerfectTree(int height, int branchShift)
+    {
+        int branchFactor = 1 << branchShift;
+        long branchSize = ObjectSizes.sizeOfReferenceArray(branchFactor * 2);
+        int branchCount = height == 2 ? 1 : 2 + treeSize2n(height - 2, branchShift);
+        long leafSize = ObjectSizes.sizeOfReferenceArray((branchFactor - 1)|1);
+        int leafCount = 1 + treeSize2n(height - 1, branchShift);
+        return (branchSize * branchCount) + (leafSize * leafCount);
+    }
+
+    /**
+     * @return the actual height of {@code tree}
+     */
+    public static int height(Object[] tree)
+    {
+        if (isLeaf(tree))
+            return 1;
+
+        int height = 1;
+        while (!isLeaf(tree))
+        {
+            height++;
+            tree = (Object[]) tree[shallowSizeOfBranch(tree)];
+        }
+        return height;
+    }
+
+    /**
+     * @return the minimum representable size at {@code height}, for an internal (non-root) node.
+     */
+    private static int sparseSize(int height)
+    {
+        return treeSize2n(height, BRANCH_SHIFT - 1);
+    }
+
+    /**
+     * @return the maximum representable size at {@code height}.
+     */
+    private static int denseSize(int height)
+    {
+        return treeSize2n(height, BRANCH_SHIFT);
+    }
+
+    /**
+     * @return the maximum representable size at {@code height}.
+     */
+    private static int checkedDenseSize(int height)
+    {
+        assert height * BRANCH_SHIFT < 32;
+        return denseSize(height);
+    }
+
+    /**
+     * Computes the number of nodes in a full tree of height {@code height}
+     * and with {@code 2^branchShift} branch factor.
+     * i.e. computes {@code (2^branchShift)^height - 1}
+     */
+    private static int treeSize2n(int height, int branchShift)
+    {
+        return (1 << (branchShift * height)) - 1;
+    }
+
+    // TODO: test
+    private static int maxRootHeight(int size)
+    {
+        if (size <= BRANCH_FACTOR)
+            return 1;
+        return 1 + heightAtSize2n((size - 1) / 2, BRANCH_SHIFT - 1);
+    }
+    
+    private static int sizeOfBranch(Object[] branch)
+    {
+        int length = branch.length;
+        // length - 1 == getChildEnd == getPositionOfSizeMap
+        // (length / 2) - 1 == getChildCount - 1 == position of full tree size
+        // hard code this, as will be used often;
+        return ((int[]) branch[length - 1])[(length / 2) - 1];
+    }
+
+    /**
+     * Checks if the UpdateFunction is an instance of {@code UpdateFunction.Simple}.
+     */
+    private static boolean isSimple(UpdateFunction<?, ?> updateF)
+    {
+        return updateF instanceof UpdateFunction.Simple;
+    }
+
+    /**
+     * @return the size map for the branch node
+     */
+    static int[] sizeMap(Object[] branch)
+    {
+        return (int[]) branch[branch.length - 1];
+    }
+
+    public static long sizeOnHeapOf(Object[] tree)
+    {
+        long size = ObjectSizes.sizeOfArray(tree);
+        if (isLeaf(tree))
+            return size;
+        for (int i = childOffset(tree); i < childEndOffset(tree) ; i++)
+            size += sizeOnHeapOf((Object[]) tree[i]);
+        size += ObjectSizes.sizeOfArray(sizeMap(tree)); // may overcount, since we share size maps
+        return size;
+    }
+
+    // Arbitrary boundaries
+    private static Object POSITIVE_INFINITY = new Object();
+    private static Object NEGATIVE_INFINITY = new Object();
+
+    /** simple static wrapper to calls to cmp.compare() which checks if either a or b are Special (i.e. represent an infinity) */
+    private static <V> int compareWellFormed(Comparator<V> cmp, Object a, Object b)
+    {
+        if (a == b)
+            return 0;
+        if (a == NEGATIVE_INFINITY | b == POSITIVE_INFINITY)
+            return -1;
+        if (b == NEGATIVE_INFINITY | a == POSITIVE_INFINITY)
+            return 1;
+        return cmp.compare((V) a, (V) b);
+    }
+
+    public static boolean isWellFormed(Object[] btree, Comparator<?> cmp)
+    {
+        return isWellFormed(cmp, btree, true, NEGATIVE_INFINITY, POSITIVE_INFINITY);
+    }
+
+    private static boolean isWellFormed(Comparator<?> cmp, Object[] node, boolean isRoot, Object min, Object max)
+    {
+        if (isEmpty(node))
+            return true;
+
+        if (cmp != null && !isNodeWellFormed(cmp, node, min, max))
+            return false;
+
+        int keyCount = shallowSize(node);
+        if (keyCount < 1)
+            return false;
+        if (!isRoot && keyCount < BRANCH_FACTOR/2 - 1)
+            return false;
+        if (keyCount >= BRANCH_FACTOR)
+            return false;
+
+        if (isLeaf(node))
+            return true;
+
+        int[] sizeMap = sizeMap(node);
+        int type = 0;
+        int size = 0;
+        // compare each child node with the branch element at the head of this node it corresponds with
+        for (int i = childOffset(node); i < childEndOffset(node) ; i++)
+        {
+            Object[] child = (Object[]) node[i];
+            Object localmax = i < node.length - 2 ? node[i - childOffset(node)] : max;
+            if (!isWellFormed(cmp, child, false, min, localmax))
+                return false;
+
+            type |= isLeaf(child) ? 1 : 2;
+            min = localmax;
+            size += size(child);
+            if (sizeMap[i - childOffset(node)] != size)
+                return false;
+            size += 1;
+        }
+
+        if (type >= 3)
+            return false;
+        return true; // either all leaves or all branches but not a mix
+    }
+
+    private static boolean isNodeWellFormed(Comparator<?> cmp, Object[] node, Object min, Object max)
+    {
+        Object previous = min;
+        int end = shallowSize(node);
+        for (int i = 0; i < end; i++)
+        {
+            Object current = node[i];
+            if (compareWellFormed(cmp, previous, current) >= 0)
+                return false;
+
+            previous = current;
+        }
+        return compareWellFormed(cmp, previous, max) < 0;
+    }
+
+    /**
+     * Build a tree of unknown size, in order.
+     *
+     * Can be used with {@link #reverseInSitu} to build a tree in reverse.
+     */
+    public static <V> FastBuilder<V> fastBuilder()
+    {
+        TinyThreadLocalPool.TinyPool<FastBuilder<?>> pool = FastBuilder.POOL.get();
+        FastBuilder<V> builder = (FastBuilder<V>) pool.poll();
+        if (builder == null)
+            builder = new FastBuilder<>();
+        builder.pool = pool;
+        return builder;
+    }
+
+    /**
+     * Base class for AbstractFastBuilder.BranchBuilder, LeafBuilder and AbstractFastBuilder,
+     * containing shared behaviour and declaring some useful abstract methods.
+     */
+    private static abstract class LeafOrBranchBuilder
+    {
+        final int height;
+        final LeafOrBranchBuilder child;
+        BranchBuilder parent;
+
+        /**
+         * The current buffer contents (if any) of the leaf or branch - always sized to contain a complete
+         * node of the form being constructed.  Always non-null, except briefly during overflow.
+         */
+        Object[] buffer;
+        /**
+         * The number of keys in our buffer, whether or not we are building a leaf or branch; if we are building
+         * a branch, we will ordinarily have the same number of children as well, except temporarily when finishing
+         * the construction of the node.
+         */
+        int count;
+
+        /**
+         * either
+         *  1) an empty leftover buffer from a past usage, which can be used when we exhaust {@code buffer}; or
+         *  2) a full {@code buffer} that has been parked until we next overflow, so we can steal some back
+         *     if we finish before reaching MIN_KEYS in {@code buffer}
+         */
+        Object[] savedBuffer;
+        /**
+         * The key we overflowed on when populating savedBuffer.  If null, {@link #savedBuffer} is logically empty.
+         */
+        Object savedNextKey;
+
+        LeafOrBranchBuilder(LeafOrBranchBuilder child)
+        {
+            this.height = child == null ? 1 : 1 + child.height;
+            this.child = child;
+        }
+
+        /**
+         * Do we have enough keys in the builder to construct at least one balanced node?
+         * We could have enough to build two.
+         */
+        final boolean isSufficient()
+        {
+            return hasOverflow() || count >= MIN_KEYS;
+        }
+
+        /**
+         * Do we have an already constructed node saved, that we can propagate or redistribute?
+         * This implies we are building two nodes, since {@link #savedNextKey} would overflow {@link #savedBuffer}
+         */
+        final boolean hasOverflow()
+        {
+            return savedNextKey != null;
+        }
+
+        /**
+         * Do we have an already constructed node saved AND insufficient keys in our buffer, so
+         * that we need to share the contents of {@link #savedBuffer} with {@link #buffer} to construct
+         * our results?
+         */
+        final boolean mustRedistribute()
+        {
+            return hasOverflow() && count < MIN_KEYS;
+        }
+
+        /**
+         * Are we empty, i.e. we have no contents in either {@link #buffer} or {@link #savedBuffer}
+         */
+        final boolean isEmpty()
+        {
+            return count == 0 && savedNextKey == null;
+        }
+
+        /**
+         * Drain the contents of this builder and build up to two nodes, as necessary.
+         * If {@code unode != null} and we are building a single node that is identical to it, use {@code unode} instead.
+         * If {@code propagateTo != null} propagate any nodes we build to it.
+         * @return the last node we construct
+         */
+        abstract Object[] drainAndPropagate(Object[] unode, int usz, BranchBuilder propagateTo);
+
+        /**
+         * Drain the contents of this builder and build at most one node.
+         * Requires {@code !hasOverflow()}
+         * @return the node we construct
+         */
+        abstract Object[] drain();
+
+        /**
+         * Takes a node that would logically occur directly preceding the current buffer contents,
+         * and the key that would separate them in a parent node, and prepends their contents
+         * to the current buffer's contents.  This can be used to redistribute already-propagated
+         * contents to a parent in cases where this is convenient (i.e. when transforming)
+         *
+         * @param predecessor directly preceding node
+         * @param predecessorNextKey key that would have separated predecessor from buffer contents
+         */
+        abstract void prepend(Object[] predecessor, Object predecessorNextKey);
+
+        /**
+         * Indicates if this builder produces dense nodes, i.e. those that are populated with MAX_KEYS
+         * at every level.  Only the last two children of any branch may be non-dense, and in some cases only
+         * the last two nodes in any tier of the tree.
+         *
+         * This flag switches whether or not we maintain a buffer of sizes, or use the globally shared contents of
+         * DENSE_SIZE_MAPS.
+         */
+        abstract boolean producesOnlyDense();
+
+        /**
+         * Ensure there is a {@code branch.parent}, and return it
+         */
+        final BranchBuilder ensureParent()
+        {
+            if (parent == null)
+                parent = new BranchBuilder(this);
+            parent.touched = true;
+            return parent;
+        }
+
+        /**
+         * Mark a branch builder as utilised, so that we must clear it when resetting any {@link AbstractFastBuilder}
+         * @return {@code branch}
+         */
+        static BranchBuilder touch(BranchBuilder branch)
+        {
+            branch.touched = true;
+            return branch;
+        }
+
+        /**
+         * @return the parent builder iff we will overflow or the parent has already been touched
+         */
+        final BranchBuilder parentIfInUse()
+        {
+            // if we have enough data to propagate, ensure there is a parent and return it
+            if (hasOverflow())
+                return ensureParent();
+
+            // otherwise return it only if it's already in use
+            return parent == null || !parent.touched ? null : parent;
+        }
+
+        /**
+         * A utility method for comparing a range of two arrays
+         */
+        static boolean areIdentical(Object[] a, int aOffset, Object[] b, int bOffset, int count)
+        {
+            for (int i = 0 ; i < count ; ++i)
+            {
+                if (a[i + aOffset] != b[i + bOffset])
+                    return false;
+            }
+            return true;
+        }
+
+        /**
+         * A utility method for comparing a range of two arrays
+         */
+        static boolean areIdentical(int[] a, int aOffset, int[] b, int bOffset, int count)
+        {
+            for (int i = 0 ; i < count ; ++i)
+            {
+                if (a[i + aOffset] != b[i + bOffset])
+                    return false;
+            }
+            return true;
+        }
+    }
+
+    /**
+     * LeafBuilder for methods pertaining specifically to building a leaf in an {@link AbstractFastBuilder}.
+     * Note that {@link AbstractFastBuilder} extends this class directly, however it is convenient to maintain
+     * distinct classes in the hierarchy for clarity of behaviour and intent.
+     */
+    private static abstract class LeafBuilder extends LeafOrBranchBuilder
+    {
+        long allocated;
+
+        LeafBuilder()
+        {
+            super(null);
+            buffer = new Object[MAX_KEYS];
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer, overflowing if necessary
+         */
+        public void addKey(Object nextKey)
+        {
+            if (count == MAX_KEYS)
+                overflow(nextKey);
+            else
+                buffer[count++] = nextKey;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void addKeyNoOverflow(Object nextKey)
+        {
+            buffer[count++] = nextKey;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void maybeAddKeyNoOverflow(Object nextKey)
+        {
+            buffer[count] = nextKey;
+            count += nextKey != null ? 1 : 0;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void maybeAddKey(Object nextKey)
+        {
+            if (count == MAX_KEYS)
+            {
+                if (nextKey != null)
+                    overflow(nextKey);
+            }
+            else
+            {
+                buffer[count] = nextKey;
+                count += nextKey != null ? 1 : 0;
+            }
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}, overflowing as necessary.
+         */
+        void copy(Object[] source, int offset, int length)
+        {
+            if (count + length > MAX_KEYS)
+            {
+                int copy = MAX_KEYS - count;
+                System.arraycopy(source, offset, buffer, count, copy);
+                offset += copy;
+//              implicitly:  count = MAX_KEYS;
+                overflow(source[offset++]);
+                length -= 1 + copy;
+            }
+            System.arraycopy(source, offset, buffer, count, length);
+            count += length;
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}; the caller specifying overflow is unnecessary
+         */
+        void copyNoOverflow(Object[] source, int offset, int length)
+        {
+            System.arraycopy(source, offset, buffer, count, length);
+            count += length;
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}, overflowing as necessary.
+         * Applies {@code updateF} to the contents before insertion.
+         */
+        <Insert, Existing> void copy(Object[] source, int offset, int length, UpdateFunction<Insert, Existing> updateF)
+        {
+            if (isSimple(updateF))
+            {
+                copy(source, offset, length);
+                return;
+            }
+
+            if (count + length > MAX_KEYS)
+            {
+                int copy = MAX_KEYS - count;
+                for (int i = 0 ; i < copy ; ++i)
+                    buffer[count + i] = updateF.apply((Insert) source[offset + i]);
+                offset += copy;
+//              implicitly:  leaf().count = MAX_KEYS;
+                overflow(updateF.apply((Insert) source[offset++]));
+                length -= 1 + copy;
+            }
+            for (int i = 0 ; i < length ; ++i)
+                buffer[count + i] = updateF.apply((Insert) source[offset + i]);
+            count += length;
+        }
+
+        /**
+         * {@link #buffer} is full, and we need to make room either by populating {@link #savedBuffer},
+         * propagating its current contents, if any, to {@link #parent}
+         */
+        void overflow(Object nextKey)
+        {
+            if (hasOverflow())
+                propagateOverflow();
+
+            // precondition: count == MAX_KEYS and savedNextKey == null
+
+            Object[] newBuffer = savedBuffer;
+            if (newBuffer == null)
+                newBuffer = new Object[MAX_KEYS];
+
+            savedBuffer = buffer;
+            savedNextKey = nextKey;
+            buffer = newBuffer;
+            count = 0;
+        }
+
+        /**
+         * Redistribute the contents of {@link #savedBuffer} into {@link #buffer}, finalise {@link #savedBuffer} and flush upwards.
+         * Invoked when we are building from {@link #buffer}, have insufficient values but a complete leaf in {@link #savedBuffer}
+         * @return the size of the leaf we flushed to our parent from {@link #savedBuffer}
+         */
+        Object[] redistributeOverflowAndDrain()
+        {
+            Object[] newLeaf = redistributeAndDrain(savedBuffer, MAX_KEYS, savedNextKey);
+            savedNextKey = null;
+            return newLeaf;
+        }
+
+        /**
+         * Redistribute the contents of {@link #buffer} and an immediate predecessor into a new leaf,
+         * then construct a new predecessor with the remaining contents and propagate up to our parent
+         * Invoked when we are building from {@link #buffer}, have insufficient values but either a complete
+         * leaf in {@link #savedBuffer} or can exfiltrate one from our parent to redistribute.
+         *
+         * @return the second of the two new leaves
+         */
+        Object[] redistributeAndDrain(Object[] pred, int predSize, Object predNextKey)
+        {
+            // precondition: savedLeafCount == MAX_KEYS && leaf().count < MIN_KEYS
+            // ensure we have at least MIN_KEYS in leaf().buffer
+            // first shift leaf().buffer and steal some keys from leaf().savedBuffer and leaf().savedBufferNextKey
+            int steal = MIN_KEYS - count;
+            Object[] newLeaf = new Object[MIN_KEYS];
+            System.arraycopy(pred,  predSize - (steal - 1), newLeaf, 0,     steal - 1);
+                                                            newLeaf [steal - 1] = predNextKey;
+            System.arraycopy(buffer, 0,                     newLeaf, steal, count);
+
+            // then create a leaf out of the remainder of savedBuffer
+            int newPredecessorCount = predSize - steal;
+            Object[] newPredecessor = new Object[newPredecessorCount | 1];
+            System.arraycopy(pred, 0, newPredecessor, 0, newPredecessorCount);
+            if (allocated >= 0)
+                allocated += ObjectSizes.sizeOfReferenceArray(newPredecessorCount | 1);
+            ensureParent().addChildAndNextKey(newPredecessor, newPredecessorCount, pred[newPredecessorCount]);
+            return newLeaf;
+        }
+
+        /**
+         * Invoked to fill our {@link #buffer} to >= MIN_KEYS with data ocurring before {@link #buffer};
+         * possibly instead fills {@link #savedBuffer}
+         *
+         * @param pred directly preceding node
+         * @param predNextKey key that would have separated predecessor from buffer contents
+         */
+        void prepend(Object[] pred, Object predNextKey)
+        {
+            int predSize = sizeOfLeaf(pred);
+            int newKeys = 1 + predSize;
+            if (newKeys + count <= MAX_KEYS)
+            {
+                System.arraycopy(buffer, 0, buffer, newKeys, count);
+                System.arraycopy(pred,   0, buffer, 0,       predSize);
+                                            buffer [predSize] = predNextKey;
+                count += newKeys;
+            }
+            else
+            {
+                if (savedBuffer == null)
+                    savedBuffer = new Object[MAX_KEYS];
+                System.arraycopy(pred, 0, savedBuffer, 0, predSize);
+                if (predSize == MAX_KEYS)
+                {
+                    savedNextKey = predNextKey;
+                }
+                else
+                {
+                    int removeKeys = MAX_KEYS - predSize;
+                    count -= removeKeys;
+                                                         savedBuffer [predSize] = predNextKey;
+                    System.arraycopy(buffer, 0,          savedBuffer, predSize + 1, MAX_KEYS - newKeys);
+                                                         savedNextKey      = buffer[MAX_KEYS - newKeys];
+                    System.arraycopy(buffer, removeKeys, buffer,      0,            count);
+                }
+            }
+        }
+
+        /**
+         * Invoked when we want to add a key to the leaf buffer, but it is full
+         */
+        void propagateOverflow()
+        {
+            // propagate the leaf we have saved in altLeafBuffer
+            // precondition: savedLeafCount == MAX_KEYS
+            if (allocated >= 0)
+                allocated += ObjectSizes.sizeOfReferenceArray(MAX_KEYS);
+            ensureParent().addChildAndNextKey(savedBuffer, MAX_KEYS, savedNextKey);
+            savedBuffer = null;
+            savedNextKey = null;
+        }
+
+        /**
+         * Construct a new leaf from the contents of {@link #buffer}, unless the contents have not changed
+         * from {@code unode}, in which case return {@code unode} to avoid allocating unnecessary objects.
+         */
+        Object[] drainAndPropagate(Object[] unode, int usz, BranchBuilder propagateTo)
+        {
+            Object[] leaf;
+            int sizeOfLeaf;
+            if (mustRedistribute())
+            {
+                // we have too few items, so spread the two buffers across two new nodes
+                leaf = redistributeOverflowAndDrain();
+                sizeOfLeaf = MIN_KEYS;
+            }
+            else if (!hasOverflow() && count == usz && areIdentical(buffer, 0, unode, 0, usz))
+            {
+                // we have exactly the same contents as the original node, so reuse it
+                leaf = unode;
+                sizeOfLeaf = usz;
+            }
+            else
+            {
+                // we have maybe one saved full buffer, and one buffer with sufficient contents to copy
+                if (hasOverflow())
+                    propagateOverflow();
+
+                sizeOfLeaf = count;
+                leaf = drain();
+                if (allocated >= 0 && sizeOfLeaf > 0)
+                    allocated += ObjectSizes.sizeOfReferenceArray(sizeOfLeaf | 1) - (unode == null ? 0 : ObjectSizes.sizeOfArray(unode));
+            }
+
+            count = 0;
+            if (propagateTo != null)
+                propagateTo.addChild(leaf, sizeOfLeaf);
+            return leaf;
+        }
+
+        /**
+         * Construct a new leaf from the contents of {@code leaf().buffer}, unless the contents have not changed
+         * from {@code unode}, in which case return {@code unode} to avoid allocating unnecessary objects.
+         */
+        Object[] drain()
+        {
+            assert !hasOverflow();
+            if (count == 0)
+                return empty();
+
+            Object[] newLeaf = new Object[count | 1];
+            System.arraycopy(buffer, 0, newLeaf, 0, count);
+            count = 0;
+            return newLeaf;
+        }
+    }
+
+    static class BranchBuilder extends LeafOrBranchBuilder
+    {
+        final LeafBuilder leaf;
+
+        /** sizes of the children in {@link #buffer}. If null, we only produce dense nodes. */
+        int[] sizes;
+        /** sizes of the children in {@link #savedBuffer} */
+        int[] savedSizes;
+        /** marker to limit unnecessary work with unused levels, esp. on reset */
+        boolean touched;
+
+        BranchBuilder(LeafOrBranchBuilder child)
+        {
+            super(child);
+            buffer = new Object[2 * (MAX_KEYS + 1)];
+            if (!child.producesOnlyDense())
+                sizes = new int[MAX_KEYS + 1];
+            this.leaf = child instanceof LeafBuilder ? (LeafBuilder) child : ((BranchBuilder) child).leaf;
+        }
+
+        /**
+         * Ensure there is room to add another key to {@code branchBuffers[branchIndex]}, and add it;
+         * invoke {@link #overflow} if necessary
+         */
+        void addKey(Object key)
+        {
+            if (count == MAX_KEYS)
+                overflow(key);
+            else
+                buffer[count++] = key;
+        }
+
+        /**
+         * To be invoked when there's a key already inserted to the buffer that requires a corresponding
+         * right-hand child, for which the buffers are sized to ensure there is always room.
+         */
+        void addChild(Object[] child, int sizeOfChild)
+        {
+            buffer[MAX_KEYS + count] = child;
+            recordSizeOfChild(sizeOfChild);
+        }
+
+        void recordSizeOfChild(int sizeOfChild)
+        {
+            if (sizes != null)
+                sizes[count] = sizeOfChild;
+        }
+
+        /**
+         * See {@link BranchBuilder#addChild(Object[], int)}
+         */
+        void addChild(Object[] child)
+        {
+            addChild(child, sizes == null ? 0 : size(child));
+        }
+
+        /**
+         * Insert a new child into a parent branch, when triggered by {@code overflowLeaf} or {@code overflowBranch}
+         */
+        void addChildAndNextKey(Object[] newChild, int newChildSize, Object nextKey)
+        {
+            // we should always have room for a child to the right of any key we have previously inserted
+            buffer[MAX_KEYS + count] = newChild;
+            recordSizeOfChild(newChildSize);
+            // but there may not be room for another key
+            addKey(nextKey);
+        }
+
+        /**
+         * Invoked when we want to add a key to the leaf buffer, but it is full
+         */
+        void propagateOverflow()
+        {
+            // propagate the leaf we have saved in leaf().savedBuffer
+            if (leaf.allocated >= 0)
+                leaf.allocated += ObjectSizes.sizeOfReferenceArray(2 * (1 + MAX_KEYS));
+            int size = setOverflowSizeMap(savedBuffer, MAX_KEYS);
+            ensureParent().addChildAndNextKey(savedBuffer, size, savedNextKey);
+            savedBuffer = null;
+            savedNextKey = null;
+        }
+
+        /**
+         * Invoked when a branch already contains {@code MAX_KEYS}, and another child is ready to be added.
+         * Creates a new neighbouring node containing MIN_KEYS items, shifting back the remaining MIN_KEYS+1
+         * items to the start of the buffer(s).
+         */
+        void overflow(Object nextKey)
+        {
+            if (hasOverflow())
+                propagateOverflow();
+
+            Object[] restoreBuffer = savedBuffer;
+            int[] restoreSizes = savedSizes;
+
+            savedBuffer = buffer;
+            savedSizes = sizes;
+            savedNextKey = nextKey;
+
+            sizes = restoreSizes == null && savedSizes != null ? new int[MAX_KEYS + 1] : restoreSizes;
+            buffer = restoreBuffer == null ? new Object[2 * (MAX_KEYS + 1)] : restoreBuffer;
+            count = 0;
+        }
+
+        /**
+         * Redistribute the contents of branch.savedBuffer into branch.buffer, finalise savedBuffer and flush upwards.
+         * Invoked when we are building from branch, have insufficient values but a complete branch in savedBuffer.
+         * @return the size of the branch we flushed to our parent from savedBuffer
+         */
+        Object[] redistributeOverflowAndDrain()
+        {
+            // now ensure we have at least MIN_KEYS in buffer
+            // both buffer and savedBuffer should be balanced, so that we have count+1 and MAX_KEYS+1 children respectively
+            // we need to utilise savedNextKey, so  we want to take {@code steal-1} keys from savedBuffer, {@code steal) children
+            // and the dangling key we use in place of savedNextKey for our parent key.
+            int steal = MIN_KEYS - count;
+            Object[] newBranch = new Object[2 * (MIN_KEYS + 1)];
+            System.arraycopy(savedBuffer,     MAX_KEYS - (steal - 1), newBranch, 0,         steal - 1);
+            newBranch[steal - 1] = savedNextKey;
+            System.arraycopy(buffer,          0,                      newBranch, steal,     count);
+            System.arraycopy(savedBuffer, 2 * MAX_KEYS + 1 - steal,   newBranch, MIN_KEYS,  steal);
+            System.arraycopy(buffer,          MAX_KEYS,               newBranch, MIN_KEYS + steal, count + 1);
+            setRedistributedSizeMap(newBranch, steal);
+
+            // then create a branch out of the remainder of savedBuffer
+            int savedBranchCount = MAX_KEYS - steal;
+            Object[] savedBranch = new Object[2 * (savedBranchCount + 1)];
+            System.arraycopy(savedBuffer, 0,        savedBranch, 0,                savedBranchCount);
+            System.arraycopy(savedBuffer, MAX_KEYS, savedBranch, savedBranchCount, savedBranchCount + 1);
+            int savedBranchSize = setOverflowSizeMap(savedBranch, savedBranchCount);
+            if (leaf.allocated >= 0)
+                leaf.allocated += ObjectSizes.sizeOfReferenceArray(2 * (1 + savedBranchCount));
+            ensureParent().addChildAndNextKey(savedBranch, savedBranchSize, savedBuffer[savedBranchCount]);
+            savedNextKey = null;
+
+            return newBranch;
+        }
+
+        /**
+         * See {@link LeafOrBranchBuilder#prepend(Object[], Object)}
+         */
+        void prepend(Object[] pred, Object predNextKey)
+        {
+            // assumes sizes != null, since only makes sense to use this method in that context
+
+            int predKeys = shallowSizeOfBranch(pred);
+            int[] sizeMap = (int[]) pred[2 * predKeys + 1];
+            int newKeys = 1 + predKeys;
+            if (newKeys + count <= MAX_KEYS)
+            {
+                System.arraycopy(buffer,    0,        buffer, newKeys,            count);
+                System.arraycopy(sizes,     0,        sizes,  newKeys,            count + 1);
+                System.arraycopy(buffer,    MAX_KEYS, buffer, MAX_KEYS + newKeys, count + 1);
+
+                System.arraycopy(pred,      0,        buffer, 0,                  predKeys);
+                buffer[predKeys] = predNextKey;
+                System.arraycopy(pred,      predKeys, buffer, MAX_KEYS,           predKeys + 1);
+                copySizeMapToSizes(sizeMap, 0,        sizes,  0,                  predKeys + 1);
+                count += newKeys;
+            }
+            else
+            {
+                if (savedBuffer == null)
+                {
+                    savedBuffer = new Object[2 * (1 + MAX_KEYS)];
+                    savedSizes = new int[1 + MAX_KEYS];
+                }
+
+                System.arraycopy(  pred,    0,        savedBuffer, 0,        predKeys);
+                System.arraycopy(  pred,    predKeys, savedBuffer, MAX_KEYS, predKeys + 1);
+                copySizeMapToSizes(sizeMap, 0,        savedSizes,  0,        predKeys + 1);
+                if (newKeys == MAX_KEYS + 1)
+                {
+                    savedNextKey = predNextKey;
+                }
+                else
+                {
+                    int removeKeys = (1 + MAX_KEYS - newKeys);
+                    int remainingKeys = count - removeKeys;
+
+                    savedBuffer [predKeys] = predNextKey;
+                    System.arraycopy(buffer, 0,                     savedBuffer, newKeys,            MAX_KEYS     - newKeys);
+                    savedNextKey =            buffer[MAX_KEYS     - newKeys];
+                    System.arraycopy(sizes,  0,                     savedSizes,  newKeys,            MAX_KEYS + 1 - newKeys);
+                    System.arraycopy(buffer, MAX_KEYS,              savedBuffer, MAX_KEYS + newKeys, MAX_KEYS + 1 - newKeys);
+                    System.arraycopy(buffer, removeKeys,            buffer,      0,                  remainingKeys);
+                    System.arraycopy(buffer, MAX_KEYS + removeKeys, buffer,      MAX_KEYS,           remainingKeys + 1);
+                    System.arraycopy(sizes,  removeKeys,            sizes,       0,                  remainingKeys + 1);
+                    count = remainingKeys;
+                }
+            }
+        }
+
+        boolean producesOnlyDense()
+        {
+            return sizes == null;
+        }
+
+        /**
+         * Construct a new branch from the contents of {@code branchBuffers[branchIndex]}, unless the contents have
+         * not changed from {@code unode}, in which case return {@code unode} to avoid allocating unnecessary objects.
+         */
+        Object[] drainAndPropagate(Object[] unode, int usz, BranchBuilder propagateTo)
+        {
+            int sizeOfBranch;
+            Object[] branch;
+            if (mustRedistribute())
+            {
+                branch = redistributeOverflowAndDrain();
+                sizeOfBranch = sizeOfBranch(branch);
+            }
+            else if (!hasOverflow() && usz == count
+                     && areIdentical(buffer, 0,        unode, 0,   usz)
+                     && areIdentical(buffer, MAX_KEYS, unode, usz, usz + 1))
+            {
+                branch = unode;
+                sizeOfBranch = sizeOfBranch(branch);
+            }
+            else
+            {
+                if (hasOverflow())
+                    propagateOverflow();
+
+                branch = new Object[2 * (count + 1)];
+                System.arraycopy(buffer, 0,        branch, 0,     count);
+                System.arraycopy(buffer, MAX_KEYS, branch, count, count + 1);
+                sizeOfBranch = setDrainSizeMap(unode, usz, branch, count);
+            }
+
+            count = 0;
+            if (propagateTo != null)
+                propagateTo.addChild(branch, sizeOfBranch);
+
+            return branch;
+        }
+
+        /**
+         * Construct a new branch from the contents of {@code branchBuffers[branchIndex]}, unless the contents have
+         * not changed from {@code unode}, in which case return {@code unode} to avoid allocating unnecessary objects.

Review comment:
       Out-of-sync doc.

##########
File path: src/java/org/apache/cassandra/utils/btree/BTree.java
##########
@@ -1448,4 +1920,2355 @@ private static boolean isStopSentinel(long v)
     {
         return accumulate(btree, accumulator, arg, null, null, initialValue);
     }
+
+    /**
+     * Calculate the minimum height needed for this size of tree
+     *
+     * @param size the tree size
+     * @return the minimum height needed for this size of tree
+     */
+    private static int minHeight(int size)
+    {
+        return heightAtSize2n(size, BRANCH_SHIFT);
+    }
+
+    private static int heightAtSize2n(int size, int branchShift)
+    {
+        //    branch factor                 =  1 <<  branchShift
+        // => full size at height            = (1 << (branchShift * height)) - 1
+        // => full size at height + 1        =  1 << (branchShift * height)
+        // => shift(full size at height + 1) = branchShift * height
+        // => shift(full size at height + 1) / branchShift = height
+        int indexOfHighestBit = 63 - Long.numberOfLeadingZeros(size + 1);
+        indexOfHighestBit += (1L << indexOfHighestBit) == (size + 1) ? 0 : 1;
+        return (branchShift - 1 + indexOfHighestBit) / branchShift;
+    }
+
+    private static int[][] buildBalancedSizeMaps(int branchShift)
+    {
+        int count = (32 / branchShift) - 1;
+        int childCount = 1 << branchShift;
+        int[][] sizeMaps = new int[count][childCount];
+        for (int height = 0 ; height < count ; ++height)
+        {
+            int childSize = treeSize2n(height + 1, branchShift);
+            int size = 0;
+            int[] sizeMap = sizeMaps[height];
+            for (int i = 0 ; i < childCount ; ++i)
+            {
+                sizeMap[i] = size += childSize;
+                size += 1;
+            }
+        }
+        return sizeMaps;
+    }
+
+    // simply utility to reverse the contents of array[from..to)
+    private static void reverse(Object[] array, int from, int to)
+    {
+        int mid = (from + to) / 2;
+        for (int i = from ; i < mid ; i++)
+        {
+            int j = to - (1 + i - from);
+            Object tmp = array[i];
+            array[i] = array[j];
+            array[j] = tmp;
+        }
+    }
+
+    // simply utility to reverse the contents of array[from..to)
+    private static void reverse(int[] array, int from, int to)
+    {
+        int mid = (from + to) / 2;
+        for (int i = from ; i < mid ; i++)
+        {
+            int j = to - (1 + i - from);
+            int tmp = array[i];
+            array[i] = array[j];
+            array[j] = tmp;
+        }
+    }
+
+    /**
+     * Mutate an array of child sizes into a cumulative sizeMap, returning the total size
+     */
+    private static int sizesToSizeMap(int[] sizeMap)
+    {
+        int total = sizeMap[0];
+        for (int i = 1 ; i < sizeMap.length ; ++i)
+            sizeMap[i] = total += 1 + sizeMap[i];
+        return total;
+    }
+
+    private static int sizesToSizeMap(int[] sizes, int count)
+    {
+        int total = sizes[0];
+        for (int i = 1 ; i < count ; ++i)
+            sizes[i] = total += 1 + sizes[i];
+        return total;
+    }
+
+    /**
+     * Mutate an array of child sizes into a cumulative sizeMap, returning the total size
+     */
+    private static void sizeMapToSizes(int[] sizeMap)
+    {
+        for (int i = sizeMap.length ; i > 1 ; --i)
+            sizeMap[i] -= 1 + sizeMap[i - 1];
+    }
+
+    /**
+     * A simple utility method to handle a null upper bound that we treat as infinity
+     */
+    private static <Compare> int compareWithMaybeInfinity(Comparator<? super Compare> comparator, Compare key, Compare ub)
+    {
+        if (ub == null)
+            return -1;
+        return comparator.compare(key, ub);
+    }
+
+    /**
+     * Perform {@link #exponentialSearch} on {@code in[from..to)}, treating a {@code find} of {@code null} as infinity.
+     */
+    static <Compare> int exponentialSearchForMaybeInfinity(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare find)
+    {
+        if (find == null)
+            return -(1 + to);
+        return exponentialSearch(comparator, in, from, to, find);
+    }
+
+    /**
+     * Equivalent to {@link Arrays#binarySearch}, only more efficient algorithmically for linear merges.
+     * Binary search has worst case complexity {@code O(n.lg n)} for a linear merge, whereas exponential search
+     * has a worst case of {@code O(n)}. However compared to a simple linear merge, the best case for exponential
+     * search is {@code O(lg(n))} instead of {@code O(n)}.
+     */
+    private static <Compare> int exponentialSearch(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare find)
+    {
+        int step = 0;
+        while (from + step < to)
+        {
+            int i = from + step;
+            int c = comparator.compare(find, (Compare) in[i]);
+            if (c < 0)
+            {
+                to = i;
+                break;
+            }
+            if (c == 0)
+                return i;
+            from = i + 1;
+            step = step * 2 + 1; // jump in perfect binary search increments
+        }
+        return Arrays.binarySearch((Compare[]) in, from, to, find, comparator);
+    }
+
+    /**
+     * Perform {@link #exponentialSearch} on {@code in[from..to)}; if the value falls outside of the range of these
+     * elements, test against {@code ub} as though it occurred at position {@code to}
+     *
+     * @return same as {@link Arrays#binarySearch} if {@code find} occurs in the range {@code [in[from]..in[to])};
+     * otherwise the insertion position {@code -(1+to)} if {@code find} is less than {@code ub}, and {@code -(2+t)}
+     * if it is greater than or equal to.
+     *
+     * {@code ub} may be {@code null}, representing infinity.
+     */
+    static <Compare> int exponentialSearchWithUpperBound(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare ub, Compare find)
+    {
+        int step = 0;
+        while (true)
+        {
+            int i = from + step;
+            if (i >= to)
+            {
+                int c = compareWithMaybeInfinity(comparator, find, ub);
+                if (c >= 0)
+                    return -(2 + to);
+                break;
+            }
+            int c = comparator.compare(find, (Compare) in[i]);
+            if (c < 0)
+            {
+                to = i;
+                break;
+            }
+            if (c == 0)
+                return i;
+            from = i + 1;
+            step = step * 2 + 1; // jump in perfect binary search increments
+        }
+        return Arrays.binarySearch((Compare[]) in, from, to, find, comparator);
+    }
+    
+    /**
+     * Compute the size-in-bytes of full trees of cardinality {@code branchFactor^height - 1}
+     */
+    private static long[] sizeOnHeapOfPerfectTrees(int branchShift)
+    {
+        long[] result = new long[heightAtSize2n(Integer.MAX_VALUE, branchShift)];
+        int branchFactor = 1 << branchShift;
+        result[0] = branchFactor - 1;
+        for (int i = 1 ; i < result.length ; ++i)
+            result[i] = sizeOnHeapOfPerfectTree(i + 1, branchFactor);
+        return result;
+    }
+
+    /**
+     * Compute the size-in-bytes of a full tree of cardinality {@code branchFactor^height - 1}
+     * TODO: test
+     */
+    private static long sizeOnHeapOfPerfectTree(int height, int branchShift)
+    {
+        int branchFactor = 1 << branchShift;
+        long branchSize = ObjectSizes.sizeOfReferenceArray(branchFactor * 2);
+        int branchCount = height == 2 ? 1 : 2 + treeSize2n(height - 2, branchShift);
+        long leafSize = ObjectSizes.sizeOfReferenceArray((branchFactor - 1)|1);
+        int leafCount = 1 + treeSize2n(height - 1, branchShift);
+        return (branchSize * branchCount) + (leafSize * leafCount);
+    }
+
+    /**
+     * @return the actual height of {@code tree}
+     */
+    public static int height(Object[] tree)
+    {
+        if (isLeaf(tree))
+            return 1;
+
+        int height = 1;
+        while (!isLeaf(tree))
+        {
+            height++;
+            tree = (Object[]) tree[shallowSizeOfBranch(tree)];
+        }
+        return height;
+    }
+
+    /**
+     * @return the minimum representable size at {@code height}, for an internal (non-root) node.
+     */
+    private static int sparseSize(int height)
+    {
+        return treeSize2n(height, BRANCH_SHIFT - 1);
+    }
+
+    /**
+     * @return the maximum representable size at {@code height}.
+     */
+    private static int denseSize(int height)
+    {
+        return treeSize2n(height, BRANCH_SHIFT);
+    }
+
+    /**
+     * @return the maximum representable size at {@code height}.
+     */
+    private static int checkedDenseSize(int height)
+    {
+        assert height * BRANCH_SHIFT < 32;
+        return denseSize(height);
+    }
+
+    /**
+     * Computes the number of nodes in a full tree of height {@code height}
+     * and with {@code 2^branchShift} branch factor.
+     * i.e. computes {@code (2^branchShift)^height - 1}
+     */
+    private static int treeSize2n(int height, int branchShift)
+    {
+        return (1 << (branchShift * height)) - 1;
+    }
+
+    // TODO: test
+    private static int maxRootHeight(int size)
+    {
+        if (size <= BRANCH_FACTOR)
+            return 1;
+        return 1 + heightAtSize2n((size - 1) / 2, BRANCH_SHIFT - 1);
+    }
+    
+    private static int sizeOfBranch(Object[] branch)
+    {
+        int length = branch.length;
+        // length - 1 == getChildEnd == getPositionOfSizeMap
+        // (length / 2) - 1 == getChildCount - 1 == position of full tree size
+        // hard code this, as will be used often;
+        return ((int[]) branch[length - 1])[(length / 2) - 1];
+    }
+
+    /**
+     * Checks if the UpdateFunction is an instance of {@code UpdateFunction.Simple}.
+     */
+    private static boolean isSimple(UpdateFunction<?, ?> updateF)
+    {
+        return updateF instanceof UpdateFunction.Simple;
+    }
+
+    /**
+     * @return the size map for the branch node
+     */
+    static int[] sizeMap(Object[] branch)
+    {
+        return (int[]) branch[branch.length - 1];
+    }
+
+    public static long sizeOnHeapOf(Object[] tree)
+    {
+        long size = ObjectSizes.sizeOfArray(tree);
+        if (isLeaf(tree))
+            return size;
+        for (int i = childOffset(tree); i < childEndOffset(tree) ; i++)
+            size += sizeOnHeapOf((Object[]) tree[i]);
+        size += ObjectSizes.sizeOfArray(sizeMap(tree)); // may overcount, since we share size maps
+        return size;
+    }
+
+    // Arbitrary boundaries
+    private static Object POSITIVE_INFINITY = new Object();
+    private static Object NEGATIVE_INFINITY = new Object();
+
+    /** simple static wrapper to calls to cmp.compare() which checks if either a or b are Special (i.e. represent an infinity) */
+    private static <V> int compareWellFormed(Comparator<V> cmp, Object a, Object b)
+    {
+        if (a == b)
+            return 0;
+        if (a == NEGATIVE_INFINITY | b == POSITIVE_INFINITY)
+            return -1;
+        if (b == NEGATIVE_INFINITY | a == POSITIVE_INFINITY)
+            return 1;
+        return cmp.compare((V) a, (V) b);
+    }
+
+    public static boolean isWellFormed(Object[] btree, Comparator<?> cmp)
+    {
+        return isWellFormed(cmp, btree, true, NEGATIVE_INFINITY, POSITIVE_INFINITY);
+    }
+
+    private static boolean isWellFormed(Comparator<?> cmp, Object[] node, boolean isRoot, Object min, Object max)
+    {
+        if (isEmpty(node))
+            return true;
+
+        if (cmp != null && !isNodeWellFormed(cmp, node, min, max))
+            return false;
+
+        int keyCount = shallowSize(node);
+        if (keyCount < 1)
+            return false;
+        if (!isRoot && keyCount < BRANCH_FACTOR/2 - 1)
+            return false;
+        if (keyCount >= BRANCH_FACTOR)
+            return false;
+
+        if (isLeaf(node))
+            return true;
+
+        int[] sizeMap = sizeMap(node);
+        int type = 0;
+        int size = 0;
+        // compare each child node with the branch element at the head of this node it corresponds with
+        for (int i = childOffset(node); i < childEndOffset(node) ; i++)
+        {
+            Object[] child = (Object[]) node[i];
+            Object localmax = i < node.length - 2 ? node[i - childOffset(node)] : max;
+            if (!isWellFormed(cmp, child, false, min, localmax))
+                return false;
+
+            type |= isLeaf(child) ? 1 : 2;
+            min = localmax;
+            size += size(child);
+            if (sizeMap[i - childOffset(node)] != size)
+                return false;
+            size += 1;
+        }
+
+        if (type >= 3)
+            return false;
+        return true; // either all leaves or all branches but not a mix
+    }
+
+    private static boolean isNodeWellFormed(Comparator<?> cmp, Object[] node, Object min, Object max)
+    {
+        Object previous = min;
+        int end = shallowSize(node);
+        for (int i = 0; i < end; i++)
+        {
+            Object current = node[i];
+            if (compareWellFormed(cmp, previous, current) >= 0)
+                return false;
+
+            previous = current;
+        }
+        return compareWellFormed(cmp, previous, max) < 0;
+    }
+
+    /**
+     * Build a tree of unknown size, in order.
+     *
+     * Can be used with {@link #reverseInSitu} to build a tree in reverse.
+     */
+    public static <V> FastBuilder<V> fastBuilder()
+    {
+        TinyThreadLocalPool.TinyPool<FastBuilder<?>> pool = FastBuilder.POOL.get();
+        FastBuilder<V> builder = (FastBuilder<V>) pool.poll();
+        if (builder == null)
+            builder = new FastBuilder<>();
+        builder.pool = pool;
+        return builder;
+    }
+
+    /**
+     * Base class for AbstractFastBuilder.BranchBuilder, LeafBuilder and AbstractFastBuilder,
+     * containing shared behaviour and declaring some useful abstract methods.
+     */
+    private static abstract class LeafOrBranchBuilder
+    {
+        final int height;
+        final LeafOrBranchBuilder child;
+        BranchBuilder parent;
+
+        /**
+         * The current buffer contents (if any) of the leaf or branch - always sized to contain a complete
+         * node of the form being constructed.  Always non-null, except briefly during overflow.
+         */
+        Object[] buffer;
+        /**
+         * The number of keys in our buffer, whether or not we are building a leaf or branch; if we are building
+         * a branch, we will ordinarily have the same number of children as well, except temporarily when finishing
+         * the construction of the node.
+         */
+        int count;
+
+        /**
+         * either
+         *  1) an empty leftover buffer from a past usage, which can be used when we exhaust {@code buffer}; or
+         *  2) a full {@code buffer} that has been parked until we next overflow, so we can steal some back
+         *     if we finish before reaching MIN_KEYS in {@code buffer}
+         */
+        Object[] savedBuffer;
+        /**
+         * The key we overflowed on when populating savedBuffer.  If null, {@link #savedBuffer} is logically empty.
+         */
+        Object savedNextKey;
+
+        LeafOrBranchBuilder(LeafOrBranchBuilder child)
+        {
+            this.height = child == null ? 1 : 1 + child.height;
+            this.child = child;
+        }
+
+        /**
+         * Do we have enough keys in the builder to construct at least one balanced node?
+         * We could have enough to build two.
+         */
+        final boolean isSufficient()
+        {
+            return hasOverflow() || count >= MIN_KEYS;
+        }
+
+        /**
+         * Do we have an already constructed node saved, that we can propagate or redistribute?
+         * This implies we are building two nodes, since {@link #savedNextKey} would overflow {@link #savedBuffer}
+         */
+        final boolean hasOverflow()
+        {
+            return savedNextKey != null;
+        }
+
+        /**
+         * Do we have an already constructed node saved AND insufficient keys in our buffer, so
+         * that we need to share the contents of {@link #savedBuffer} with {@link #buffer} to construct
+         * our results?
+         */
+        final boolean mustRedistribute()
+        {
+            return hasOverflow() && count < MIN_KEYS;
+        }
+
+        /**
+         * Are we empty, i.e. we have no contents in either {@link #buffer} or {@link #savedBuffer}
+         */
+        final boolean isEmpty()
+        {
+            return count == 0 && savedNextKey == null;
+        }
+
+        /**
+         * Drain the contents of this builder and build up to two nodes, as necessary.
+         * If {@code unode != null} and we are building a single node that is identical to it, use {@code unode} instead.
+         * If {@code propagateTo != null} propagate any nodes we build to it.
+         * @return the last node we construct
+         */
+        abstract Object[] drainAndPropagate(Object[] unode, int usz, BranchBuilder propagateTo);
+
+        /**
+         * Drain the contents of this builder and build at most one node.
+         * Requires {@code !hasOverflow()}
+         * @return the node we construct
+         */
+        abstract Object[] drain();
+
+        /**
+         * Takes a node that would logically occur directly preceding the current buffer contents,
+         * and the key that would separate them in a parent node, and prepends their contents
+         * to the current buffer's contents.  This can be used to redistribute already-propagated
+         * contents to a parent in cases where this is convenient (i.e. when transforming)
+         *
+         * @param predecessor directly preceding node
+         * @param predecessorNextKey key that would have separated predecessor from buffer contents
+         */
+        abstract void prepend(Object[] predecessor, Object predecessorNextKey);
+
+        /**
+         * Indicates if this builder produces dense nodes, i.e. those that are populated with MAX_KEYS
+         * at every level.  Only the last two children of any branch may be non-dense, and in some cases only
+         * the last two nodes in any tier of the tree.
+         *
+         * This flag switches whether or not we maintain a buffer of sizes, or use the globally shared contents of
+         * DENSE_SIZE_MAPS.
+         */
+        abstract boolean producesOnlyDense();
+
+        /**
+         * Ensure there is a {@code branch.parent}, and return it
+         */
+        final BranchBuilder ensureParent()
+        {
+            if (parent == null)
+                parent = new BranchBuilder(this);
+            parent.touched = true;
+            return parent;
+        }
+
+        /**
+         * Mark a branch builder as utilised, so that we must clear it when resetting any {@link AbstractFastBuilder}
+         * @return {@code branch}
+         */
+        static BranchBuilder touch(BranchBuilder branch)
+        {
+            branch.touched = true;
+            return branch;
+        }
+
+        /**
+         * @return the parent builder iff we will overflow or the parent has already been touched
+         */
+        final BranchBuilder parentIfInUse()
+        {
+            // if we have enough data to propagate, ensure there is a parent and return it
+            if (hasOverflow())
+                return ensureParent();
+
+            // otherwise return it only if it's already in use
+            return parent == null || !parent.touched ? null : parent;
+        }
+
+        /**
+         * A utility method for comparing a range of two arrays
+         */
+        static boolean areIdentical(Object[] a, int aOffset, Object[] b, int bOffset, int count)
+        {
+            for (int i = 0 ; i < count ; ++i)
+            {
+                if (a[i + aOffset] != b[i + bOffset])
+                    return false;
+            }
+            return true;
+        }
+
+        /**
+         * A utility method for comparing a range of two arrays
+         */
+        static boolean areIdentical(int[] a, int aOffset, int[] b, int bOffset, int count)
+        {
+            for (int i = 0 ; i < count ; ++i)
+            {
+                if (a[i + aOffset] != b[i + bOffset])
+                    return false;
+            }
+            return true;
+        }
+    }
+
+    /**
+     * LeafBuilder for methods pertaining specifically to building a leaf in an {@link AbstractFastBuilder}.
+     * Note that {@link AbstractFastBuilder} extends this class directly, however it is convenient to maintain
+     * distinct classes in the hierarchy for clarity of behaviour and intent.
+     */
+    private static abstract class LeafBuilder extends LeafOrBranchBuilder
+    {
+        long allocated;
+
+        LeafBuilder()
+        {
+            super(null);
+            buffer = new Object[MAX_KEYS];
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer, overflowing if necessary
+         */
+        public void addKey(Object nextKey)
+        {
+            if (count == MAX_KEYS)
+                overflow(nextKey);
+            else
+                buffer[count++] = nextKey;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void addKeyNoOverflow(Object nextKey)
+        {
+            buffer[count++] = nextKey;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void maybeAddKeyNoOverflow(Object nextKey)
+        {
+            buffer[count] = nextKey;
+            count += nextKey != null ? 1 : 0;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void maybeAddKey(Object nextKey)
+        {
+            if (count == MAX_KEYS)
+            {
+                if (nextKey != null)
+                    overflow(nextKey);
+            }
+            else
+            {
+                buffer[count] = nextKey;
+                count += nextKey != null ? 1 : 0;
+            }
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}, overflowing as necessary.
+         */
+        void copy(Object[] source, int offset, int length)
+        {
+            if (count + length > MAX_KEYS)
+            {
+                int copy = MAX_KEYS - count;
+                System.arraycopy(source, offset, buffer, count, copy);
+                offset += copy;
+//              implicitly:  count = MAX_KEYS;
+                overflow(source[offset++]);
+                length -= 1 + copy;
+            }
+            System.arraycopy(source, offset, buffer, count, length);
+            count += length;
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}; the caller specifying overflow is unnecessary
+         */
+        void copyNoOverflow(Object[] source, int offset, int length)
+        {
+            System.arraycopy(source, offset, buffer, count, length);
+            count += length;
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}, overflowing as necessary.
+         * Applies {@code updateF} to the contents before insertion.
+         */
+        <Insert, Existing> void copy(Object[] source, int offset, int length, UpdateFunction<Insert, Existing> updateF)
+        {
+            if (isSimple(updateF))
+            {
+                copy(source, offset, length);
+                return;
+            }
+
+            if (count + length > MAX_KEYS)
+            {
+                int copy = MAX_KEYS - count;
+                for (int i = 0 ; i < copy ; ++i)
+                    buffer[count + i] = updateF.apply((Insert) source[offset + i]);
+                offset += copy;
+//              implicitly:  leaf().count = MAX_KEYS;
+                overflow(updateF.apply((Insert) source[offset++]));
+                length -= 1 + copy;
+            }
+            for (int i = 0 ; i < length ; ++i)
+                buffer[count + i] = updateF.apply((Insert) source[offset + i]);
+            count += length;
+        }
+
+        /**
+         * {@link #buffer} is full, and we need to make room either by populating {@link #savedBuffer},
+         * propagating its current contents, if any, to {@link #parent}
+         */
+        void overflow(Object nextKey)
+        {
+            if (hasOverflow())
+                propagateOverflow();
+
+            // precondition: count == MAX_KEYS and savedNextKey == null
+
+            Object[] newBuffer = savedBuffer;
+            if (newBuffer == null)
+                newBuffer = new Object[MAX_KEYS];
+
+            savedBuffer = buffer;
+            savedNextKey = nextKey;
+            buffer = newBuffer;
+            count = 0;
+        }
+
+        /**
+         * Redistribute the contents of {@link #savedBuffer} into {@link #buffer}, finalise {@link #savedBuffer} and flush upwards.
+         * Invoked when we are building from {@link #buffer}, have insufficient values but a complete leaf in {@link #savedBuffer}
+         * @return the size of the leaf we flushed to our parent from {@link #savedBuffer}
+         */
+        Object[] redistributeOverflowAndDrain()
+        {
+            Object[] newLeaf = redistributeAndDrain(savedBuffer, MAX_KEYS, savedNextKey);
+            savedNextKey = null;
+            return newLeaf;
+        }
+
+        /**
+         * Redistribute the contents of {@link #buffer} and an immediate predecessor into a new leaf,
+         * then construct a new predecessor with the remaining contents and propagate up to our parent
+         * Invoked when we are building from {@link #buffer}, have insufficient values but either a complete
+         * leaf in {@link #savedBuffer} or can exfiltrate one from our parent to redistribute.
+         *
+         * @return the second of the two new leaves
+         */
+        Object[] redistributeAndDrain(Object[] pred, int predSize, Object predNextKey)
+        {
+            // precondition: savedLeafCount == MAX_KEYS && leaf().count < MIN_KEYS
+            // ensure we have at least MIN_KEYS in leaf().buffer
+            // first shift leaf().buffer and steal some keys from leaf().savedBuffer and leaf().savedBufferNextKey
+            int steal = MIN_KEYS - count;
+            Object[] newLeaf = new Object[MIN_KEYS];
+            System.arraycopy(pred,  predSize - (steal - 1), newLeaf, 0,     steal - 1);
+                                                            newLeaf [steal - 1] = predNextKey;
+            System.arraycopy(buffer, 0,                     newLeaf, steal, count);
+
+            // then create a leaf out of the remainder of savedBuffer
+            int newPredecessorCount = predSize - steal;
+            Object[] newPredecessor = new Object[newPredecessorCount | 1];
+            System.arraycopy(pred, 0, newPredecessor, 0, newPredecessorCount);
+            if (allocated >= 0)
+                allocated += ObjectSizes.sizeOfReferenceArray(newPredecessorCount | 1);
+            ensureParent().addChildAndNextKey(newPredecessor, newPredecessorCount, pred[newPredecessorCount]);
+            return newLeaf;
+        }
+
+        /**
+         * Invoked to fill our {@link #buffer} to >= MIN_KEYS with data ocurring before {@link #buffer};
+         * possibly instead fills {@link #savedBuffer}
+         *
+         * @param pred directly preceding node
+         * @param predNextKey key that would have separated predecessor from buffer contents
+         */
+        void prepend(Object[] pred, Object predNextKey)
+        {
+            int predSize = sizeOfLeaf(pred);
+            int newKeys = 1 + predSize;
+            if (newKeys + count <= MAX_KEYS)
+            {
+                System.arraycopy(buffer, 0, buffer, newKeys, count);
+                System.arraycopy(pred,   0, buffer, 0,       predSize);
+                                            buffer [predSize] = predNextKey;
+                count += newKeys;
+            }
+            else
+            {
+                if (savedBuffer == null)
+                    savedBuffer = new Object[MAX_KEYS];
+                System.arraycopy(pred, 0, savedBuffer, 0, predSize);
+                if (predSize == MAX_KEYS)
+                {
+                    savedNextKey = predNextKey;
+                }
+                else
+                {
+                    int removeKeys = MAX_KEYS - predSize;
+                    count -= removeKeys;
+                                                         savedBuffer [predSize] = predNextKey;
+                    System.arraycopy(buffer, 0,          savedBuffer, predSize + 1, MAX_KEYS - newKeys);
+                                                         savedNextKey      = buffer[MAX_KEYS - newKeys];
+                    System.arraycopy(buffer, removeKeys, buffer,      0,            count);
+                }
+            }
+        }
+
+        /**
+         * Invoked when we want to add a key to the leaf buffer, but it is full
+         */
+        void propagateOverflow()
+        {
+            // propagate the leaf we have saved in altLeafBuffer
+            // precondition: savedLeafCount == MAX_KEYS
+            if (allocated >= 0)
+                allocated += ObjectSizes.sizeOfReferenceArray(MAX_KEYS);
+            ensureParent().addChildAndNextKey(savedBuffer, MAX_KEYS, savedNextKey);
+            savedBuffer = null;
+            savedNextKey = null;
+        }
+
+        /**
+         * Construct a new leaf from the contents of {@link #buffer}, unless the contents have not changed
+         * from {@code unode}, in which case return {@code unode} to avoid allocating unnecessary objects.
+         */
+        Object[] drainAndPropagate(Object[] unode, int usz, BranchBuilder propagateTo)
+        {
+            Object[] leaf;
+            int sizeOfLeaf;
+            if (mustRedistribute())
+            {
+                // we have too few items, so spread the two buffers across two new nodes
+                leaf = redistributeOverflowAndDrain();
+                sizeOfLeaf = MIN_KEYS;
+            }
+            else if (!hasOverflow() && count == usz && areIdentical(buffer, 0, unode, 0, usz))
+            {
+                // we have exactly the same contents as the original node, so reuse it
+                leaf = unode;
+                sizeOfLeaf = usz;
+            }
+            else
+            {
+                // we have maybe one saved full buffer, and one buffer with sufficient contents to copy
+                if (hasOverflow())
+                    propagateOverflow();
+
+                sizeOfLeaf = count;

Review comment:
       Why can't `count` be smaller than `MIN_KEYS` here?

##########
File path: src/java/org/apache/cassandra/utils/btree/BTree.java
##########
@@ -1448,4 +1920,2355 @@ private static boolean isStopSentinel(long v)
     {
         return accumulate(btree, accumulator, arg, null, null, initialValue);
     }
+
+    /**
+     * Calculate the minimum height needed for this size of tree
+     *
+     * @param size the tree size
+     * @return the minimum height needed for this size of tree
+     */
+    private static int minHeight(int size)
+    {
+        return heightAtSize2n(size, BRANCH_SHIFT);
+    }
+
+    private static int heightAtSize2n(int size, int branchShift)
+    {
+        //    branch factor                 =  1 <<  branchShift
+        // => full size at height            = (1 << (branchShift * height)) - 1
+        // => full size at height + 1        =  1 << (branchShift * height)
+        // => shift(full size at height + 1) = branchShift * height
+        // => shift(full size at height + 1) / branchShift = height
+        int indexOfHighestBit = 63 - Long.numberOfLeadingZeros(size + 1);
+        indexOfHighestBit += (1L << indexOfHighestBit) == (size + 1) ? 0 : 1;
+        return (branchShift - 1 + indexOfHighestBit) / branchShift;
+    }
+
+    private static int[][] buildBalancedSizeMaps(int branchShift)
+    {
+        int count = (32 / branchShift) - 1;
+        int childCount = 1 << branchShift;
+        int[][] sizeMaps = new int[count][childCount];
+        for (int height = 0 ; height < count ; ++height)
+        {
+            int childSize = treeSize2n(height + 1, branchShift);
+            int size = 0;
+            int[] sizeMap = sizeMaps[height];
+            for (int i = 0 ; i < childCount ; ++i)
+            {
+                sizeMap[i] = size += childSize;
+                size += 1;
+            }
+        }
+        return sizeMaps;
+    }
+
+    // simply utility to reverse the contents of array[from..to)
+    private static void reverse(Object[] array, int from, int to)
+    {
+        int mid = (from + to) / 2;
+        for (int i = from ; i < mid ; i++)
+        {
+            int j = to - (1 + i - from);
+            Object tmp = array[i];
+            array[i] = array[j];
+            array[j] = tmp;
+        }
+    }
+
+    // simply utility to reverse the contents of array[from..to)
+    private static void reverse(int[] array, int from, int to)
+    {
+        int mid = (from + to) / 2;
+        for (int i = from ; i < mid ; i++)
+        {
+            int j = to - (1 + i - from);
+            int tmp = array[i];
+            array[i] = array[j];
+            array[j] = tmp;
+        }
+    }
+
+    /**
+     * Mutate an array of child sizes into a cumulative sizeMap, returning the total size
+     */
+    private static int sizesToSizeMap(int[] sizeMap)
+    {
+        int total = sizeMap[0];
+        for (int i = 1 ; i < sizeMap.length ; ++i)
+            sizeMap[i] = total += 1 + sizeMap[i];
+        return total;
+    }
+
+    private static int sizesToSizeMap(int[] sizes, int count)
+    {
+        int total = sizes[0];
+        for (int i = 1 ; i < count ; ++i)
+            sizes[i] = total += 1 + sizes[i];
+        return total;
+    }
+
+    /**
+     * Mutate an array of child sizes into a cumulative sizeMap, returning the total size
+     */
+    private static void sizeMapToSizes(int[] sizeMap)
+    {
+        for (int i = sizeMap.length ; i > 1 ; --i)
+            sizeMap[i] -= 1 + sizeMap[i - 1];
+    }
+
+    /**
+     * A simple utility method to handle a null upper bound that we treat as infinity
+     */
+    private static <Compare> int compareWithMaybeInfinity(Comparator<? super Compare> comparator, Compare key, Compare ub)
+    {
+        if (ub == null)
+            return -1;
+        return comparator.compare(key, ub);
+    }
+
+    /**
+     * Perform {@link #exponentialSearch} on {@code in[from..to)}, treating a {@code find} of {@code null} as infinity.
+     */
+    static <Compare> int exponentialSearchForMaybeInfinity(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare find)
+    {
+        if (find == null)
+            return -(1 + to);
+        return exponentialSearch(comparator, in, from, to, find);
+    }
+
+    /**
+     * Equivalent to {@link Arrays#binarySearch}, only more efficient algorithmically for linear merges.
+     * Binary search has worst case complexity {@code O(n.lg n)} for a linear merge, whereas exponential search
+     * has a worst case of {@code O(n)}. However compared to a simple linear merge, the best case for exponential
+     * search is {@code O(lg(n))} instead of {@code O(n)}.
+     */
+    private static <Compare> int exponentialSearch(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare find)
+    {
+        int step = 0;
+        while (from + step < to)
+        {
+            int i = from + step;
+            int c = comparator.compare(find, (Compare) in[i]);
+            if (c < 0)
+            {
+                to = i;
+                break;
+            }
+            if (c == 0)
+                return i;
+            from = i + 1;
+            step = step * 2 + 1; // jump in perfect binary search increments
+        }
+        return Arrays.binarySearch((Compare[]) in, from, to, find, comparator);
+    }
+
+    /**
+     * Perform {@link #exponentialSearch} on {@code in[from..to)}; if the value falls outside of the range of these
+     * elements, test against {@code ub} as though it occurred at position {@code to}
+     *
+     * @return same as {@link Arrays#binarySearch} if {@code find} occurs in the range {@code [in[from]..in[to])};
+     * otherwise the insertion position {@code -(1+to)} if {@code find} is less than {@code ub}, and {@code -(2+t)}
+     * if it is greater than or equal to.
+     *
+     * {@code ub} may be {@code null}, representing infinity.
+     */
+    static <Compare> int exponentialSearchWithUpperBound(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare ub, Compare find)
+    {
+        int step = 0;
+        while (true)
+        {
+            int i = from + step;
+            if (i >= to)
+            {
+                int c = compareWithMaybeInfinity(comparator, find, ub);
+                if (c >= 0)
+                    return -(2 + to);
+                break;
+            }
+            int c = comparator.compare(find, (Compare) in[i]);
+            if (c < 0)
+            {
+                to = i;
+                break;
+            }
+            if (c == 0)
+                return i;
+            from = i + 1;
+            step = step * 2 + 1; // jump in perfect binary search increments
+        }
+        return Arrays.binarySearch((Compare[]) in, from, to, find, comparator);
+    }
+    
+    /**
+     * Compute the size-in-bytes of full trees of cardinality {@code branchFactor^height - 1}
+     */
+    private static long[] sizeOnHeapOfPerfectTrees(int branchShift)
+    {
+        long[] result = new long[heightAtSize2n(Integer.MAX_VALUE, branchShift)];
+        int branchFactor = 1 << branchShift;
+        result[0] = branchFactor - 1;
+        for (int i = 1 ; i < result.length ; ++i)
+            result[i] = sizeOnHeapOfPerfectTree(i + 1, branchFactor);
+        return result;
+    }
+
+    /**
+     * Compute the size-in-bytes of a full tree of cardinality {@code branchFactor^height - 1}
+     * TODO: test
+     */
+    private static long sizeOnHeapOfPerfectTree(int height, int branchShift)
+    {
+        int branchFactor = 1 << branchShift;
+        long branchSize = ObjectSizes.sizeOfReferenceArray(branchFactor * 2);
+        int branchCount = height == 2 ? 1 : 2 + treeSize2n(height - 2, branchShift);
+        long leafSize = ObjectSizes.sizeOfReferenceArray((branchFactor - 1)|1);
+        int leafCount = 1 + treeSize2n(height - 1, branchShift);
+        return (branchSize * branchCount) + (leafSize * leafCount);
+    }
+
+    /**
+     * @return the actual height of {@code tree}
+     */
+    public static int height(Object[] tree)
+    {
+        if (isLeaf(tree))
+            return 1;
+
+        int height = 1;
+        while (!isLeaf(tree))
+        {
+            height++;
+            tree = (Object[]) tree[shallowSizeOfBranch(tree)];
+        }
+        return height;
+    }
+
+    /**
+     * @return the minimum representable size at {@code height}, for an internal (non-root) node.
+     */
+    private static int sparseSize(int height)
+    {
+        return treeSize2n(height, BRANCH_SHIFT - 1);
+    }
+
+    /**
+     * @return the maximum representable size at {@code height}.
+     */
+    private static int denseSize(int height)
+    {
+        return treeSize2n(height, BRANCH_SHIFT);
+    }
+
+    /**
+     * @return the maximum representable size at {@code height}.
+     */
+    private static int checkedDenseSize(int height)
+    {
+        assert height * BRANCH_SHIFT < 32;
+        return denseSize(height);
+    }
+
+    /**
+     * Computes the number of nodes in a full tree of height {@code height}
+     * and with {@code 2^branchShift} branch factor.
+     * i.e. computes {@code (2^branchShift)^height - 1}
+     */
+    private static int treeSize2n(int height, int branchShift)
+    {
+        return (1 << (branchShift * height)) - 1;
+    }
+
+    // TODO: test
+    private static int maxRootHeight(int size)
+    {
+        if (size <= BRANCH_FACTOR)
+            return 1;
+        return 1 + heightAtSize2n((size - 1) / 2, BRANCH_SHIFT - 1);
+    }
+    
+    private static int sizeOfBranch(Object[] branch)
+    {
+        int length = branch.length;
+        // length - 1 == getChildEnd == getPositionOfSizeMap
+        // (length / 2) - 1 == getChildCount - 1 == position of full tree size
+        // hard code this, as will be used often;
+        return ((int[]) branch[length - 1])[(length / 2) - 1];
+    }
+
+    /**
+     * Checks if the UpdateFunction is an instance of {@code UpdateFunction.Simple}.
+     */
+    private static boolean isSimple(UpdateFunction<?, ?> updateF)
+    {
+        return updateF instanceof UpdateFunction.Simple;
+    }
+
+    /**
+     * @return the size map for the branch node
+     */
+    static int[] sizeMap(Object[] branch)
+    {
+        return (int[]) branch[branch.length - 1];
+    }
+
+    public static long sizeOnHeapOf(Object[] tree)
+    {
+        long size = ObjectSizes.sizeOfArray(tree);
+        if (isLeaf(tree))
+            return size;
+        for (int i = childOffset(tree); i < childEndOffset(tree) ; i++)
+            size += sizeOnHeapOf((Object[]) tree[i]);
+        size += ObjectSizes.sizeOfArray(sizeMap(tree)); // may overcount, since we share size maps
+        return size;
+    }
+
+    // Arbitrary boundaries
+    private static Object POSITIVE_INFINITY = new Object();
+    private static Object NEGATIVE_INFINITY = new Object();
+
+    /** simple static wrapper to calls to cmp.compare() which checks if either a or b are Special (i.e. represent an infinity) */
+    private static <V> int compareWellFormed(Comparator<V> cmp, Object a, Object b)
+    {
+        if (a == b)
+            return 0;
+        if (a == NEGATIVE_INFINITY | b == POSITIVE_INFINITY)
+            return -1;
+        if (b == NEGATIVE_INFINITY | a == POSITIVE_INFINITY)
+            return 1;
+        return cmp.compare((V) a, (V) b);
+    }
+
+    public static boolean isWellFormed(Object[] btree, Comparator<?> cmp)
+    {
+        return isWellFormed(cmp, btree, true, NEGATIVE_INFINITY, POSITIVE_INFINITY);
+    }
+
+    private static boolean isWellFormed(Comparator<?> cmp, Object[] node, boolean isRoot, Object min, Object max)
+    {
+        if (isEmpty(node))
+            return true;
+
+        if (cmp != null && !isNodeWellFormed(cmp, node, min, max))
+            return false;
+
+        int keyCount = shallowSize(node);
+        if (keyCount < 1)
+            return false;
+        if (!isRoot && keyCount < BRANCH_FACTOR/2 - 1)
+            return false;
+        if (keyCount >= BRANCH_FACTOR)
+            return false;
+
+        if (isLeaf(node))
+            return true;
+
+        int[] sizeMap = sizeMap(node);
+        int type = 0;
+        int size = 0;
+        // compare each child node with the branch element at the head of this node it corresponds with
+        for (int i = childOffset(node); i < childEndOffset(node) ; i++)
+        {
+            Object[] child = (Object[]) node[i];
+            Object localmax = i < node.length - 2 ? node[i - childOffset(node)] : max;
+            if (!isWellFormed(cmp, child, false, min, localmax))
+                return false;
+
+            type |= isLeaf(child) ? 1 : 2;
+            min = localmax;
+            size += size(child);
+            if (sizeMap[i - childOffset(node)] != size)
+                return false;
+            size += 1;
+        }
+
+        if (type >= 3)
+            return false;
+        return true; // either all leaves or all branches but not a mix
+    }
+
+    private static boolean isNodeWellFormed(Comparator<?> cmp, Object[] node, Object min, Object max)
+    {
+        Object previous = min;
+        int end = shallowSize(node);
+        for (int i = 0; i < end; i++)
+        {
+            Object current = node[i];
+            if (compareWellFormed(cmp, previous, current) >= 0)
+                return false;
+
+            previous = current;
+        }
+        return compareWellFormed(cmp, previous, max) < 0;
+    }
+
+    /**
+     * Build a tree of unknown size, in order.
+     *
+     * Can be used with {@link #reverseInSitu} to build a tree in reverse.
+     */
+    public static <V> FastBuilder<V> fastBuilder()
+    {
+        TinyThreadLocalPool.TinyPool<FastBuilder<?>> pool = FastBuilder.POOL.get();
+        FastBuilder<V> builder = (FastBuilder<V>) pool.poll();
+        if (builder == null)
+            builder = new FastBuilder<>();
+        builder.pool = pool;
+        return builder;
+    }
+
+    /**
+     * Base class for AbstractFastBuilder.BranchBuilder, LeafBuilder and AbstractFastBuilder,
+     * containing shared behaviour and declaring some useful abstract methods.
+     */
+    private static abstract class LeafOrBranchBuilder
+    {
+        final int height;
+        final LeafOrBranchBuilder child;
+        BranchBuilder parent;
+
+        /**
+         * The current buffer contents (if any) of the leaf or branch - always sized to contain a complete
+         * node of the form being constructed.  Always non-null, except briefly during overflow.
+         */
+        Object[] buffer;
+        /**
+         * The number of keys in our buffer, whether or not we are building a leaf or branch; if we are building
+         * a branch, we will ordinarily have the same number of children as well, except temporarily when finishing
+         * the construction of the node.
+         */
+        int count;
+
+        /**
+         * either
+         *  1) an empty leftover buffer from a past usage, which can be used when we exhaust {@code buffer}; or
+         *  2) a full {@code buffer} that has been parked until we next overflow, so we can steal some back
+         *     if we finish before reaching MIN_KEYS in {@code buffer}
+         */
+        Object[] savedBuffer;
+        /**
+         * The key we overflowed on when populating savedBuffer.  If null, {@link #savedBuffer} is logically empty.
+         */
+        Object savedNextKey;
+
+        LeafOrBranchBuilder(LeafOrBranchBuilder child)
+        {
+            this.height = child == null ? 1 : 1 + child.height;
+            this.child = child;
+        }
+
+        /**
+         * Do we have enough keys in the builder to construct at least one balanced node?
+         * We could have enough to build two.
+         */
+        final boolean isSufficient()
+        {
+            return hasOverflow() || count >= MIN_KEYS;
+        }
+
+        /**
+         * Do we have an already constructed node saved, that we can propagate or redistribute?
+         * This implies we are building two nodes, since {@link #savedNextKey} would overflow {@link #savedBuffer}
+         */
+        final boolean hasOverflow()
+        {
+            return savedNextKey != null;
+        }
+
+        /**
+         * Do we have an already constructed node saved AND insufficient keys in our buffer, so
+         * that we need to share the contents of {@link #savedBuffer} with {@link #buffer} to construct
+         * our results?
+         */
+        final boolean mustRedistribute()
+        {
+            return hasOverflow() && count < MIN_KEYS;
+        }
+
+        /**
+         * Are we empty, i.e. we have no contents in either {@link #buffer} or {@link #savedBuffer}
+         */
+        final boolean isEmpty()
+        {
+            return count == 0 && savedNextKey == null;
+        }
+
+        /**
+         * Drain the contents of this builder and build up to two nodes, as necessary.
+         * If {@code unode != null} and we are building a single node that is identical to it, use {@code unode} instead.
+         * If {@code propagateTo != null} propagate any nodes we build to it.
+         * @return the last node we construct
+         */
+        abstract Object[] drainAndPropagate(Object[] unode, int usz, BranchBuilder propagateTo);
+
+        /**
+         * Drain the contents of this builder and build at most one node.
+         * Requires {@code !hasOverflow()}
+         * @return the node we construct
+         */
+        abstract Object[] drain();
+
+        /**
+         * Takes a node that would logically occur directly preceding the current buffer contents,
+         * and the key that would separate them in a parent node, and prepends their contents
+         * to the current buffer's contents.  This can be used to redistribute already-propagated
+         * contents to a parent in cases where this is convenient (i.e. when transforming)
+         *
+         * @param predecessor directly preceding node
+         * @param predecessorNextKey key that would have separated predecessor from buffer contents
+         */
+        abstract void prepend(Object[] predecessor, Object predecessorNextKey);
+
+        /**
+         * Indicates if this builder produces dense nodes, i.e. those that are populated with MAX_KEYS
+         * at every level.  Only the last two children of any branch may be non-dense, and in some cases only
+         * the last two nodes in any tier of the tree.
+         *
+         * This flag switches whether or not we maintain a buffer of sizes, or use the globally shared contents of
+         * DENSE_SIZE_MAPS.
+         */
+        abstract boolean producesOnlyDense();
+
+        /**
+         * Ensure there is a {@code branch.parent}, and return it
+         */
+        final BranchBuilder ensureParent()
+        {
+            if (parent == null)
+                parent = new BranchBuilder(this);
+            parent.touched = true;
+            return parent;
+        }
+
+        /**
+         * Mark a branch builder as utilised, so that we must clear it when resetting any {@link AbstractFastBuilder}
+         * @return {@code branch}
+         */
+        static BranchBuilder touch(BranchBuilder branch)
+        {
+            branch.touched = true;
+            return branch;
+        }
+
+        /**
+         * @return the parent builder iff we will overflow or the parent has already been touched
+         */
+        final BranchBuilder parentIfInUse()
+        {
+            // if we have enough data to propagate, ensure there is a parent and return it
+            if (hasOverflow())
+                return ensureParent();
+
+            // otherwise return it only if it's already in use
+            return parent == null || !parent.touched ? null : parent;
+        }
+
+        /**
+         * A utility method for comparing a range of two arrays
+         */
+        static boolean areIdentical(Object[] a, int aOffset, Object[] b, int bOffset, int count)
+        {
+            for (int i = 0 ; i < count ; ++i)
+            {
+                if (a[i + aOffset] != b[i + bOffset])
+                    return false;
+            }
+            return true;
+        }
+
+        /**
+         * A utility method for comparing a range of two arrays
+         */
+        static boolean areIdentical(int[] a, int aOffset, int[] b, int bOffset, int count)
+        {
+            for (int i = 0 ; i < count ; ++i)
+            {
+                if (a[i + aOffset] != b[i + bOffset])
+                    return false;
+            }
+            return true;
+        }
+    }
+
+    /**
+     * LeafBuilder for methods pertaining specifically to building a leaf in an {@link AbstractFastBuilder}.
+     * Note that {@link AbstractFastBuilder} extends this class directly, however it is convenient to maintain
+     * distinct classes in the hierarchy for clarity of behaviour and intent.
+     */
+    private static abstract class LeafBuilder extends LeafOrBranchBuilder
+    {
+        long allocated;
+
+        LeafBuilder()
+        {
+            super(null);
+            buffer = new Object[MAX_KEYS];
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer, overflowing if necessary
+         */
+        public void addKey(Object nextKey)
+        {
+            if (count == MAX_KEYS)
+                overflow(nextKey);
+            else
+                buffer[count++] = nextKey;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void addKeyNoOverflow(Object nextKey)
+        {
+            buffer[count++] = nextKey;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void maybeAddKeyNoOverflow(Object nextKey)
+        {
+            buffer[count] = nextKey;
+            count += nextKey != null ? 1 : 0;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void maybeAddKey(Object nextKey)
+        {
+            if (count == MAX_KEYS)
+            {
+                if (nextKey != null)
+                    overflow(nextKey);
+            }
+            else
+            {
+                buffer[count] = nextKey;
+                count += nextKey != null ? 1 : 0;
+            }
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}, overflowing as necessary.
+         */
+        void copy(Object[] source, int offset, int length)
+        {
+            if (count + length > MAX_KEYS)
+            {
+                int copy = MAX_KEYS - count;
+                System.arraycopy(source, offset, buffer, count, copy);
+                offset += copy;
+//              implicitly:  count = MAX_KEYS;
+                overflow(source[offset++]);
+                length -= 1 + copy;
+            }
+            System.arraycopy(source, offset, buffer, count, length);
+            count += length;
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}; the caller specifying overflow is unnecessary
+         */
+        void copyNoOverflow(Object[] source, int offset, int length)
+        {
+            System.arraycopy(source, offset, buffer, count, length);
+            count += length;
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}, overflowing as necessary.
+         * Applies {@code updateF} to the contents before insertion.
+         */
+        <Insert, Existing> void copy(Object[] source, int offset, int length, UpdateFunction<Insert, Existing> updateF)
+        {
+            if (isSimple(updateF))
+            {
+                copy(source, offset, length);
+                return;
+            }
+
+            if (count + length > MAX_KEYS)
+            {
+                int copy = MAX_KEYS - count;
+                for (int i = 0 ; i < copy ; ++i)
+                    buffer[count + i] = updateF.apply((Insert) source[offset + i]);
+                offset += copy;
+//              implicitly:  leaf().count = MAX_KEYS;
+                overflow(updateF.apply((Insert) source[offset++]));
+                length -= 1 + copy;
+            }
+            for (int i = 0 ; i < length ; ++i)
+                buffer[count + i] = updateF.apply((Insert) source[offset + i]);
+            count += length;
+        }
+
+        /**
+         * {@link #buffer} is full, and we need to make room either by populating {@link #savedBuffer},
+         * propagating its current contents, if any, to {@link #parent}
+         */
+        void overflow(Object nextKey)
+        {
+            if (hasOverflow())
+                propagateOverflow();
+
+            // precondition: count == MAX_KEYS and savedNextKey == null
+
+            Object[] newBuffer = savedBuffer;
+            if (newBuffer == null)
+                newBuffer = new Object[MAX_KEYS];
+
+            savedBuffer = buffer;
+            savedNextKey = nextKey;
+            buffer = newBuffer;
+            count = 0;
+        }
+
+        /**
+         * Redistribute the contents of {@link #savedBuffer} into {@link #buffer}, finalise {@link #savedBuffer} and flush upwards.
+         * Invoked when we are building from {@link #buffer}, have insufficient values but a complete leaf in {@link #savedBuffer}
+         * @return the size of the leaf we flushed to our parent from {@link #savedBuffer}
+         */
+        Object[] redistributeOverflowAndDrain()
+        {
+            Object[] newLeaf = redistributeAndDrain(savedBuffer, MAX_KEYS, savedNextKey);
+            savedNextKey = null;
+            return newLeaf;
+        }
+
+        /**
+         * Redistribute the contents of {@link #buffer} and an immediate predecessor into a new leaf,
+         * then construct a new predecessor with the remaining contents and propagate up to our parent
+         * Invoked when we are building from {@link #buffer}, have insufficient values but either a complete
+         * leaf in {@link #savedBuffer} or can exfiltrate one from our parent to redistribute.
+         *
+         * @return the second of the two new leaves
+         */
+        Object[] redistributeAndDrain(Object[] pred, int predSize, Object predNextKey)
+        {
+            // precondition: savedLeafCount == MAX_KEYS && leaf().count < MIN_KEYS
+            // ensure we have at least MIN_KEYS in leaf().buffer
+            // first shift leaf().buffer and steal some keys from leaf().savedBuffer and leaf().savedBufferNextKey
+            int steal = MIN_KEYS - count;
+            Object[] newLeaf = new Object[MIN_KEYS];
+            System.arraycopy(pred,  predSize - (steal - 1), newLeaf, 0,     steal - 1);
+                                                            newLeaf [steal - 1] = predNextKey;
+            System.arraycopy(buffer, 0,                     newLeaf, steal, count);
+
+            // then create a leaf out of the remainder of savedBuffer
+            int newPredecessorCount = predSize - steal;
+            Object[] newPredecessor = new Object[newPredecessorCount | 1];
+            System.arraycopy(pred, 0, newPredecessor, 0, newPredecessorCount);
+            if (allocated >= 0)
+                allocated += ObjectSizes.sizeOfReferenceArray(newPredecessorCount | 1);
+            ensureParent().addChildAndNextKey(newPredecessor, newPredecessorCount, pred[newPredecessorCount]);
+            return newLeaf;
+        }
+
+        /**
+         * Invoked to fill our {@link #buffer} to >= MIN_KEYS with data ocurring before {@link #buffer};
+         * possibly instead fills {@link #savedBuffer}
+         *
+         * @param pred directly preceding node
+         * @param predNextKey key that would have separated predecessor from buffer contents
+         */
+        void prepend(Object[] pred, Object predNextKey)
+        {
+            int predSize = sizeOfLeaf(pred);
+            int newKeys = 1 + predSize;
+            if (newKeys + count <= MAX_KEYS)
+            {
+                System.arraycopy(buffer, 0, buffer, newKeys, count);
+                System.arraycopy(pred,   0, buffer, 0,       predSize);
+                                            buffer [predSize] = predNextKey;
+                count += newKeys;
+            }
+            else
+            {
+                if (savedBuffer == null)
+                    savedBuffer = new Object[MAX_KEYS];
+                System.arraycopy(pred, 0, savedBuffer, 0, predSize);
+                if (predSize == MAX_KEYS)
+                {
+                    savedNextKey = predNextKey;
+                }
+                else
+                {
+                    int removeKeys = MAX_KEYS - predSize;
+                    count -= removeKeys;
+                                                         savedBuffer [predSize] = predNextKey;
+                    System.arraycopy(buffer, 0,          savedBuffer, predSize + 1, MAX_KEYS - newKeys);
+                                                         savedNextKey      = buffer[MAX_KEYS - newKeys];
+                    System.arraycopy(buffer, removeKeys, buffer,      0,            count);
+                }
+            }
+        }
+
+        /**
+         * Invoked when we want to add a key to the leaf buffer, but it is full
+         */
+        void propagateOverflow()
+        {
+            // propagate the leaf we have saved in altLeafBuffer
+            // precondition: savedLeafCount == MAX_KEYS
+            if (allocated >= 0)
+                allocated += ObjectSizes.sizeOfReferenceArray(MAX_KEYS);
+            ensureParent().addChildAndNextKey(savedBuffer, MAX_KEYS, savedNextKey);
+            savedBuffer = null;
+            savedNextKey = null;
+        }
+
+        /**
+         * Construct a new leaf from the contents of {@link #buffer}, unless the contents have not changed
+         * from {@code unode}, in which case return {@code unode} to avoid allocating unnecessary objects.
+         */
+        Object[] drainAndPropagate(Object[] unode, int usz, BranchBuilder propagateTo)
+        {
+            Object[] leaf;
+            int sizeOfLeaf;
+            if (mustRedistribute())
+            {
+                // we have too few items, so spread the two buffers across two new nodes
+                leaf = redistributeOverflowAndDrain();
+                sizeOfLeaf = MIN_KEYS;
+            }
+            else if (!hasOverflow() && count == usz && areIdentical(buffer, 0, unode, 0, usz))
+            {
+                // we have exactly the same contents as the original node, so reuse it
+                leaf = unode;
+                sizeOfLeaf = usz;
+            }
+            else
+            {
+                // we have maybe one saved full buffer, and one buffer with sufficient contents to copy
+                if (hasOverflow())
+                    propagateOverflow();
+
+                sizeOfLeaf = count;
+                leaf = drain();
+                if (allocated >= 0 && sizeOfLeaf > 0)
+                    allocated += ObjectSizes.sizeOfReferenceArray(sizeOfLeaf | 1) - (unode == null ? 0 : ObjectSizes.sizeOfArray(unode));
+            }
+
+            count = 0;
+            if (propagateTo != null)
+                propagateTo.addChild(leaf, sizeOfLeaf);
+            return leaf;
+        }
+
+        /**
+         * Construct a new leaf from the contents of {@code leaf().buffer}, unless the contents have not changed
+         * from {@code unode}, in which case return {@code unode} to avoid allocating unnecessary objects.

Review comment:
       Doc doesn't match code.

##########
File path: src/java/org/apache/cassandra/utils/btree/BTree.java
##########
@@ -1448,4 +1920,2355 @@ private static boolean isStopSentinel(long v)
     {
         return accumulate(btree, accumulator, arg, null, null, initialValue);
     }
+
+    /**
+     * Calculate the minimum height needed for this size of tree
+     *
+     * @param size the tree size
+     * @return the minimum height needed for this size of tree
+     */
+    private static int minHeight(int size)
+    {
+        return heightAtSize2n(size, BRANCH_SHIFT);
+    }
+
+    private static int heightAtSize2n(int size, int branchShift)
+    {
+        //    branch factor                 =  1 <<  branchShift
+        // => full size at height            = (1 << (branchShift * height)) - 1
+        // => full size at height + 1        =  1 << (branchShift * height)
+        // => shift(full size at height + 1) = branchShift * height
+        // => shift(full size at height + 1) / branchShift = height
+        int indexOfHighestBit = 63 - Long.numberOfLeadingZeros(size + 1);
+        indexOfHighestBit += (1L << indexOfHighestBit) == (size + 1) ? 0 : 1;
+        return (branchShift - 1 + indexOfHighestBit) / branchShift;
+    }
+
+    private static int[][] buildBalancedSizeMaps(int branchShift)
+    {
+        int count = (32 / branchShift) - 1;
+        int childCount = 1 << branchShift;
+        int[][] sizeMaps = new int[count][childCount];
+        for (int height = 0 ; height < count ; ++height)
+        {
+            int childSize = treeSize2n(height + 1, branchShift);
+            int size = 0;
+            int[] sizeMap = sizeMaps[height];
+            for (int i = 0 ; i < childCount ; ++i)
+            {
+                sizeMap[i] = size += childSize;
+                size += 1;
+            }
+        }
+        return sizeMaps;
+    }
+
+    // simply utility to reverse the contents of array[from..to)
+    private static void reverse(Object[] array, int from, int to)
+    {
+        int mid = (from + to) / 2;
+        for (int i = from ; i < mid ; i++)
+        {
+            int j = to - (1 + i - from);
+            Object tmp = array[i];
+            array[i] = array[j];
+            array[j] = tmp;
+        }
+    }
+
+    // simply utility to reverse the contents of array[from..to)
+    private static void reverse(int[] array, int from, int to)
+    {
+        int mid = (from + to) / 2;
+        for (int i = from ; i < mid ; i++)
+        {
+            int j = to - (1 + i - from);
+            int tmp = array[i];
+            array[i] = array[j];
+            array[j] = tmp;
+        }
+    }
+
+    /**
+     * Mutate an array of child sizes into a cumulative sizeMap, returning the total size
+     */
+    private static int sizesToSizeMap(int[] sizeMap)
+    {
+        int total = sizeMap[0];
+        for (int i = 1 ; i < sizeMap.length ; ++i)
+            sizeMap[i] = total += 1 + sizeMap[i];
+        return total;
+    }
+
+    private static int sizesToSizeMap(int[] sizes, int count)
+    {
+        int total = sizes[0];
+        for (int i = 1 ; i < count ; ++i)
+            sizes[i] = total += 1 + sizes[i];
+        return total;
+    }
+
+    /**
+     * Mutate an array of child sizes into a cumulative sizeMap, returning the total size
+     */
+    private static void sizeMapToSizes(int[] sizeMap)
+    {
+        for (int i = sizeMap.length ; i > 1 ; --i)
+            sizeMap[i] -= 1 + sizeMap[i - 1];
+    }
+
+    /**
+     * A simple utility method to handle a null upper bound that we treat as infinity
+     */
+    private static <Compare> int compareWithMaybeInfinity(Comparator<? super Compare> comparator, Compare key, Compare ub)
+    {
+        if (ub == null)
+            return -1;
+        return comparator.compare(key, ub);
+    }
+
+    /**
+     * Perform {@link #exponentialSearch} on {@code in[from..to)}, treating a {@code find} of {@code null} as infinity.
+     */
+    static <Compare> int exponentialSearchForMaybeInfinity(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare find)
+    {
+        if (find == null)
+            return -(1 + to);
+        return exponentialSearch(comparator, in, from, to, find);
+    }
+
+    /**
+     * Equivalent to {@link Arrays#binarySearch}, only more efficient algorithmically for linear merges.
+     * Binary search has worst case complexity {@code O(n.lg n)} for a linear merge, whereas exponential search
+     * has a worst case of {@code O(n)}. However compared to a simple linear merge, the best case for exponential
+     * search is {@code O(lg(n))} instead of {@code O(n)}.
+     */
+    private static <Compare> int exponentialSearch(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare find)
+    {
+        int step = 0;
+        while (from + step < to)
+        {
+            int i = from + step;
+            int c = comparator.compare(find, (Compare) in[i]);
+            if (c < 0)
+            {
+                to = i;
+                break;
+            }
+            if (c == 0)
+                return i;
+            from = i + 1;
+            step = step * 2 + 1; // jump in perfect binary search increments
+        }
+        return Arrays.binarySearch((Compare[]) in, from, to, find, comparator);
+    }
+
+    /**
+     * Perform {@link #exponentialSearch} on {@code in[from..to)}; if the value falls outside of the range of these
+     * elements, test against {@code ub} as though it occurred at position {@code to}
+     *
+     * @return same as {@link Arrays#binarySearch} if {@code find} occurs in the range {@code [in[from]..in[to])};
+     * otherwise the insertion position {@code -(1+to)} if {@code find} is less than {@code ub}, and {@code -(2+t)}
+     * if it is greater than or equal to.
+     *
+     * {@code ub} may be {@code null}, representing infinity.
+     */
+    static <Compare> int exponentialSearchWithUpperBound(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare ub, Compare find)
+    {
+        int step = 0;
+        while (true)
+        {
+            int i = from + step;
+            if (i >= to)
+            {
+                int c = compareWithMaybeInfinity(comparator, find, ub);
+                if (c >= 0)
+                    return -(2 + to);
+                break;
+            }
+            int c = comparator.compare(find, (Compare) in[i]);
+            if (c < 0)
+            {
+                to = i;
+                break;
+            }
+            if (c == 0)
+                return i;
+            from = i + 1;
+            step = step * 2 + 1; // jump in perfect binary search increments
+        }
+        return Arrays.binarySearch((Compare[]) in, from, to, find, comparator);
+    }
+    
+    /**
+     * Compute the size-in-bytes of full trees of cardinality {@code branchFactor^height - 1}
+     */
+    private static long[] sizeOnHeapOfPerfectTrees(int branchShift)
+    {
+        long[] result = new long[heightAtSize2n(Integer.MAX_VALUE, branchShift)];
+        int branchFactor = 1 << branchShift;
+        result[0] = branchFactor - 1;
+        for (int i = 1 ; i < result.length ; ++i)
+            result[i] = sizeOnHeapOfPerfectTree(i + 1, branchFactor);
+        return result;
+    }
+
+    /**
+     * Compute the size-in-bytes of a full tree of cardinality {@code branchFactor^height - 1}
+     * TODO: test
+     */
+    private static long sizeOnHeapOfPerfectTree(int height, int branchShift)
+    {
+        int branchFactor = 1 << branchShift;
+        long branchSize = ObjectSizes.sizeOfReferenceArray(branchFactor * 2);
+        int branchCount = height == 2 ? 1 : 2 + treeSize2n(height - 2, branchShift);
+        long leafSize = ObjectSizes.sizeOfReferenceArray((branchFactor - 1)|1);
+        int leafCount = 1 + treeSize2n(height - 1, branchShift);
+        return (branchSize * branchCount) + (leafSize * leafCount);
+    }
+
+    /**
+     * @return the actual height of {@code tree}
+     */
+    public static int height(Object[] tree)
+    {
+        if (isLeaf(tree))
+            return 1;
+
+        int height = 1;
+        while (!isLeaf(tree))
+        {
+            height++;
+            tree = (Object[]) tree[shallowSizeOfBranch(tree)];
+        }
+        return height;
+    }
+
+    /**
+     * @return the minimum representable size at {@code height}, for an internal (non-root) node.
+     */
+    private static int sparseSize(int height)
+    {
+        return treeSize2n(height, BRANCH_SHIFT - 1);
+    }
+
+    /**
+     * @return the maximum representable size at {@code height}.
+     */
+    private static int denseSize(int height)
+    {
+        return treeSize2n(height, BRANCH_SHIFT);
+    }
+
+    /**
+     * @return the maximum representable size at {@code height}.
+     */
+    private static int checkedDenseSize(int height)
+    {
+        assert height * BRANCH_SHIFT < 32;
+        return denseSize(height);
+    }
+
+    /**
+     * Computes the number of nodes in a full tree of height {@code height}
+     * and with {@code 2^branchShift} branch factor.
+     * i.e. computes {@code (2^branchShift)^height - 1}
+     */
+    private static int treeSize2n(int height, int branchShift)
+    {
+        return (1 << (branchShift * height)) - 1;
+    }
+
+    // TODO: test
+    private static int maxRootHeight(int size)
+    {
+        if (size <= BRANCH_FACTOR)
+            return 1;
+        return 1 + heightAtSize2n((size - 1) / 2, BRANCH_SHIFT - 1);
+    }
+    
+    private static int sizeOfBranch(Object[] branch)
+    {
+        int length = branch.length;
+        // length - 1 == getChildEnd == getPositionOfSizeMap
+        // (length / 2) - 1 == getChildCount - 1 == position of full tree size
+        // hard code this, as will be used often;
+        return ((int[]) branch[length - 1])[(length / 2) - 1];
+    }
+
+    /**
+     * Checks if the UpdateFunction is an instance of {@code UpdateFunction.Simple}.
+     */
+    private static boolean isSimple(UpdateFunction<?, ?> updateF)
+    {
+        return updateF instanceof UpdateFunction.Simple;
+    }
+
+    /**
+     * @return the size map for the branch node
+     */
+    static int[] sizeMap(Object[] branch)
+    {
+        return (int[]) branch[branch.length - 1];
+    }
+
+    public static long sizeOnHeapOf(Object[] tree)
+    {
+        long size = ObjectSizes.sizeOfArray(tree);
+        if (isLeaf(tree))
+            return size;
+        for (int i = childOffset(tree); i < childEndOffset(tree) ; i++)
+            size += sizeOnHeapOf((Object[]) tree[i]);
+        size += ObjectSizes.sizeOfArray(sizeMap(tree)); // may overcount, since we share size maps
+        return size;
+    }
+
+    // Arbitrary boundaries
+    private static Object POSITIVE_INFINITY = new Object();
+    private static Object NEGATIVE_INFINITY = new Object();
+
+    /** simple static wrapper to calls to cmp.compare() which checks if either a or b are Special (i.e. represent an infinity) */
+    private static <V> int compareWellFormed(Comparator<V> cmp, Object a, Object b)
+    {
+        if (a == b)
+            return 0;
+        if (a == NEGATIVE_INFINITY | b == POSITIVE_INFINITY)
+            return -1;
+        if (b == NEGATIVE_INFINITY | a == POSITIVE_INFINITY)
+            return 1;
+        return cmp.compare((V) a, (V) b);
+    }
+
+    public static boolean isWellFormed(Object[] btree, Comparator<?> cmp)
+    {
+        return isWellFormed(cmp, btree, true, NEGATIVE_INFINITY, POSITIVE_INFINITY);
+    }
+
+    private static boolean isWellFormed(Comparator<?> cmp, Object[] node, boolean isRoot, Object min, Object max)
+    {
+        if (isEmpty(node))
+            return true;
+
+        if (cmp != null && !isNodeWellFormed(cmp, node, min, max))
+            return false;
+
+        int keyCount = shallowSize(node);
+        if (keyCount < 1)
+            return false;
+        if (!isRoot && keyCount < BRANCH_FACTOR/2 - 1)
+            return false;
+        if (keyCount >= BRANCH_FACTOR)
+            return false;
+
+        if (isLeaf(node))
+            return true;
+
+        int[] sizeMap = sizeMap(node);
+        int type = 0;
+        int size = 0;
+        // compare each child node with the branch element at the head of this node it corresponds with
+        for (int i = childOffset(node); i < childEndOffset(node) ; i++)
+        {
+            Object[] child = (Object[]) node[i];
+            Object localmax = i < node.length - 2 ? node[i - childOffset(node)] : max;
+            if (!isWellFormed(cmp, child, false, min, localmax))
+                return false;
+
+            type |= isLeaf(child) ? 1 : 2;
+            min = localmax;
+            size += size(child);
+            if (sizeMap[i - childOffset(node)] != size)
+                return false;
+            size += 1;
+        }
+
+        if (type >= 3)
+            return false;
+        return true; // either all leaves or all branches but not a mix
+    }
+
+    private static boolean isNodeWellFormed(Comparator<?> cmp, Object[] node, Object min, Object max)
+    {
+        Object previous = min;
+        int end = shallowSize(node);
+        for (int i = 0; i < end; i++)
+        {
+            Object current = node[i];
+            if (compareWellFormed(cmp, previous, current) >= 0)
+                return false;
+
+            previous = current;
+        }
+        return compareWellFormed(cmp, previous, max) < 0;
+    }
+
+    /**
+     * Build a tree of unknown size, in order.
+     *
+     * Can be used with {@link #reverseInSitu} to build a tree in reverse.
+     */
+    public static <V> FastBuilder<V> fastBuilder()
+    {
+        TinyThreadLocalPool.TinyPool<FastBuilder<?>> pool = FastBuilder.POOL.get();
+        FastBuilder<V> builder = (FastBuilder<V>) pool.poll();
+        if (builder == null)
+            builder = new FastBuilder<>();
+        builder.pool = pool;
+        return builder;
+    }
+
+    /**
+     * Base class for AbstractFastBuilder.BranchBuilder, LeafBuilder and AbstractFastBuilder,
+     * containing shared behaviour and declaring some useful abstract methods.
+     */
+    private static abstract class LeafOrBranchBuilder
+    {
+        final int height;
+        final LeafOrBranchBuilder child;
+        BranchBuilder parent;
+
+        /**
+         * The current buffer contents (if any) of the leaf or branch - always sized to contain a complete
+         * node of the form being constructed.  Always non-null, except briefly during overflow.
+         */
+        Object[] buffer;
+        /**
+         * The number of keys in our buffer, whether or not we are building a leaf or branch; if we are building
+         * a branch, we will ordinarily have the same number of children as well, except temporarily when finishing
+         * the construction of the node.
+         */
+        int count;
+
+        /**
+         * either
+         *  1) an empty leftover buffer from a past usage, which can be used when we exhaust {@code buffer}; or
+         *  2) a full {@code buffer} that has been parked until we next overflow, so we can steal some back
+         *     if we finish before reaching MIN_KEYS in {@code buffer}
+         */
+        Object[] savedBuffer;
+        /**
+         * The key we overflowed on when populating savedBuffer.  If null, {@link #savedBuffer} is logically empty.
+         */
+        Object savedNextKey;
+
+        LeafOrBranchBuilder(LeafOrBranchBuilder child)
+        {
+            this.height = child == null ? 1 : 1 + child.height;
+            this.child = child;
+        }
+
+        /**
+         * Do we have enough keys in the builder to construct at least one balanced node?
+         * We could have enough to build two.
+         */
+        final boolean isSufficient()
+        {
+            return hasOverflow() || count >= MIN_KEYS;
+        }
+
+        /**
+         * Do we have an already constructed node saved, that we can propagate or redistribute?
+         * This implies we are building two nodes, since {@link #savedNextKey} would overflow {@link #savedBuffer}
+         */
+        final boolean hasOverflow()
+        {
+            return savedNextKey != null;
+        }
+
+        /**
+         * Do we have an already constructed node saved AND insufficient keys in our buffer, so
+         * that we need to share the contents of {@link #savedBuffer} with {@link #buffer} to construct
+         * our results?
+         */
+        final boolean mustRedistribute()
+        {
+            return hasOverflow() && count < MIN_KEYS;
+        }
+
+        /**
+         * Are we empty, i.e. we have no contents in either {@link #buffer} or {@link #savedBuffer}
+         */
+        final boolean isEmpty()
+        {
+            return count == 0 && savedNextKey == null;
+        }
+
+        /**
+         * Drain the contents of this builder and build up to two nodes, as necessary.
+         * If {@code unode != null} and we are building a single node that is identical to it, use {@code unode} instead.
+         * If {@code propagateTo != null} propagate any nodes we build to it.
+         * @return the last node we construct
+         */
+        abstract Object[] drainAndPropagate(Object[] unode, int usz, BranchBuilder propagateTo);
+
+        /**
+         * Drain the contents of this builder and build at most one node.
+         * Requires {@code !hasOverflow()}
+         * @return the node we construct
+         */
+        abstract Object[] drain();
+
+        /**
+         * Takes a node that would logically occur directly preceding the current buffer contents,
+         * and the key that would separate them in a parent node, and prepends their contents
+         * to the current buffer's contents.  This can be used to redistribute already-propagated
+         * contents to a parent in cases where this is convenient (i.e. when transforming)
+         *
+         * @param predecessor directly preceding node
+         * @param predecessorNextKey key that would have separated predecessor from buffer contents
+         */
+        abstract void prepend(Object[] predecessor, Object predecessorNextKey);
+
+        /**
+         * Indicates if this builder produces dense nodes, i.e. those that are populated with MAX_KEYS
+         * at every level.  Only the last two children of any branch may be non-dense, and in some cases only
+         * the last two nodes in any tier of the tree.
+         *
+         * This flag switches whether or not we maintain a buffer of sizes, or use the globally shared contents of
+         * DENSE_SIZE_MAPS.
+         */
+        abstract boolean producesOnlyDense();
+
+        /**
+         * Ensure there is a {@code branch.parent}, and return it
+         */
+        final BranchBuilder ensureParent()
+        {
+            if (parent == null)
+                parent = new BranchBuilder(this);
+            parent.touched = true;
+            return parent;
+        }
+
+        /**
+         * Mark a branch builder as utilised, so that we must clear it when resetting any {@link AbstractFastBuilder}
+         * @return {@code branch}
+         */
+        static BranchBuilder touch(BranchBuilder branch)
+        {
+            branch.touched = true;
+            return branch;
+        }
+
+        /**
+         * @return the parent builder iff we will overflow or the parent has already been touched
+         */
+        final BranchBuilder parentIfInUse()
+        {
+            // if we have enough data to propagate, ensure there is a parent and return it
+            if (hasOverflow())
+                return ensureParent();
+
+            // otherwise return it only if it's already in use
+            return parent == null || !parent.touched ? null : parent;
+        }
+
+        /**
+         * A utility method for comparing a range of two arrays
+         */
+        static boolean areIdentical(Object[] a, int aOffset, Object[] b, int bOffset, int count)
+        {
+            for (int i = 0 ; i < count ; ++i)
+            {
+                if (a[i + aOffset] != b[i + bOffset])
+                    return false;
+            }
+            return true;
+        }
+
+        /**
+         * A utility method for comparing a range of two arrays
+         */
+        static boolean areIdentical(int[] a, int aOffset, int[] b, int bOffset, int count)
+        {
+            for (int i = 0 ; i < count ; ++i)
+            {
+                if (a[i + aOffset] != b[i + bOffset])
+                    return false;
+            }
+            return true;
+        }
+    }
+
+    /**
+     * LeafBuilder for methods pertaining specifically to building a leaf in an {@link AbstractFastBuilder}.
+     * Note that {@link AbstractFastBuilder} extends this class directly, however it is convenient to maintain
+     * distinct classes in the hierarchy for clarity of behaviour and intent.
+     */
+    private static abstract class LeafBuilder extends LeafOrBranchBuilder
+    {
+        long allocated;
+
+        LeafBuilder()
+        {
+            super(null);
+            buffer = new Object[MAX_KEYS];
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer, overflowing if necessary
+         */
+        public void addKey(Object nextKey)
+        {
+            if (count == MAX_KEYS)
+                overflow(nextKey);
+            else
+                buffer[count++] = nextKey;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void addKeyNoOverflow(Object nextKey)
+        {
+            buffer[count++] = nextKey;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void maybeAddKeyNoOverflow(Object nextKey)
+        {
+            buffer[count] = nextKey;
+            count += nextKey != null ? 1 : 0;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void maybeAddKey(Object nextKey)
+        {
+            if (count == MAX_KEYS)
+            {
+                if (nextKey != null)
+                    overflow(nextKey);
+            }
+            else
+            {
+                buffer[count] = nextKey;
+                count += nextKey != null ? 1 : 0;
+            }
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}, overflowing as necessary.
+         */
+        void copy(Object[] source, int offset, int length)
+        {
+            if (count + length > MAX_KEYS)
+            {
+                int copy = MAX_KEYS - count;
+                System.arraycopy(source, offset, buffer, count, copy);
+                offset += copy;
+//              implicitly:  count = MAX_KEYS;
+                overflow(source[offset++]);
+                length -= 1 + copy;
+            }
+            System.arraycopy(source, offset, buffer, count, length);
+            count += length;
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}; the caller specifying overflow is unnecessary
+         */
+        void copyNoOverflow(Object[] source, int offset, int length)
+        {
+            System.arraycopy(source, offset, buffer, count, length);
+            count += length;
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}, overflowing as necessary.
+         * Applies {@code updateF} to the contents before insertion.
+         */
+        <Insert, Existing> void copy(Object[] source, int offset, int length, UpdateFunction<Insert, Existing> updateF)
+        {
+            if (isSimple(updateF))
+            {
+                copy(source, offset, length);
+                return;
+            }
+
+            if (count + length > MAX_KEYS)
+            {
+                int copy = MAX_KEYS - count;
+                for (int i = 0 ; i < copy ; ++i)
+                    buffer[count + i] = updateF.apply((Insert) source[offset + i]);
+                offset += copy;
+//              implicitly:  leaf().count = MAX_KEYS;
+                overflow(updateF.apply((Insert) source[offset++]));
+                length -= 1 + copy;
+            }
+            for (int i = 0 ; i < length ; ++i)
+                buffer[count + i] = updateF.apply((Insert) source[offset + i]);
+            count += length;
+        }
+
+        /**
+         * {@link #buffer} is full, and we need to make room either by populating {@link #savedBuffer},
+         * propagating its current contents, if any, to {@link #parent}
+         */
+        void overflow(Object nextKey)
+        {
+            if (hasOverflow())
+                propagateOverflow();
+
+            // precondition: count == MAX_KEYS and savedNextKey == null
+
+            Object[] newBuffer = savedBuffer;
+            if (newBuffer == null)
+                newBuffer = new Object[MAX_KEYS];
+
+            savedBuffer = buffer;
+            savedNextKey = nextKey;
+            buffer = newBuffer;
+            count = 0;
+        }
+
+        /**
+         * Redistribute the contents of {@link #savedBuffer} into {@link #buffer}, finalise {@link #savedBuffer} and flush upwards.
+         * Invoked when we are building from {@link #buffer}, have insufficient values but a complete leaf in {@link #savedBuffer}
+         * @return the size of the leaf we flushed to our parent from {@link #savedBuffer}
+         */
+        Object[] redistributeOverflowAndDrain()
+        {
+            Object[] newLeaf = redistributeAndDrain(savedBuffer, MAX_KEYS, savedNextKey);
+            savedNextKey = null;
+            return newLeaf;
+        }
+
+        /**
+         * Redistribute the contents of {@link #buffer} and an immediate predecessor into a new leaf,
+         * then construct a new predecessor with the remaining contents and propagate up to our parent
+         * Invoked when we are building from {@link #buffer}, have insufficient values but either a complete
+         * leaf in {@link #savedBuffer} or can exfiltrate one from our parent to redistribute.
+         *
+         * @return the second of the two new leaves
+         */
+        Object[] redistributeAndDrain(Object[] pred, int predSize, Object predNextKey)
+        {
+            // precondition: savedLeafCount == MAX_KEYS && leaf().count < MIN_KEYS
+            // ensure we have at least MIN_KEYS in leaf().buffer
+            // first shift leaf().buffer and steal some keys from leaf().savedBuffer and leaf().savedBufferNextKey
+            int steal = MIN_KEYS - count;
+            Object[] newLeaf = new Object[MIN_KEYS];
+            System.arraycopy(pred,  predSize - (steal - 1), newLeaf, 0,     steal - 1);
+                                                            newLeaf [steal - 1] = predNextKey;
+            System.arraycopy(buffer, 0,                     newLeaf, steal, count);
+
+            // then create a leaf out of the remainder of savedBuffer
+            int newPredecessorCount = predSize - steal;
+            Object[] newPredecessor = new Object[newPredecessorCount | 1];
+            System.arraycopy(pred, 0, newPredecessor, 0, newPredecessorCount);
+            if (allocated >= 0)
+                allocated += ObjectSizes.sizeOfReferenceArray(newPredecessorCount | 1);
+            ensureParent().addChildAndNextKey(newPredecessor, newPredecessorCount, pred[newPredecessorCount]);
+            return newLeaf;
+        }
+
+        /**
+         * Invoked to fill our {@link #buffer} to >= MIN_KEYS with data ocurring before {@link #buffer};
+         * possibly instead fills {@link #savedBuffer}
+         *
+         * @param pred directly preceding node
+         * @param predNextKey key that would have separated predecessor from buffer contents
+         */
+        void prepend(Object[] pred, Object predNextKey)
+        {
+            int predSize = sizeOfLeaf(pred);
+            int newKeys = 1 + predSize;
+            if (newKeys + count <= MAX_KEYS)
+            {
+                System.arraycopy(buffer, 0, buffer, newKeys, count);
+                System.arraycopy(pred,   0, buffer, 0,       predSize);
+                                            buffer [predSize] = predNextKey;
+                count += newKeys;
+            }
+            else
+            {
+                if (savedBuffer == null)
+                    savedBuffer = new Object[MAX_KEYS];
+                System.arraycopy(pred, 0, savedBuffer, 0, predSize);
+                if (predSize == MAX_KEYS)
+                {
+                    savedNextKey = predNextKey;
+                }
+                else
+                {
+                    int removeKeys = MAX_KEYS - predSize;
+                    count -= removeKeys;
+                                                         savedBuffer [predSize] = predNextKey;
+                    System.arraycopy(buffer, 0,          savedBuffer, predSize + 1, MAX_KEYS - newKeys);
+                                                         savedNextKey      = buffer[MAX_KEYS - newKeys];
+                    System.arraycopy(buffer, removeKeys, buffer,      0,            count);
+                }
+            }
+        }
+
+        /**
+         * Invoked when we want to add a key to the leaf buffer, but it is full
+         */
+        void propagateOverflow()
+        {
+            // propagate the leaf we have saved in altLeafBuffer
+            // precondition: savedLeafCount == MAX_KEYS
+            if (allocated >= 0)
+                allocated += ObjectSizes.sizeOfReferenceArray(MAX_KEYS);
+            ensureParent().addChildAndNextKey(savedBuffer, MAX_KEYS, savedNextKey);
+            savedBuffer = null;
+            savedNextKey = null;
+        }
+
+        /**
+         * Construct a new leaf from the contents of {@link #buffer}, unless the contents have not changed
+         * from {@code unode}, in which case return {@code unode} to avoid allocating unnecessary objects.
+         */
+        Object[] drainAndPropagate(Object[] unode, int usz, BranchBuilder propagateTo)
+        {
+            Object[] leaf;
+            int sizeOfLeaf;
+            if (mustRedistribute())
+            {
+                // we have too few items, so spread the two buffers across two new nodes
+                leaf = redistributeOverflowAndDrain();
+                sizeOfLeaf = MIN_KEYS;
+            }
+            else if (!hasOverflow() && count == usz && areIdentical(buffer, 0, unode, 0, usz))
+            {
+                // we have exactly the same contents as the original node, so reuse it
+                leaf = unode;
+                sizeOfLeaf = usz;
+            }
+            else
+            {
+                // we have maybe one saved full buffer, and one buffer with sufficient contents to copy
+                if (hasOverflow())
+                    propagateOverflow();
+
+                sizeOfLeaf = count;
+                leaf = drain();
+                if (allocated >= 0 && sizeOfLeaf > 0)
+                    allocated += ObjectSizes.sizeOfReferenceArray(sizeOfLeaf | 1) - (unode == null ? 0 : ObjectSizes.sizeOfArray(unode));
+            }
+
+            count = 0;
+            if (propagateTo != null)
+                propagateTo.addChild(leaf, sizeOfLeaf);
+            return leaf;
+        }
+
+        /**
+         * Construct a new leaf from the contents of {@code leaf().buffer}, unless the contents have not changed
+         * from {@code unode}, in which case return {@code unode} to avoid allocating unnecessary objects.
+         */
+        Object[] drain()
+        {
+            assert !hasOverflow();
+            if (count == 0)
+                return empty();
+
+            Object[] newLeaf = new Object[count | 1];
+            System.arraycopy(buffer, 0, newLeaf, 0, count);
+            count = 0;
+            return newLeaf;
+        }
+    }
+
+    static class BranchBuilder extends LeafOrBranchBuilder
+    {
+        final LeafBuilder leaf;
+
+        /** sizes of the children in {@link #buffer}. If null, we only produce dense nodes. */
+        int[] sizes;
+        /** sizes of the children in {@link #savedBuffer} */
+        int[] savedSizes;
+        /** marker to limit unnecessary work with unused levels, esp. on reset */
+        boolean touched;
+
+        BranchBuilder(LeafOrBranchBuilder child)
+        {
+            super(child);
+            buffer = new Object[2 * (MAX_KEYS + 1)];
+            if (!child.producesOnlyDense())
+                sizes = new int[MAX_KEYS + 1];
+            this.leaf = child instanceof LeafBuilder ? (LeafBuilder) child : ((BranchBuilder) child).leaf;
+        }
+
+        /**
+         * Ensure there is room to add another key to {@code branchBuffers[branchIndex]}, and add it;
+         * invoke {@link #overflow} if necessary
+         */
+        void addKey(Object key)
+        {
+            if (count == MAX_KEYS)
+                overflow(key);
+            else
+                buffer[count++] = key;
+        }
+
+        /**
+         * To be invoked when there's a key already inserted to the buffer that requires a corresponding
+         * right-hand child, for which the buffers are sized to ensure there is always room.
+         */
+        void addChild(Object[] child, int sizeOfChild)
+        {
+            buffer[MAX_KEYS + count] = child;
+            recordSizeOfChild(sizeOfChild);
+        }
+
+        void recordSizeOfChild(int sizeOfChild)
+        {
+            if (sizes != null)
+                sizes[count] = sizeOfChild;
+        }
+
+        /**
+         * See {@link BranchBuilder#addChild(Object[], int)}
+         */
+        void addChild(Object[] child)
+        {
+            addChild(child, sizes == null ? 0 : size(child));
+        }
+
+        /**
+         * Insert a new child into a parent branch, when triggered by {@code overflowLeaf} or {@code overflowBranch}
+         */
+        void addChildAndNextKey(Object[] newChild, int newChildSize, Object nextKey)
+        {
+            // we should always have room for a child to the right of any key we have previously inserted
+            buffer[MAX_KEYS + count] = newChild;
+            recordSizeOfChild(newChildSize);
+            // but there may not be room for another key
+            addKey(nextKey);
+        }
+
+        /**
+         * Invoked when we want to add a key to the leaf buffer, but it is full
+         */
+        void propagateOverflow()
+        {
+            // propagate the leaf we have saved in leaf().savedBuffer
+            if (leaf.allocated >= 0)
+                leaf.allocated += ObjectSizes.sizeOfReferenceArray(2 * (1 + MAX_KEYS));
+            int size = setOverflowSizeMap(savedBuffer, MAX_KEYS);
+            ensureParent().addChildAndNextKey(savedBuffer, size, savedNextKey);
+            savedBuffer = null;
+            savedNextKey = null;
+        }
+
+        /**
+         * Invoked when a branch already contains {@code MAX_KEYS}, and another child is ready to be added.
+         * Creates a new neighbouring node containing MIN_KEYS items, shifting back the remaining MIN_KEYS+1
+         * items to the start of the buffer(s).
+         */
+        void overflow(Object nextKey)
+        {
+            if (hasOverflow())
+                propagateOverflow();
+
+            Object[] restoreBuffer = savedBuffer;
+            int[] restoreSizes = savedSizes;
+
+            savedBuffer = buffer;
+            savedSizes = sizes;
+            savedNextKey = nextKey;
+
+            sizes = restoreSizes == null && savedSizes != null ? new int[MAX_KEYS + 1] : restoreSizes;
+            buffer = restoreBuffer == null ? new Object[2 * (MAX_KEYS + 1)] : restoreBuffer;
+            count = 0;
+        }
+
+        /**
+         * Redistribute the contents of branch.savedBuffer into branch.buffer, finalise savedBuffer and flush upwards.
+         * Invoked when we are building from branch, have insufficient values but a complete branch in savedBuffer.
+         * @return the size of the branch we flushed to our parent from savedBuffer
+         */
+        Object[] redistributeOverflowAndDrain()
+        {
+            // now ensure we have at least MIN_KEYS in buffer
+            // both buffer and savedBuffer should be balanced, so that we have count+1 and MAX_KEYS+1 children respectively
+            // we need to utilise savedNextKey, so  we want to take {@code steal-1} keys from savedBuffer, {@code steal) children
+            // and the dangling key we use in place of savedNextKey for our parent key.
+            int steal = MIN_KEYS - count;
+            Object[] newBranch = new Object[2 * (MIN_KEYS + 1)];
+            System.arraycopy(savedBuffer,     MAX_KEYS - (steal - 1), newBranch, 0,         steal - 1);
+            newBranch[steal - 1] = savedNextKey;
+            System.arraycopy(buffer,          0,                      newBranch, steal,     count);
+            System.arraycopy(savedBuffer, 2 * MAX_KEYS + 1 - steal,   newBranch, MIN_KEYS,  steal);
+            System.arraycopy(buffer,          MAX_KEYS,               newBranch, MIN_KEYS + steal, count + 1);
+            setRedistributedSizeMap(newBranch, steal);
+
+            // then create a branch out of the remainder of savedBuffer
+            int savedBranchCount = MAX_KEYS - steal;
+            Object[] savedBranch = new Object[2 * (savedBranchCount + 1)];
+            System.arraycopy(savedBuffer, 0,        savedBranch, 0,                savedBranchCount);
+            System.arraycopy(savedBuffer, MAX_KEYS, savedBranch, savedBranchCount, savedBranchCount + 1);
+            int savedBranchSize = setOverflowSizeMap(savedBranch, savedBranchCount);
+            if (leaf.allocated >= 0)
+                leaf.allocated += ObjectSizes.sizeOfReferenceArray(2 * (1 + savedBranchCount));
+            ensureParent().addChildAndNextKey(savedBranch, savedBranchSize, savedBuffer[savedBranchCount]);
+            savedNextKey = null;
+
+            return newBranch;
+        }
+
+        /**
+         * See {@link LeafOrBranchBuilder#prepend(Object[], Object)}
+         */
+        void prepend(Object[] pred, Object predNextKey)
+        {
+            // assumes sizes != null, since only makes sense to use this method in that context
+
+            int predKeys = shallowSizeOfBranch(pred);
+            int[] sizeMap = (int[]) pred[2 * predKeys + 1];
+            int newKeys = 1 + predKeys;
+            if (newKeys + count <= MAX_KEYS)
+            {
+                System.arraycopy(buffer,    0,        buffer, newKeys,            count);
+                System.arraycopy(sizes,     0,        sizes,  newKeys,            count + 1);
+                System.arraycopy(buffer,    MAX_KEYS, buffer, MAX_KEYS + newKeys, count + 1);
+
+                System.arraycopy(pred,      0,        buffer, 0,                  predKeys);
+                buffer[predKeys] = predNextKey;
+                System.arraycopy(pred,      predKeys, buffer, MAX_KEYS,           predKeys + 1);
+                copySizeMapToSizes(sizeMap, 0,        sizes,  0,                  predKeys + 1);
+                count += newKeys;
+            }
+            else
+            {
+                if (savedBuffer == null)
+                {
+                    savedBuffer = new Object[2 * (1 + MAX_KEYS)];
+                    savedSizes = new int[1 + MAX_KEYS];
+                }
+
+                System.arraycopy(  pred,    0,        savedBuffer, 0,        predKeys);
+                System.arraycopy(  pred,    predKeys, savedBuffer, MAX_KEYS, predKeys + 1);
+                copySizeMapToSizes(sizeMap, 0,        savedSizes,  0,        predKeys + 1);
+                if (newKeys == MAX_KEYS + 1)
+                {
+                    savedNextKey = predNextKey;
+                }
+                else
+                {
+                    int removeKeys = (1 + MAX_KEYS - newKeys);
+                    int remainingKeys = count - removeKeys;
+
+                    savedBuffer [predKeys] = predNextKey;
+                    System.arraycopy(buffer, 0,                     savedBuffer, newKeys,            MAX_KEYS     - newKeys);
+                    savedNextKey =            buffer[MAX_KEYS     - newKeys];
+                    System.arraycopy(sizes,  0,                     savedSizes,  newKeys,            MAX_KEYS + 1 - newKeys);
+                    System.arraycopy(buffer, MAX_KEYS,              savedBuffer, MAX_KEYS + newKeys, MAX_KEYS + 1 - newKeys);
+                    System.arraycopy(buffer, removeKeys,            buffer,      0,                  remainingKeys);
+                    System.arraycopy(buffer, MAX_KEYS + removeKeys, buffer,      MAX_KEYS,           remainingKeys + 1);
+                    System.arraycopy(sizes,  removeKeys,            sizes,       0,                  remainingKeys + 1);
+                    count = remainingKeys;
+                }
+            }
+        }
+
+        boolean producesOnlyDense()
+        {
+            return sizes == null;
+        }
+
+        /**
+         * Construct a new branch from the contents of {@code branchBuffers[branchIndex]}, unless the contents have
+         * not changed from {@code unode}, in which case return {@code unode} to avoid allocating unnecessary objects.
+         */
+        Object[] drainAndPropagate(Object[] unode, int usz, BranchBuilder propagateTo)
+        {
+            int sizeOfBranch;
+            Object[] branch;
+            if (mustRedistribute())
+            {
+                branch = redistributeOverflowAndDrain();
+                sizeOfBranch = sizeOfBranch(branch);
+            }
+            else if (!hasOverflow() && usz == count
+                     && areIdentical(buffer, 0,        unode, 0,   usz)
+                     && areIdentical(buffer, MAX_KEYS, unode, usz, usz + 1))
+            {
+                branch = unode;
+                sizeOfBranch = sizeOfBranch(branch);
+            }
+            else
+            {
+                if (hasOverflow())
+                    propagateOverflow();
+
+                branch = new Object[2 * (count + 1)];
+                System.arraycopy(buffer, 0,        branch, 0,     count);
+                System.arraycopy(buffer, MAX_KEYS, branch, count, count + 1);
+                sizeOfBranch = setDrainSizeMap(unode, usz, branch, count);
+            }
+
+            count = 0;
+            if (propagateTo != null)
+                propagateTo.addChild(branch, sizeOfBranch);
+
+            return branch;
+        }
+
+        /**
+         * Construct a new branch from the contents of {@code branchBuffers[branchIndex]}, unless the contents have
+         * not changed from {@code unode}, in which case return {@code unode} to avoid allocating unnecessary objects.
+         */
+        Object[] drain()
+        {
+            assert !hasOverflow();
+            int keys = count;
+            count = 0;
+
+            Object[] branch = new Object[2 * (keys + 1)];
+            if (keys == MAX_KEYS)
+            {
+                Object[] tmp = buffer;
+                buffer = branch;
+                branch = tmp;
+            }
+            else
+            {
+                System.arraycopy(buffer, 0,        branch, 0,    keys);
+                System.arraycopy(buffer, MAX_KEYS, branch, keys, keys + 1);
+            }
+            setDrainSizeMap(null, -1, branch, keys);
+            return branch;
+        }
+
+        /**
+         * Compute (or fetch from cache) and set the sizeMap in {@code branch}, knowing that it
+         * was constructed from for the contents of {@code buffer}.
+         *
+         * For {@link FastBuilder} these are mostly the same, so they are fetched from a global cache and
+         * resized accordingly, but for {@link AbstractUpdater} we maintain a buffer of sizes.
+         */
+        int setDrainSizeMap(Object[] original, int keysInOriginal, Object[] branch, int keysInBranch)
+        {
+            if (producesOnlyDense())
+                return setImperfectSizeMap(branch, keysInBranch);
+
+            // first convert our buffer contents of sizes to represent a sizeMap
+            int size = sizesToSizeMap(this.sizes, keysInBranch + 1);
+            // then attempt to reuse the sizeMap from the original node, by comparing the buffer's contents with it
+            int[] sizeMap;
+            if (keysInOriginal != keysInBranch || !areIdentical(sizeMap = sizeMap(original), 0, this.sizes, 0, keysInBranch + 1))
+            {
+                // if we cannot, then we either take the buffer wholesale and replace its buffer, or copy a prefix
+                sizeMap = this.sizes;
+                if (keysInBranch < MAX_KEYS) sizeMap = Arrays.copyOf(sizeMap, keysInBranch + 1);
+                else this.sizes = new int[MAX_KEYS + 1];
+            }
+            branch[2 * keysInBranch + 1] = sizeMap;
+            return size;
+        }
+
+        /**
+         * Compute (or fetch from cache) and set the sizeMap in {@code branch}, knowing that it
+         * was constructed from for the contents of {@code savedBuffer}.
+         *
+         * For {@link FastBuilder} these are always the same size, so they are fetched from a global cache,
+         * but for {@link AbstractUpdater} we maintain a buffer of sizes.
+         *
+         * @return the size of {@code branch}
+         */
+        int setOverflowSizeMap(Object[] branch, int keys)
+        {
+            if (producesOnlyDense())
+            {
+                int[] sizeMap = DENSE_SIZE_MAPS[height - 2];
+                if (keys < MAX_KEYS) sizeMap = Arrays.copyOf(sizeMap, keys + 1);
+                branch[2 * keys + 1] = sizeMap;
+                return keys < MAX_KEYS ? sizeMap[keys] : checkedDenseSize(height + 1);
+            }
+            else
+            {
+                int[] sizes = savedSizes;
+                if (keys < MAX_KEYS) sizes = Arrays.copyOf(sizes, keys + 1);
+                else savedSizes = null;
+                branch[2 * keys + 1] = sizes;
+                return sizesToSizeMap(sizes);
+            }
+        }
+
+        /**
+         * Compute (or fetch from cache) and set the sizeMap in {@code branch}, knowing that it
+         * was constructed from the contents of both {@code savedBuffer} and {@code buffer}
+         *
+         * For {@link FastBuilder} these are mostly the same size, so they are fetched from a global cache
+         * and only the last items updated, but for {@link AbstractUpdater} we maintain a buffer of sizes.
+         */
+        void setRedistributedSizeMap(Object[] branch, int steal)
+        {
+            if (producesOnlyDense())
+            {
+                setImperfectSizeMap(branch, MIN_KEYS);
+            }
+            else
+            {
+                int[] sizeMap = new int[MIN_KEYS + 1];
+                System.arraycopy(sizes, 0, sizeMap, steal, count + 1);
+                System.arraycopy(savedSizes, MAX_KEYS + 1 - steal, sizeMap, 0, steal);
+                branch[2 * MIN_KEYS + 1] = sizeMap;
+                sizesToSizeMap(sizeMap);
+            }
+        }
+
+        /**
+         * Like {@link #setOverflowSizeMap}, but used for building the sizeMap of a node whose
+         * last two children may have had their contents redistributed; uses the perfect size map
+         * for all but the final two children, and queries the size of the last children directly
+         */
+        private int setImperfectSizeMap(Object[] branch, int keys)
+        {
+            int[] sizeMap = Arrays.copyOf(DENSE_SIZE_MAPS[height - 2], keys + 1);
+            int size = keys == 1 ? 0 : 1 + sizeMap[keys - 2];
+            sizeMap[keys - 1] = size +=     size((Object[]) branch[2 * keys - 1]);
+            sizeMap[keys]     = size += 1 + size((Object[]) branch[2 * keys]);
+            branch[2 * keys + 1] = sizeMap;
+            return size;
+        }
+
+        /**
+         * Copy the contents of {@code unode} into {@code branchBuffers[branchIndex]},
+         * between keys {@code from} and {@code to}, overflowing as necessary.
+         * {@code from} may be {@code -1}, representing the first child only;
+         * all other indices represent the key/child pairs that follow (i.e. a key and its right-hand child).
+         */
+        void copy(Object[] unode, int usz, int offset, int length)
+        {
+            int[] uszmap = sizeMap(unode);
+            if (offset == -1)
+            {
+                // copy the first child
+                buffer[MAX_KEYS] = unode[usz];
+                sizes[0] = uszmap[0];
+                if (--length == 0)
+                    return;
+                offset = 0;
+            }
+
+            if (count + length > MAX_KEYS)
+            {
+                // we will overflow, so copy to MAX_KEYS and trigger overflow
+                int copy = MAX_KEYS - count;
+                copyNoOverflow(unode, usz, uszmap, offset, copy);
+                offset += copy;
+
+                overflow(unode[offset]);
+
+                // copy first child again to restore invariants
+                buffer[MAX_KEYS] = unode[usz + offset + 1];
+                sizes[0] = uszmap[offset + 1] - (1 + uszmap[offset]);
+
+                length -= 1 + copy;
+                ++offset;
+            }
+
+            copyNoOverflow(unode, usz, uszmap, offset, length);
+        }
+
+        /**
+         * Copy the contents of {@code unode} into {@code branchBuffers[branchIndex]},
+         * between keys {@code from} and {@code to}, with the caller declaring overflow is unnecessary.
+         * {@code from} may be {@code -1}, representing the first child only;
+         * all other indices represent the key/child pairs that follow (i.e. a key and its right-hand child).
+         */
+        private void copyNoOverflow(Object[] unode, int usz, int[] uszmap, int offset, int length)
+        {
+            if (length <= 1)
+            {
+                if (length == 0)
+                    return;
+
+                buffer[count] = unode[offset];
+                buffer[MAX_KEYS + count + 1] = unode[usz + offset + 1];
+                sizes[count + 1] = uszmap[offset + 1] - (1 + uszmap[offset]);
+                ++count;
+            }
+            else
+            {
+                System.arraycopy(unode, offset, buffer, count, length);
+                System.arraycopy(unode, usz + offset + 1, buffer, MAX_KEYS + count + 1, length);
+                copySizeMapToSizes(uszmap, offset + 1, sizes, count + 1, length);
+                count += length;
+            }
+        }
+
+        /**
+         * Copy a region of a cumulative sizeMap into an array of plain sizes
+         */
+        static void copySizeMapToSizes(int[] in, int inOffset, int[] out, int outOffset, int count)
+        {
+            assert count > 0;
+            if (inOffset == 0)
+            {
+                // we don't need to subtract anything from the first node, so just copy it so we can keep the rest of the loop simple
+                out[outOffset++] = in[inOffset++];
+                --count;
+            }
+            for (int i = 0 ; i < count ; ++i)
+                out[outOffset + i] = in[inOffset + i] - (1 + in[inOffset + i - 1]);
+        }
+    }
+
+    /**
+     * Shared parent of {@link FastBuilder} and {@link Updater}, both of which
+     * construct their trees in order without knowing the resultant size upfront.
+     *
+     * Maintains a simple stack of buffers that we provide utilities to navigate and update.
+     */
+    private static abstract class AbstractFastBuilder extends LeafBuilder
+    {
+        final boolean producesOnlyDense()
+        {
+            return getClass() == FastBuilder.class;
+        }
+
+        /**
+         * An aesthetic convenience for declaring when we are interacting with the leaf, instead of invoking {@code this} directly
+         */
+        final LeafBuilder leaf() { return this; }
+
+        /**
+         * Clear any references we might still retain, to avoid holding onto memory.
+         *
+         * While this method is not strictly  necessary, it exists to
+         * ensure the implementing classes are aware they must handle it.
+         */
+        abstract void reset();
+    }
+
+    /**
+     * A pooled builder for constructing a tree in-order, and without needing any reconciliation.
+     *
+     * Constructs whole nodes in place, so that a flush of a complete node can take its buffer entirely.
+     * Since we build trees of a predictable shape (i.e. perfectly dense) we do not construct a size map.
+     */
+    public static class FastBuilder<V> extends AbstractFastBuilder implements AutoCloseable
+    {
+        private static final TinyThreadLocalPool<FastBuilder<?>> POOL = new TinyThreadLocalPool<>();
+        private TinyThreadLocalPool.TinyPool<FastBuilder<?>> pool;
+
+        FastBuilder() { allocated = -1; } // disable allocation tracking
+
+        public void add(V value)
+        {
+            leaf().addKey(value);
+        }
+
+        public void add(Object[] from, int offset, int count)
+        {
+            leaf().copy(from, offset, count);
+        }
+
+        public Object[] build()
+        {
+            LeafOrBranchBuilder level = leaf();
+            while (true)
+            {
+                if (!level.hasOverflow())
+                    return level.drain();
+
+                BranchBuilder parent = level.ensureParent();
+                level.drainAndPropagate(null, -1, parent);
+                if (level.savedBuffer != null)
+                    Arrays.fill(level.savedBuffer, null);
+                level = parent;
+            }
+        }
+
+        public Object[] buildReverse()
+        {
+            Object[] result = build();
+            reverseInSitu(result, height(result), false);
+            return result;
+        }
+
+        @Override
+        public void close()
+        {
+            reset();
+            pool.offer(this);
+            pool = null;
+        }
+
+        @Override
+        void reset()
+        {
+            // we clear precisely to leaf().count and branch.count because, in the case of a builder,
+            // if we ever fill the buffer we will consume it entirely for the tree we are building
+            // so the last count should match the number of non-null entries
+            Arrays.fill(leaf().buffer, 0, leaf().count, null);
+            leaf().count = 0;
+            BranchBuilder branch = leaf().parent;
+            while (branch != null && branch.touched)
+            {
+                Arrays.fill(branch.buffer, 0, branch.count, null);
+                Arrays.fill(branch.buffer, MAX_KEYS, MAX_KEYS + 1 + branch.count, null);
+                branch.count = 0;
+                branch.touched = false;
+                branch = branch.parent;
+            }
+        }
+    }
+
+    private static abstract class AbstractUpdater extends AbstractFastBuilder implements AutoCloseable
+    {
+        void reset()
+        {
+            assert leaf().count == 0;
+            clearLeafBuffer(leaf().buffer);
+            if (leaf().savedBuffer != null)
+                Arrays.fill(leaf().savedBuffer, null);
+
+            BranchBuilder branch = leaf().parent;
+            while (branch != null && branch.touched)
+            {
+                assert branch.count == 0;
+                clearBranchBuffer(branch.buffer);
+                if (branch.savedBuffer != null && branch.savedBuffer[0] != null)
+                    Arrays.fill(branch.savedBuffer, null); // by definition full, if non-empty
+                branch.touched = false;
+                branch = branch.parent;
+            }
+        }
+
+        /**
+         * Clear the contents of a branch buffer, aborting once we encounter a null entry
+         * to save time on small trees
+         */
+        private void clearLeafBuffer(Object[] array)
+        {
+            if (array[0] == null)
+                return;
+            // find first null entry; loop from beginning, to amortise cost over size of working set
+            int i = 1;
+            while (i < array.length && array[i] != null) ++i;
+            Arrays.fill(array, 0, i, null);
+        }
+
+        /**
+         * Clear the contents of a branch buffer, aborting once we encounter a null entry
+         * to save time on small trees
+         */
+        private void clearBranchBuffer(Object[] array)
+        {
+            if (array[0] == null)
+                return;
+
+            // find first null entry; loop from beginning, to amortise cost over size of working set
+            int i = 1;
+            while (i < MAX_KEYS && array[i] != null) ++i;
+            Arrays.fill(array, 0, i, null);
+            Arrays.fill(array, MAX_KEYS, MAX_KEYS + i + 1, null);
+        }
+    }
+
+    /**
+     * A pooled object for modifying an existing tree with a new (typically smaller) tree.
+     *
+     * Constructs the new tree around the shape of the existing tree, as though we had performed inserts in
+     * order, copying as much of the original tree as possible.  We achieve this by simply merging leaf nodes
+     * up to the immediately following key in an ancestor, maintaining up to two complete nodes in a buffer until
+     * this happens, and flushing any nodes we produce in excess of this immediately into the parent buffer.
+     *
+     * We construct whole nodes in place, except the size map, so that a flush of a complete node can take its buffer
+     * entirely.
+     *
+     * Searches within both trees to accelerate the process of modification, instead of performing a simple
+     * iteration over the new tree.
+     */
+    private static class Updater<Compare, Existing extends Compare, Insert extends Compare> extends AbstractUpdater implements AutoCloseable
+    {
+        static final TinyThreadLocalPool<Updater> POOL = new TinyThreadLocalPool<>();
+        TinyThreadLocalPool.TinyPool<Updater> pool;
+
+        // the old tree we navigate one tier at a time, deciding if we descend or not
+        // we do not always logically point to a key, as we may point to the position
+        // before the start of the node we're on
+        final SimpleTreeNavigator update = new SimpleTreeNavigator();
+        // the new tree we navigate linearly, and are always on a key or at the end
+        final SimpleTreeIterator insert = new SimpleTreeIterator();
+        Object[] result;
+
+
+        static <Compare, Existing extends Compare, Insert extends Compare> Updater<Compare, Existing, Insert> get()
+        {
+            TinyThreadLocalPool.TinyPool<Updater> pool = POOL.get();
+            Updater<Compare, Existing, Insert> updater = pool.poll();
+            if (updater == null)
+                updater = new Updater<>();
+            updater.pool = pool;
+            return updater;
+        }
+
+        /**
+         * Precondition: {@code update} should not be empty.
+         *
+         * Inserts {@code insert} into {@code update}, after applying {@code updateF} to each item, or matched item pairs.
+         */
+        Object[] update(Object[] update, Object[] insert, Comparator<? super Compare> comparator, UpdateFunction<Insert, Existing> updateF)
+        {
+            this.update.init(update);
+            this.insert.init(insert);
+            this.allocated = isSimple(updateF) ? -1 : 0;
+            return update(comparator, updateF);
+        }
+
+        /**
+         * We base our operation on the shape of {@code update}, trying to steal as much of the original tree as
+         * possible for our new tree
+         */
+        private Object[] update(Comparator<? super Compare> comparator, UpdateFunction<Insert, Existing> updateF)

Review comment:
       This is much too complex, but it would be really hard to improve.
   
   I wonder if there really is any benefit in tracking the `insert` tree rather that simply iterating over its content, as we never take whole nodes from it. This change could significantly simplify the method.

##########
File path: src/java/org/apache/cassandra/utils/btree/BTree.java
##########
@@ -1448,4 +1920,2355 @@ private static boolean isStopSentinel(long v)
     {
         return accumulate(btree, accumulator, arg, null, null, initialValue);
     }
+
+    /**
+     * Calculate the minimum height needed for this size of tree
+     *
+     * @param size the tree size
+     * @return the minimum height needed for this size of tree
+     */
+    private static int minHeight(int size)
+    {
+        return heightAtSize2n(size, BRANCH_SHIFT);
+    }
+
+    private static int heightAtSize2n(int size, int branchShift)
+    {
+        //    branch factor                 =  1 <<  branchShift
+        // => full size at height            = (1 << (branchShift * height)) - 1
+        // => full size at height + 1        =  1 << (branchShift * height)
+        // => shift(full size at height + 1) = branchShift * height
+        // => shift(full size at height + 1) / branchShift = height
+        int indexOfHighestBit = 63 - Long.numberOfLeadingZeros(size + 1);
+        indexOfHighestBit += (1L << indexOfHighestBit) == (size + 1) ? 0 : 1;
+        return (branchShift - 1 + indexOfHighestBit) / branchShift;
+    }
+
+    private static int[][] buildBalancedSizeMaps(int branchShift)
+    {
+        int count = (32 / branchShift) - 1;
+        int childCount = 1 << branchShift;
+        int[][] sizeMaps = new int[count][childCount];
+        for (int height = 0 ; height < count ; ++height)
+        {
+            int childSize = treeSize2n(height + 1, branchShift);
+            int size = 0;
+            int[] sizeMap = sizeMaps[height];
+            for (int i = 0 ; i < childCount ; ++i)
+            {
+                sizeMap[i] = size += childSize;
+                size += 1;
+            }
+        }
+        return sizeMaps;
+    }
+
+    // simply utility to reverse the contents of array[from..to)
+    private static void reverse(Object[] array, int from, int to)
+    {
+        int mid = (from + to) / 2;
+        for (int i = from ; i < mid ; i++)
+        {
+            int j = to - (1 + i - from);
+            Object tmp = array[i];
+            array[i] = array[j];
+            array[j] = tmp;
+        }
+    }
+
+    // simply utility to reverse the contents of array[from..to)
+    private static void reverse(int[] array, int from, int to)
+    {
+        int mid = (from + to) / 2;
+        for (int i = from ; i < mid ; i++)
+        {
+            int j = to - (1 + i - from);
+            int tmp = array[i];
+            array[i] = array[j];
+            array[j] = tmp;
+        }
+    }
+
+    /**
+     * Mutate an array of child sizes into a cumulative sizeMap, returning the total size
+     */
+    private static int sizesToSizeMap(int[] sizeMap)
+    {
+        int total = sizeMap[0];
+        for (int i = 1 ; i < sizeMap.length ; ++i)
+            sizeMap[i] = total += 1 + sizeMap[i];
+        return total;
+    }
+
+    private static int sizesToSizeMap(int[] sizes, int count)
+    {
+        int total = sizes[0];
+        for (int i = 1 ; i < count ; ++i)
+            sizes[i] = total += 1 + sizes[i];
+        return total;
+    }
+
+    /**
+     * Mutate an array of child sizes into a cumulative sizeMap, returning the total size
+     */
+    private static void sizeMapToSizes(int[] sizeMap)
+    {
+        for (int i = sizeMap.length ; i > 1 ; --i)
+            sizeMap[i] -= 1 + sizeMap[i - 1];
+    }
+
+    /**
+     * A simple utility method to handle a null upper bound that we treat as infinity
+     */
+    private static <Compare> int compareWithMaybeInfinity(Comparator<? super Compare> comparator, Compare key, Compare ub)
+    {
+        if (ub == null)
+            return -1;
+        return comparator.compare(key, ub);
+    }
+
+    /**
+     * Perform {@link #exponentialSearch} on {@code in[from..to)}, treating a {@code find} of {@code null} as infinity.
+     */
+    static <Compare> int exponentialSearchForMaybeInfinity(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare find)
+    {
+        if (find == null)
+            return -(1 + to);
+        return exponentialSearch(comparator, in, from, to, find);
+    }
+
+    /**
+     * Equivalent to {@link Arrays#binarySearch}, only more efficient algorithmically for linear merges.
+     * Binary search has worst case complexity {@code O(n.lg n)} for a linear merge, whereas exponential search
+     * has a worst case of {@code O(n)}. However compared to a simple linear merge, the best case for exponential
+     * search is {@code O(lg(n))} instead of {@code O(n)}.
+     */
+    private static <Compare> int exponentialSearch(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare find)
+    {
+        int step = 0;
+        while (from + step < to)
+        {
+            int i = from + step;
+            int c = comparator.compare(find, (Compare) in[i]);
+            if (c < 0)
+            {
+                to = i;
+                break;
+            }
+            if (c == 0)
+                return i;
+            from = i + 1;
+            step = step * 2 + 1; // jump in perfect binary search increments
+        }
+        return Arrays.binarySearch((Compare[]) in, from, to, find, comparator);
+    }
+
+    /**
+     * Perform {@link #exponentialSearch} on {@code in[from..to)}; if the value falls outside of the range of these
+     * elements, test against {@code ub} as though it occurred at position {@code to}
+     *
+     * @return same as {@link Arrays#binarySearch} if {@code find} occurs in the range {@code [in[from]..in[to])};
+     * otherwise the insertion position {@code -(1+to)} if {@code find} is less than {@code ub}, and {@code -(2+t)}
+     * if it is greater than or equal to.
+     *
+     * {@code ub} may be {@code null}, representing infinity.
+     */
+    static <Compare> int exponentialSearchWithUpperBound(Comparator<? super Compare> comparator, Object[] in, int from, int to, Compare ub, Compare find)
+    {
+        int step = 0;
+        while (true)
+        {
+            int i = from + step;
+            if (i >= to)
+            {
+                int c = compareWithMaybeInfinity(comparator, find, ub);
+                if (c >= 0)
+                    return -(2 + to);
+                break;
+            }
+            int c = comparator.compare(find, (Compare) in[i]);
+            if (c < 0)
+            {
+                to = i;
+                break;
+            }
+            if (c == 0)
+                return i;
+            from = i + 1;
+            step = step * 2 + 1; // jump in perfect binary search increments
+        }
+        return Arrays.binarySearch((Compare[]) in, from, to, find, comparator);
+    }
+    
+    /**
+     * Compute the size-in-bytes of full trees of cardinality {@code branchFactor^height - 1}
+     */
+    private static long[] sizeOnHeapOfPerfectTrees(int branchShift)
+    {
+        long[] result = new long[heightAtSize2n(Integer.MAX_VALUE, branchShift)];
+        int branchFactor = 1 << branchShift;
+        result[0] = branchFactor - 1;
+        for (int i = 1 ; i < result.length ; ++i)
+            result[i] = sizeOnHeapOfPerfectTree(i + 1, branchFactor);
+        return result;
+    }
+
+    /**
+     * Compute the size-in-bytes of a full tree of cardinality {@code branchFactor^height - 1}
+     * TODO: test
+     */
+    private static long sizeOnHeapOfPerfectTree(int height, int branchShift)
+    {
+        int branchFactor = 1 << branchShift;
+        long branchSize = ObjectSizes.sizeOfReferenceArray(branchFactor * 2);
+        int branchCount = height == 2 ? 1 : 2 + treeSize2n(height - 2, branchShift);
+        long leafSize = ObjectSizes.sizeOfReferenceArray((branchFactor - 1)|1);
+        int leafCount = 1 + treeSize2n(height - 1, branchShift);
+        return (branchSize * branchCount) + (leafSize * leafCount);
+    }
+
+    /**
+     * @return the actual height of {@code tree}
+     */
+    public static int height(Object[] tree)
+    {
+        if (isLeaf(tree))
+            return 1;
+
+        int height = 1;
+        while (!isLeaf(tree))
+        {
+            height++;
+            tree = (Object[]) tree[shallowSizeOfBranch(tree)];
+        }
+        return height;
+    }
+
+    /**
+     * @return the minimum representable size at {@code height}, for an internal (non-root) node.
+     */
+    private static int sparseSize(int height)
+    {
+        return treeSize2n(height, BRANCH_SHIFT - 1);
+    }
+
+    /**
+     * @return the maximum representable size at {@code height}.
+     */
+    private static int denseSize(int height)
+    {
+        return treeSize2n(height, BRANCH_SHIFT);
+    }
+
+    /**
+     * @return the maximum representable size at {@code height}.
+     */
+    private static int checkedDenseSize(int height)
+    {
+        assert height * BRANCH_SHIFT < 32;
+        return denseSize(height);
+    }
+
+    /**
+     * Computes the number of nodes in a full tree of height {@code height}
+     * and with {@code 2^branchShift} branch factor.
+     * i.e. computes {@code (2^branchShift)^height - 1}
+     */
+    private static int treeSize2n(int height, int branchShift)
+    {
+        return (1 << (branchShift * height)) - 1;
+    }
+
+    // TODO: test
+    private static int maxRootHeight(int size)
+    {
+        if (size <= BRANCH_FACTOR)
+            return 1;
+        return 1 + heightAtSize2n((size - 1) / 2, BRANCH_SHIFT - 1);
+    }
+    
+    private static int sizeOfBranch(Object[] branch)
+    {
+        int length = branch.length;
+        // length - 1 == getChildEnd == getPositionOfSizeMap
+        // (length / 2) - 1 == getChildCount - 1 == position of full tree size
+        // hard code this, as will be used often;
+        return ((int[]) branch[length - 1])[(length / 2) - 1];
+    }
+
+    /**
+     * Checks if the UpdateFunction is an instance of {@code UpdateFunction.Simple}.
+     */
+    private static boolean isSimple(UpdateFunction<?, ?> updateF)
+    {
+        return updateF instanceof UpdateFunction.Simple;
+    }
+
+    /**
+     * @return the size map for the branch node
+     */
+    static int[] sizeMap(Object[] branch)
+    {
+        return (int[]) branch[branch.length - 1];
+    }
+
+    public static long sizeOnHeapOf(Object[] tree)
+    {
+        long size = ObjectSizes.sizeOfArray(tree);
+        if (isLeaf(tree))
+            return size;
+        for (int i = childOffset(tree); i < childEndOffset(tree) ; i++)
+            size += sizeOnHeapOf((Object[]) tree[i]);
+        size += ObjectSizes.sizeOfArray(sizeMap(tree)); // may overcount, since we share size maps
+        return size;
+    }
+
+    // Arbitrary boundaries
+    private static Object POSITIVE_INFINITY = new Object();
+    private static Object NEGATIVE_INFINITY = new Object();
+
+    /** simple static wrapper to calls to cmp.compare() which checks if either a or b are Special (i.e. represent an infinity) */
+    private static <V> int compareWellFormed(Comparator<V> cmp, Object a, Object b)
+    {
+        if (a == b)
+            return 0;
+        if (a == NEGATIVE_INFINITY | b == POSITIVE_INFINITY)
+            return -1;
+        if (b == NEGATIVE_INFINITY | a == POSITIVE_INFINITY)
+            return 1;
+        return cmp.compare((V) a, (V) b);
+    }
+
+    public static boolean isWellFormed(Object[] btree, Comparator<?> cmp)
+    {
+        return isWellFormed(cmp, btree, true, NEGATIVE_INFINITY, POSITIVE_INFINITY);
+    }
+
+    private static boolean isWellFormed(Comparator<?> cmp, Object[] node, boolean isRoot, Object min, Object max)
+    {
+        if (isEmpty(node))
+            return true;
+
+        if (cmp != null && !isNodeWellFormed(cmp, node, min, max))
+            return false;
+
+        int keyCount = shallowSize(node);
+        if (keyCount < 1)
+            return false;
+        if (!isRoot && keyCount < BRANCH_FACTOR/2 - 1)
+            return false;
+        if (keyCount >= BRANCH_FACTOR)
+            return false;
+
+        if (isLeaf(node))
+            return true;
+
+        int[] sizeMap = sizeMap(node);
+        int type = 0;
+        int size = 0;
+        // compare each child node with the branch element at the head of this node it corresponds with
+        for (int i = childOffset(node); i < childEndOffset(node) ; i++)
+        {
+            Object[] child = (Object[]) node[i];
+            Object localmax = i < node.length - 2 ? node[i - childOffset(node)] : max;
+            if (!isWellFormed(cmp, child, false, min, localmax))
+                return false;
+
+            type |= isLeaf(child) ? 1 : 2;
+            min = localmax;
+            size += size(child);
+            if (sizeMap[i - childOffset(node)] != size)
+                return false;
+            size += 1;
+        }
+
+        if (type >= 3)
+            return false;
+        return true; // either all leaves or all branches but not a mix
+    }
+
+    private static boolean isNodeWellFormed(Comparator<?> cmp, Object[] node, Object min, Object max)
+    {
+        Object previous = min;
+        int end = shallowSize(node);
+        for (int i = 0; i < end; i++)
+        {
+            Object current = node[i];
+            if (compareWellFormed(cmp, previous, current) >= 0)
+                return false;
+
+            previous = current;
+        }
+        return compareWellFormed(cmp, previous, max) < 0;
+    }
+
+    /**
+     * Build a tree of unknown size, in order.
+     *
+     * Can be used with {@link #reverseInSitu} to build a tree in reverse.
+     */
+    public static <V> FastBuilder<V> fastBuilder()
+    {
+        TinyThreadLocalPool.TinyPool<FastBuilder<?>> pool = FastBuilder.POOL.get();
+        FastBuilder<V> builder = (FastBuilder<V>) pool.poll();
+        if (builder == null)
+            builder = new FastBuilder<>();
+        builder.pool = pool;
+        return builder;
+    }
+
+    /**
+     * Base class for AbstractFastBuilder.BranchBuilder, LeafBuilder and AbstractFastBuilder,
+     * containing shared behaviour and declaring some useful abstract methods.
+     */
+    private static abstract class LeafOrBranchBuilder
+    {
+        final int height;
+        final LeafOrBranchBuilder child;
+        BranchBuilder parent;
+
+        /**
+         * The current buffer contents (if any) of the leaf or branch - always sized to contain a complete
+         * node of the form being constructed.  Always non-null, except briefly during overflow.
+         */
+        Object[] buffer;
+        /**
+         * The number of keys in our buffer, whether or not we are building a leaf or branch; if we are building
+         * a branch, we will ordinarily have the same number of children as well, except temporarily when finishing
+         * the construction of the node.
+         */
+        int count;
+
+        /**
+         * either
+         *  1) an empty leftover buffer from a past usage, which can be used when we exhaust {@code buffer}; or
+         *  2) a full {@code buffer} that has been parked until we next overflow, so we can steal some back
+         *     if we finish before reaching MIN_KEYS in {@code buffer}
+         */
+        Object[] savedBuffer;
+        /**
+         * The key we overflowed on when populating savedBuffer.  If null, {@link #savedBuffer} is logically empty.
+         */
+        Object savedNextKey;
+
+        LeafOrBranchBuilder(LeafOrBranchBuilder child)
+        {
+            this.height = child == null ? 1 : 1 + child.height;
+            this.child = child;
+        }
+
+        /**
+         * Do we have enough keys in the builder to construct at least one balanced node?
+         * We could have enough to build two.
+         */
+        final boolean isSufficient()
+        {
+            return hasOverflow() || count >= MIN_KEYS;
+        }
+
+        /**
+         * Do we have an already constructed node saved, that we can propagate or redistribute?
+         * This implies we are building two nodes, since {@link #savedNextKey} would overflow {@link #savedBuffer}
+         */
+        final boolean hasOverflow()
+        {
+            return savedNextKey != null;
+        }
+
+        /**
+         * Do we have an already constructed node saved AND insufficient keys in our buffer, so
+         * that we need to share the contents of {@link #savedBuffer} with {@link #buffer} to construct
+         * our results?
+         */
+        final boolean mustRedistribute()
+        {
+            return hasOverflow() && count < MIN_KEYS;
+        }
+
+        /**
+         * Are we empty, i.e. we have no contents in either {@link #buffer} or {@link #savedBuffer}
+         */
+        final boolean isEmpty()
+        {
+            return count == 0 && savedNextKey == null;
+        }
+
+        /**
+         * Drain the contents of this builder and build up to two nodes, as necessary.
+         * If {@code unode != null} and we are building a single node that is identical to it, use {@code unode} instead.
+         * If {@code propagateTo != null} propagate any nodes we build to it.
+         * @return the last node we construct
+         */
+        abstract Object[] drainAndPropagate(Object[] unode, int usz, BranchBuilder propagateTo);
+
+        /**
+         * Drain the contents of this builder and build at most one node.
+         * Requires {@code !hasOverflow()}
+         * @return the node we construct
+         */
+        abstract Object[] drain();
+
+        /**
+         * Takes a node that would logically occur directly preceding the current buffer contents,
+         * and the key that would separate them in a parent node, and prepends their contents
+         * to the current buffer's contents.  This can be used to redistribute already-propagated
+         * contents to a parent in cases where this is convenient (i.e. when transforming)
+         *
+         * @param predecessor directly preceding node
+         * @param predecessorNextKey key that would have separated predecessor from buffer contents
+         */
+        abstract void prepend(Object[] predecessor, Object predecessorNextKey);
+
+        /**
+         * Indicates if this builder produces dense nodes, i.e. those that are populated with MAX_KEYS
+         * at every level.  Only the last two children of any branch may be non-dense, and in some cases only
+         * the last two nodes in any tier of the tree.
+         *
+         * This flag switches whether or not we maintain a buffer of sizes, or use the globally shared contents of
+         * DENSE_SIZE_MAPS.
+         */
+        abstract boolean producesOnlyDense();
+
+        /**
+         * Ensure there is a {@code branch.parent}, and return it
+         */
+        final BranchBuilder ensureParent()
+        {
+            if (parent == null)
+                parent = new BranchBuilder(this);
+            parent.touched = true;
+            return parent;
+        }
+
+        /**
+         * Mark a branch builder as utilised, so that we must clear it when resetting any {@link AbstractFastBuilder}
+         * @return {@code branch}
+         */
+        static BranchBuilder touch(BranchBuilder branch)
+        {
+            branch.touched = true;
+            return branch;
+        }
+
+        /**
+         * @return the parent builder iff we will overflow or the parent has already been touched
+         */
+        final BranchBuilder parentIfInUse()
+        {
+            // if we have enough data to propagate, ensure there is a parent and return it
+            if (hasOverflow())
+                return ensureParent();
+
+            // otherwise return it only if it's already in use
+            return parent == null || !parent.touched ? null : parent;
+        }
+
+        /**
+         * A utility method for comparing a range of two arrays
+         */
+        static boolean areIdentical(Object[] a, int aOffset, Object[] b, int bOffset, int count)
+        {
+            for (int i = 0 ; i < count ; ++i)
+            {
+                if (a[i + aOffset] != b[i + bOffset])
+                    return false;
+            }
+            return true;
+        }
+
+        /**
+         * A utility method for comparing a range of two arrays
+         */
+        static boolean areIdentical(int[] a, int aOffset, int[] b, int bOffset, int count)
+        {
+            for (int i = 0 ; i < count ; ++i)
+            {
+                if (a[i + aOffset] != b[i + bOffset])
+                    return false;
+            }
+            return true;
+        }
+    }
+
+    /**
+     * LeafBuilder for methods pertaining specifically to building a leaf in an {@link AbstractFastBuilder}.
+     * Note that {@link AbstractFastBuilder} extends this class directly, however it is convenient to maintain
+     * distinct classes in the hierarchy for clarity of behaviour and intent.
+     */
+    private static abstract class LeafBuilder extends LeafOrBranchBuilder
+    {
+        long allocated;
+
+        LeafBuilder()
+        {
+            super(null);
+            buffer = new Object[MAX_KEYS];
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer, overflowing if necessary
+         */
+        public void addKey(Object nextKey)
+        {
+            if (count == MAX_KEYS)
+                overflow(nextKey);
+            else
+                buffer[count++] = nextKey;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void addKeyNoOverflow(Object nextKey)
+        {
+            buffer[count++] = nextKey;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void maybeAddKeyNoOverflow(Object nextKey)
+        {
+            buffer[count] = nextKey;
+            count += nextKey != null ? 1 : 0;
+        }
+
+        /**
+         * Add {@code nextKey} to the buffer; the caller specifying overflow is unnecessary
+         */
+        public void maybeAddKey(Object nextKey)
+        {
+            if (count == MAX_KEYS)
+            {
+                if (nextKey != null)
+                    overflow(nextKey);
+            }
+            else
+            {
+                buffer[count] = nextKey;
+                count += nextKey != null ? 1 : 0;
+            }
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}, overflowing as necessary.
+         */
+        void copy(Object[] source, int offset, int length)
+        {
+            if (count + length > MAX_KEYS)
+            {
+                int copy = MAX_KEYS - count;
+                System.arraycopy(source, offset, buffer, count, copy);
+                offset += copy;
+//              implicitly:  count = MAX_KEYS;
+                overflow(source[offset++]);
+                length -= 1 + copy;
+            }
+            System.arraycopy(source, offset, buffer, count, length);
+            count += length;
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}; the caller specifying overflow is unnecessary
+         */
+        void copyNoOverflow(Object[] source, int offset, int length)
+        {
+            System.arraycopy(source, offset, buffer, count, length);
+            count += length;
+        }
+
+        /**
+         * Copy the contents of {@code source[from..to)} to {@code buffer}, overflowing as necessary.
+         * Applies {@code updateF} to the contents before insertion.
+         */
+        <Insert, Existing> void copy(Object[] source, int offset, int length, UpdateFunction<Insert, Existing> updateF)
+        {
+            if (isSimple(updateF))
+            {
+                copy(source, offset, length);
+                return;
+            }
+
+            if (count + length > MAX_KEYS)
+            {
+                int copy = MAX_KEYS - count;
+                for (int i = 0 ; i < copy ; ++i)
+                    buffer[count + i] = updateF.apply((Insert) source[offset + i]);
+                offset += copy;
+//              implicitly:  leaf().count = MAX_KEYS;
+                overflow(updateF.apply((Insert) source[offset++]));
+                length -= 1 + copy;
+            }
+            for (int i = 0 ; i < length ; ++i)
+                buffer[count + i] = updateF.apply((Insert) source[offset + i]);
+            count += length;
+        }
+
+        /**
+         * {@link #buffer} is full, and we need to make room either by populating {@link #savedBuffer},
+         * propagating its current contents, if any, to {@link #parent}
+         */
+        void overflow(Object nextKey)
+        {
+            if (hasOverflow())
+                propagateOverflow();
+
+            // precondition: count == MAX_KEYS and savedNextKey == null
+
+            Object[] newBuffer = savedBuffer;
+            if (newBuffer == null)
+                newBuffer = new Object[MAX_KEYS];
+
+            savedBuffer = buffer;
+            savedNextKey = nextKey;
+            buffer = newBuffer;
+            count = 0;
+        }
+
+        /**
+         * Redistribute the contents of {@link #savedBuffer} into {@link #buffer}, finalise {@link #savedBuffer} and flush upwards.
+         * Invoked when we are building from {@link #buffer}, have insufficient values but a complete leaf in {@link #savedBuffer}
+         * @return the size of the leaf we flushed to our parent from {@link #savedBuffer}
+         */
+        Object[] redistributeOverflowAndDrain()
+        {
+            Object[] newLeaf = redistributeAndDrain(savedBuffer, MAX_KEYS, savedNextKey);
+            savedNextKey = null;
+            return newLeaf;
+        }
+
+        /**
+         * Redistribute the contents of {@link #buffer} and an immediate predecessor into a new leaf,
+         * then construct a new predecessor with the remaining contents and propagate up to our parent
+         * Invoked when we are building from {@link #buffer}, have insufficient values but either a complete
+         * leaf in {@link #savedBuffer} or can exfiltrate one from our parent to redistribute.
+         *
+         * @return the second of the two new leaves
+         */
+        Object[] redistributeAndDrain(Object[] pred, int predSize, Object predNextKey)
+        {
+            // precondition: savedLeafCount == MAX_KEYS && leaf().count < MIN_KEYS
+            // ensure we have at least MIN_KEYS in leaf().buffer
+            // first shift leaf().buffer and steal some keys from leaf().savedBuffer and leaf().savedBufferNextKey
+            int steal = MIN_KEYS - count;
+            Object[] newLeaf = new Object[MIN_KEYS];
+            System.arraycopy(pred,  predSize - (steal - 1), newLeaf, 0,     steal - 1);
+                                                            newLeaf [steal - 1] = predNextKey;
+            System.arraycopy(buffer, 0,                     newLeaf, steal, count);
+
+            // then create a leaf out of the remainder of savedBuffer
+            int newPredecessorCount = predSize - steal;
+            Object[] newPredecessor = new Object[newPredecessorCount | 1];
+            System.arraycopy(pred, 0, newPredecessor, 0, newPredecessorCount);
+            if (allocated >= 0)
+                allocated += ObjectSizes.sizeOfReferenceArray(newPredecessorCount | 1);
+            ensureParent().addChildAndNextKey(newPredecessor, newPredecessorCount, pred[newPredecessorCount]);
+            return newLeaf;
+        }
+
+        /**
+         * Invoked to fill our {@link #buffer} to >= MIN_KEYS with data ocurring before {@link #buffer};
+         * possibly instead fills {@link #savedBuffer}
+         *
+         * @param pred directly preceding node
+         * @param predNextKey key that would have separated predecessor from buffer contents
+         */
+        void prepend(Object[] pred, Object predNextKey)
+        {
+            int predSize = sizeOfLeaf(pred);
+            int newKeys = 1 + predSize;
+            if (newKeys + count <= MAX_KEYS)
+            {
+                System.arraycopy(buffer, 0, buffer, newKeys, count);
+                System.arraycopy(pred,   0, buffer, 0,       predSize);
+                                            buffer [predSize] = predNextKey;
+                count += newKeys;
+            }
+            else
+            {
+                if (savedBuffer == null)
+                    savedBuffer = new Object[MAX_KEYS];
+                System.arraycopy(pred, 0, savedBuffer, 0, predSize);
+                if (predSize == MAX_KEYS)
+                {
+                    savedNextKey = predNextKey;
+                }
+                else
+                {
+                    int removeKeys = MAX_KEYS - predSize;
+                    count -= removeKeys;
+                                                         savedBuffer [predSize] = predNextKey;
+                    System.arraycopy(buffer, 0,          savedBuffer, predSize + 1, MAX_KEYS - newKeys);
+                                                         savedNextKey      = buffer[MAX_KEYS - newKeys];
+                    System.arraycopy(buffer, removeKeys, buffer,      0,            count);
+                }
+            }
+        }
+
+        /**
+         * Invoked when we want to add a key to the leaf buffer, but it is full
+         */
+        void propagateOverflow()
+        {
+            // propagate the leaf we have saved in altLeafBuffer
+            // precondition: savedLeafCount == MAX_KEYS
+            if (allocated >= 0)
+                allocated += ObjectSizes.sizeOfReferenceArray(MAX_KEYS);
+            ensureParent().addChildAndNextKey(savedBuffer, MAX_KEYS, savedNextKey);
+            savedBuffer = null;
+            savedNextKey = null;
+        }
+
+        /**
+         * Construct a new leaf from the contents of {@link #buffer}, unless the contents have not changed
+         * from {@code unode}, in which case return {@code unode} to avoid allocating unnecessary objects.
+         */
+        Object[] drainAndPropagate(Object[] unode, int usz, BranchBuilder propagateTo)
+        {
+            Object[] leaf;
+            int sizeOfLeaf;
+            if (mustRedistribute())
+            {
+                // we have too few items, so spread the two buffers across two new nodes
+                leaf = redistributeOverflowAndDrain();
+                sizeOfLeaf = MIN_KEYS;
+            }
+            else if (!hasOverflow() && count == usz && areIdentical(buffer, 0, unode, 0, usz))
+            {
+                // we have exactly the same contents as the original node, so reuse it
+                leaf = unode;
+                sizeOfLeaf = usz;
+            }
+            else
+            {
+                // we have maybe one saved full buffer, and one buffer with sufficient contents to copy
+                if (hasOverflow())
+                    propagateOverflow();
+
+                sizeOfLeaf = count;
+                leaf = drain();
+                if (allocated >= 0 && sizeOfLeaf > 0)
+                    allocated += ObjectSizes.sizeOfReferenceArray(sizeOfLeaf | 1) - (unode == null ? 0 : ObjectSizes.sizeOfArray(unode));
+            }
+
+            count = 0;
+            if (propagateTo != null)
+                propagateTo.addChild(leaf, sizeOfLeaf);
+            return leaf;
+        }
+
+        /**
+         * Construct a new leaf from the contents of {@code leaf().buffer}, unless the contents have not changed
+         * from {@code unode}, in which case return {@code unode} to avoid allocating unnecessary objects.
+         */
+        Object[] drain()
+        {
+            assert !hasOverflow();
+            if (count == 0)
+                return empty();
+
+            Object[] newLeaf = new Object[count | 1];
+            System.arraycopy(buffer, 0, newLeaf, 0, count);
+            count = 0;
+            return newLeaf;
+        }
+    }
+
+    static class BranchBuilder extends LeafOrBranchBuilder
+    {
+        final LeafBuilder leaf;
+
+        /** sizes of the children in {@link #buffer}. If null, we only produce dense nodes. */
+        int[] sizes;
+        /** sizes of the children in {@link #savedBuffer} */
+        int[] savedSizes;
+        /** marker to limit unnecessary work with unused levels, esp. on reset */
+        boolean touched;
+
+        BranchBuilder(LeafOrBranchBuilder child)
+        {
+            super(child);
+            buffer = new Object[2 * (MAX_KEYS + 1)];
+            if (!child.producesOnlyDense())
+                sizes = new int[MAX_KEYS + 1];
+            this.leaf = child instanceof LeafBuilder ? (LeafBuilder) child : ((BranchBuilder) child).leaf;
+        }
+
+        /**
+         * Ensure there is room to add another key to {@code branchBuffers[branchIndex]}, and add it;
+         * invoke {@link #overflow} if necessary
+         */
+        void addKey(Object key)
+        {
+            if (count == MAX_KEYS)
+                overflow(key);
+            else
+                buffer[count++] = key;
+        }
+
+        /**
+         * To be invoked when there's a key already inserted to the buffer that requires a corresponding
+         * right-hand child, for which the buffers are sized to ensure there is always room.
+         */
+        void addChild(Object[] child, int sizeOfChild)
+        {
+            buffer[MAX_KEYS + count] = child;
+            recordSizeOfChild(sizeOfChild);
+        }
+
+        void recordSizeOfChild(int sizeOfChild)
+        {
+            if (sizes != null)
+                sizes[count] = sizeOfChild;
+        }
+
+        /**
+         * See {@link BranchBuilder#addChild(Object[], int)}
+         */
+        void addChild(Object[] child)
+        {
+            addChild(child, sizes == null ? 0 : size(child));
+        }
+
+        /**
+         * Insert a new child into a parent branch, when triggered by {@code overflowLeaf} or {@code overflowBranch}
+         */
+        void addChildAndNextKey(Object[] newChild, int newChildSize, Object nextKey)
+        {
+            // we should always have room for a child to the right of any key we have previously inserted
+            buffer[MAX_KEYS + count] = newChild;
+            recordSizeOfChild(newChildSize);
+            // but there may not be room for another key
+            addKey(nextKey);
+        }
+
+        /**
+         * Invoked when we want to add a key to the leaf buffer, but it is full
+         */
+        void propagateOverflow()
+        {
+            // propagate the leaf we have saved in leaf().savedBuffer
+            if (leaf.allocated >= 0)
+                leaf.allocated += ObjectSizes.sizeOfReferenceArray(2 * (1 + MAX_KEYS));
+            int size = setOverflowSizeMap(savedBuffer, MAX_KEYS);
+            ensureParent().addChildAndNextKey(savedBuffer, size, savedNextKey);
+            savedBuffer = null;
+            savedNextKey = null;
+        }
+
+        /**
+         * Invoked when a branch already contains {@code MAX_KEYS}, and another child is ready to be added.
+         * Creates a new neighbouring node containing MIN_KEYS items, shifting back the remaining MIN_KEYS+1
+         * items to the start of the buffer(s).
+         */
+        void overflow(Object nextKey)
+        {
+            if (hasOverflow())
+                propagateOverflow();
+
+            Object[] restoreBuffer = savedBuffer;
+            int[] restoreSizes = savedSizes;
+
+            savedBuffer = buffer;
+            savedSizes = sizes;
+            savedNextKey = nextKey;
+
+            sizes = restoreSizes == null && savedSizes != null ? new int[MAX_KEYS + 1] : restoreSizes;
+            buffer = restoreBuffer == null ? new Object[2 * (MAX_KEYS + 1)] : restoreBuffer;
+            count = 0;
+        }
+
+        /**
+         * Redistribute the contents of branch.savedBuffer into branch.buffer, finalise savedBuffer and flush upwards.
+         * Invoked when we are building from branch, have insufficient values but a complete branch in savedBuffer.
+         * @return the size of the branch we flushed to our parent from savedBuffer
+         */
+        Object[] redistributeOverflowAndDrain()
+        {
+            // now ensure we have at least MIN_KEYS in buffer
+            // both buffer and savedBuffer should be balanced, so that we have count+1 and MAX_KEYS+1 children respectively
+            // we need to utilise savedNextKey, so  we want to take {@code steal-1} keys from savedBuffer, {@code steal) children
+            // and the dangling key we use in place of savedNextKey for our parent key.
+            int steal = MIN_KEYS - count;
+            Object[] newBranch = new Object[2 * (MIN_KEYS + 1)];
+            System.arraycopy(savedBuffer,     MAX_KEYS - (steal - 1), newBranch, 0,         steal - 1);
+            newBranch[steal - 1] = savedNextKey;
+            System.arraycopy(buffer,          0,                      newBranch, steal,     count);
+            System.arraycopy(savedBuffer, 2 * MAX_KEYS + 1 - steal,   newBranch, MIN_KEYS,  steal);
+            System.arraycopy(buffer,          MAX_KEYS,               newBranch, MIN_KEYS + steal, count + 1);
+            setRedistributedSizeMap(newBranch, steal);
+
+            // then create a branch out of the remainder of savedBuffer
+            int savedBranchCount = MAX_KEYS - steal;
+            Object[] savedBranch = new Object[2 * (savedBranchCount + 1)];
+            System.arraycopy(savedBuffer, 0,        savedBranch, 0,                savedBranchCount);
+            System.arraycopy(savedBuffer, MAX_KEYS, savedBranch, savedBranchCount, savedBranchCount + 1);
+            int savedBranchSize = setOverflowSizeMap(savedBranch, savedBranchCount);
+            if (leaf.allocated >= 0)
+                leaf.allocated += ObjectSizes.sizeOfReferenceArray(2 * (1 + savedBranchCount));
+            ensureParent().addChildAndNextKey(savedBranch, savedBranchSize, savedBuffer[savedBranchCount]);
+            savedNextKey = null;
+
+            return newBranch;
+        }
+
+        /**
+         * See {@link LeafOrBranchBuilder#prepend(Object[], Object)}
+         */
+        void prepend(Object[] pred, Object predNextKey)
+        {
+            // assumes sizes != null, since only makes sense to use this method in that context
+
+            int predKeys = shallowSizeOfBranch(pred);
+            int[] sizeMap = (int[]) pred[2 * predKeys + 1];
+            int newKeys = 1 + predKeys;
+            if (newKeys + count <= MAX_KEYS)
+            {
+                System.arraycopy(buffer,    0,        buffer, newKeys,            count);
+                System.arraycopy(sizes,     0,        sizes,  newKeys,            count + 1);
+                System.arraycopy(buffer,    MAX_KEYS, buffer, MAX_KEYS + newKeys, count + 1);
+
+                System.arraycopy(pred,      0,        buffer, 0,                  predKeys);
+                buffer[predKeys] = predNextKey;
+                System.arraycopy(pred,      predKeys, buffer, MAX_KEYS,           predKeys + 1);
+                copySizeMapToSizes(sizeMap, 0,        sizes,  0,                  predKeys + 1);
+                count += newKeys;
+            }
+            else
+            {
+                if (savedBuffer == null)
+                {
+                    savedBuffer = new Object[2 * (1 + MAX_KEYS)];
+                    savedSizes = new int[1 + MAX_KEYS];
+                }
+
+                System.arraycopy(  pred,    0,        savedBuffer, 0,        predKeys);
+                System.arraycopy(  pred,    predKeys, savedBuffer, MAX_KEYS, predKeys + 1);
+                copySizeMapToSizes(sizeMap, 0,        savedSizes,  0,        predKeys + 1);
+                if (newKeys == MAX_KEYS + 1)
+                {
+                    savedNextKey = predNextKey;
+                }
+                else
+                {
+                    int removeKeys = (1 + MAX_KEYS - newKeys);
+                    int remainingKeys = count - removeKeys;
+
+                    savedBuffer [predKeys] = predNextKey;
+                    System.arraycopy(buffer, 0,                     savedBuffer, newKeys,            MAX_KEYS     - newKeys);
+                    savedNextKey =            buffer[MAX_KEYS     - newKeys];
+                    System.arraycopy(sizes,  0,                     savedSizes,  newKeys,            MAX_KEYS + 1 - newKeys);
+                    System.arraycopy(buffer, MAX_KEYS,              savedBuffer, MAX_KEYS + newKeys, MAX_KEYS + 1 - newKeys);
+                    System.arraycopy(buffer, removeKeys,            buffer,      0,                  remainingKeys);
+                    System.arraycopy(buffer, MAX_KEYS + removeKeys, buffer,      MAX_KEYS,           remainingKeys + 1);
+                    System.arraycopy(sizes,  removeKeys,            sizes,       0,                  remainingKeys + 1);
+                    count = remainingKeys;
+                }
+            }
+        }
+
+        boolean producesOnlyDense()
+        {
+            return sizes == null;
+        }
+
+        /**
+         * Construct a new branch from the contents of {@code branchBuffers[branchIndex]}, unless the contents have
+         * not changed from {@code unode}, in which case return {@code unode} to avoid allocating unnecessary objects.
+         */
+        Object[] drainAndPropagate(Object[] unode, int usz, BranchBuilder propagateTo)
+        {
+            int sizeOfBranch;
+            Object[] branch;
+            if (mustRedistribute())
+            {
+                branch = redistributeOverflowAndDrain();
+                sizeOfBranch = sizeOfBranch(branch);
+            }
+            else if (!hasOverflow() && usz == count
+                     && areIdentical(buffer, 0,        unode, 0,   usz)
+                     && areIdentical(buffer, MAX_KEYS, unode, usz, usz + 1))
+            {
+                branch = unode;
+                sizeOfBranch = sizeOfBranch(branch);
+            }
+            else
+            {
+                if (hasOverflow())
+                    propagateOverflow();
+
+                branch = new Object[2 * (count + 1)];

Review comment:
       Why can't `count` be smaller than `MIN_KEYS` here?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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